diff --git a/BUILDING b/BUILDING index c6e23f594e1..75a72ccafc8 100644 --- a/BUILDING +++ b/BUILDING @@ -4,7 +4,7 @@ Build Instructions for Apache RocketMQ (1) Prerequisites - JDK 1.7+ is required in order to compile and run RocketMQ. + JDK 1.8+ is required in order to compile and run RocketMQ. RocketMQ utilizes Maven as a distribution management and packaging tool. Version 3.0.3 or later is required. Maven installation and configuration instructions can be found here: diff --git a/README.md b/README.md index 765f1eb2a6f..db7716e1f06 100644 --- a/README.md +++ b/README.md @@ -5,8 +5,8 @@ [![Maven Central][maven-central-image]][maven-central-url] [![Release][release-image]][release-url] [![License][license-image]][license-url] -[![Average Time to Resolve An Issue][percentage-of-issues-still-open-image]][percentage-of-issues-still-open-url] -[![Percentage of Issues Still Open][average-time-to-resolve-an-issue-image]][average-time-to-resolve-an-issue-url] +[![Average Time to Resolve An Issue][average-time-to-resolve-an-issue-image]][average-time-to-resolve-an-issue-url] +[![Percentage of Issues Still Open][percentage-of-issues-still-open-image]][percentage-of-issues-still-open-url] [![Twitter Follow][twitter-follow-image]][twitter-follow-url] **[Apache RocketMQ](https://rocketmq.apache.org) is a distributed messaging and streaming platform with low latency, high performance and reliability, trillion-level capacity and flexible scalability.** @@ -49,21 +49,21 @@ $ java -version java version "1.8.0_121" ``` -For Windows users, click [here](https://dist.apache.org/repos/dist/release/rocketmq/5.3.3/rocketmq-all-5.3.3-bin-release.zip) to download the 5.3.3 RocketMQ binary release, +For Windows users, click [here](https://dist.apache.org/repos/dist/release/rocketmq/5.3.4/rocketmq-all-5.3.4-bin-release.zip) to download the 5.3.4 RocketMQ binary release, unpack it to your local disk, such as `D:\rocketmq`. For macOS and Linux users, execute following commands: ```shell # Download release from the Apache mirror -$ wget https://dist.apache.org/repos/dist/release/rocketmq/5.3.3/rocketmq-all-5.3.3-bin-release.zip +$ wget https://dist.apache.org/repos/dist/release/rocketmq/5.3.4/rocketmq-all-5.3.4-bin-release.zip # Unpack the release -$ unzip rocketmq-all-5.3.3-bin-release.zip +$ unzip rocketmq-all-5.3.4-bin-release.zip ``` Prepare a terminal and change to the extracted `bin` directory: ```shell -$ cd rocketmq-all-5.3.3-bin-release/bin +$ cd rocketmq-all-5.3.4-bin-release/bin ``` **1) Start NameServer** @@ -98,17 +98,17 @@ The Name Server boot success... For macOS and Linux users: ```shell ### start Broker -$ nohup sh bin/mqbroker -n localhost:9876 & +$ nohup sh mqbroker -n localhost:9876 & ### check whether Broker is successfully started, eg: Broker's IP is 192.168.1.2, Broker's name is broker-a $ tail -f ~/logs/rocketmqlogs/broker.log -The broker[broker-a, 192.169.1.2:10911] boot success... +The broker[broker-a, 192.168.1.2:10911] boot success... ``` For Windows users: ```shell $ mqbroker.cmd -n localhost:9876 -The broker[broker-a, 192.169.1.2:10911] boot success... +The broker[broker-a, 192.168.1.2:10911] boot success... ``` ### Run RocketMQ in Docker @@ -239,7 +239,7 @@ services. [maven-central-image]: https://maven-badges.herokuapp.com/maven-central/org.apache.rocketmq/rocketmq-all/badge.svg [maven-central-url]: http://search.maven.org/#search%7Cga%7C1%7Corg.apache.rocketmq [release-image]: https://img.shields.io/badge/release-download-orange.svg -[release-url]: https://www.apache.org/licenses/LICENSE-2.0.html +[release-url]: https://rocketmq.apache.org/download/ [license-image]: https://img.shields.io/badge/license-Apache%202-4EB1BA.svg [license-url]: https://www.apache.org/licenses/LICENSE-2.0.html [average-time-to-resolve-an-issue-image]: http://isitmaintained.com/badge/resolution/apache/rocketmq.svg diff --git a/WORKSPACE b/WORKSPACE index da58ae67634..c3e14548be4 100644 --- a/WORKSPACE +++ b/WORKSPACE @@ -71,7 +71,7 @@ maven_install( "org.bouncycastle:bcpkix-jdk15on:1.69", "com.google.code.gson:gson:2.8.9", "com.googlecode.concurrentlinkedhashmap:concurrentlinkedhashmap-lru:1.4.2", - "org.apache.rocketmq:rocketmq-proto:2.0.4", + "org.apache.rocketmq:rocketmq-proto:2.1.0", "com.google.protobuf:protobuf-java:3.20.1", "com.google.protobuf:protobuf-java-util:3.20.1", "com.conversantmedia:disruptor:1.2.10", diff --git a/auth/src/main/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProvider.java b/auth/src/main/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProvider.java index 04e745eaaf1..93d03272712 100644 --- a/auth/src/main/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProvider.java +++ b/auth/src/main/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProvider.java @@ -46,6 +46,8 @@ public class LocalAuthenticationMetadataProvider implements AuthenticationMetada private LoadingCache userCache; + protected ThreadPoolExecutor cacheRefreshExecutor; + @Override public void initialize(AuthConfig authConfig, Supplier metadataService) { this.storage = ConfigRocksDBStorage.getStore(authConfig.getAuthConfigPath() + File.separator + "users", false); @@ -53,7 +55,7 @@ public void initialize(AuthConfig authConfig, Supplier metadataService) { throw new RuntimeException("Failed to load rocksdb for auth_user, please check whether it is occupied"); } - ThreadPoolExecutor cacheRefreshExecutor = ThreadPoolMonitor.createAndMonitor( + this.cacheRefreshExecutor = ThreadPoolMonitor.createAndMonitor( 1, 1, 1000 * 60, @@ -144,6 +146,9 @@ public void shutdown() { if (this.storage != null) { this.storage.shutdown(); } + if (this.cacheRefreshExecutor != null) { + this.cacheRefreshExecutor.shutdown(); + } } private static class UserCacheLoader implements CacheLoader { 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 5725d4feac8..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()); @@ -192,11 +200,7 @@ public List build(ChannelHandlerContext context, Re break; case RequestCode.SEND_MESSAGE: if (NamespaceUtil.isRetryTopic(fields.get(TOPIC))) { - if (StringUtils.isNotBlank(fields.get(GROUP))) { - group = Resource.ofGroup(fields.get(GROUP)); - } else { - group = Resource.ofGroup(fields.get(TOPIC)); - } + group = Resource.ofGroup(fields.get(TOPIC)); result.add(DefaultAuthorizationContext.of(subject, group, Action.SUB, sourceIp)); } else { topic = Resource.ofTopic(fields.get(TOPIC)); @@ -206,11 +210,7 @@ public List build(ChannelHandlerContext context, Re case RequestCode.SEND_MESSAGE_V2: case RequestCode.SEND_BATCH_MESSAGE: if (NamespaceUtil.isRetryTopic(fields.get(B))) { - if (StringUtils.isNotBlank(fields.get(A))) { - group = Resource.ofGroup(fields.get(A)); - } else { - group = Resource.ofGroup(fields.get(B)); - } + group = Resource.ofGroup(fields.get(B)); result.add(DefaultAuthorizationContext.of(subject, group, Action.SUB, sourceIp)); } else { topic = Resource.ofTopic(fields.get(B)); diff --git a/auth/src/main/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerImpl.java b/auth/src/main/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerImpl.java index dd6b63679d3..5a8dede18e7 100644 --- a/auth/src/main/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerImpl.java +++ b/auth/src/main/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerImpl.java @@ -176,19 +176,26 @@ public CompletableFuture deleteAcl(Subject subject, PolicyType policyType, @Override public CompletableFuture getAcl(Subject subject) { - CompletableFuture subjectFuture; - if (subject.isSubject(SubjectType.USER)) { - User user = (User) subject; - subjectFuture = this.getAuthenticationMetadataProvider().getUser(user.getUsername()); - } else { - subjectFuture = CompletableFuture.completedFuture(subject); - } - return subjectFuture.thenCompose(sub -> { - if (sub == null) { - throw new AuthorizationException("The subject is not exist."); + try { + if (subject == null) { + throw new AuthorizationException("The subject is null."); } - return this.getAuthorizationMetadataProvider().getAcl(subject); - }); + CompletableFuture subjectFuture; + if (subject.isSubject(SubjectType.USER)) { + User user = (User) subject; + subjectFuture = this.getAuthenticationMetadataProvider().getUser(user.getUsername()); + } else { + subjectFuture = CompletableFuture.completedFuture(subject); + } + return subjectFuture.thenCompose(sub -> { + if (sub == null) { + throw new AuthorizationException("The subject is not exist."); + } + return this.getAuthorizationMetadataProvider().getAcl(sub); + }); + } catch (Exception e) { + return this.handleException(e); + } } @Override diff --git a/auth/src/main/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProvider.java b/auth/src/main/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProvider.java index 6db999bee70..f6b8ecaf3db 100644 --- a/auth/src/main/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProvider.java +++ b/auth/src/main/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProvider.java @@ -51,13 +51,15 @@ public class LocalAuthorizationMetadataProvider implements AuthorizationMetadata private LoadingCache aclCache; + protected ThreadPoolExecutor cacheRefreshExecutor; + @Override public void initialize(AuthConfig authConfig, Supplier metadataService) { this.storage = ConfigRocksDBStorage.getStore(authConfig.getAuthConfigPath() + File.separator + "acls", false); if (!this.storage.start()) { throw new RuntimeException("Failed to load rocksdb for auth_acl, please check whether it is occupied."); } - ThreadPoolExecutor cacheRefreshExecutor = ThreadPoolMonitor.createAndMonitor( + this.cacheRefreshExecutor = ThreadPoolMonitor.createAndMonitor( 1, 1, 1000 * 60, @@ -172,6 +174,9 @@ public void shutdown() { if (this.storage != null) { this.storage.shutdown(); } + if (this.cacheRefreshExecutor != null) { + this.cacheRefreshExecutor.shutdown(); + } } private static class AclCacheLoader implements CacheLoader { diff --git a/auth/src/main/java/org/apache/rocketmq/auth/authorization/strategy/AbstractAuthorizationStrategy.java b/auth/src/main/java/org/apache/rocketmq/auth/authorization/strategy/AbstractAuthorizationStrategy.java index 849c3082d31..fef7969ade2 100644 --- a/auth/src/main/java/org/apache/rocketmq/auth/authorization/strategy/AbstractAuthorizationStrategy.java +++ b/auth/src/main/java/org/apache/rocketmq/auth/authorization/strategy/AbstractAuthorizationStrategy.java @@ -16,8 +16,8 @@ */ package org.apache.rocketmq.auth.authorization.strategy; -import java.util.ArrayList; -import java.util.List; +import java.util.HashSet; +import java.util.Set; import java.util.function.Supplier; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.auth.authorization.context.AuthorizationContext; @@ -30,7 +30,7 @@ public abstract class AbstractAuthorizationStrategy implements AuthorizationStrategy { protected final AuthConfig authConfig; - protected final List authorizationWhitelist = new ArrayList<>(); + protected final Set authorizationWhiteSet = new HashSet<>(); protected final AuthorizationProvider authorizationProvider; public AbstractAuthorizationStrategy(AuthConfig authConfig, Supplier metadataService) { @@ -42,7 +42,7 @@ public AbstractAuthorizationStrategy(AuthConfig authConfig, Supplier metadata if (StringUtils.isNotBlank(authConfig.getAuthorizationWhitelist())) { String[] whitelist = StringUtils.split(authConfig.getAuthorizationWhitelist(), ","); for (String rpcCode : whitelist) { - this.authorizationWhitelist.add(StringUtils.trim(rpcCode)); + this.authorizationWhiteSet.add(StringUtils.trim(rpcCode)); } } } @@ -57,7 +57,7 @@ public void doEvaluate(AuthorizationContext context) { if (this.authorizationProvider == null) { return; } - if (this.authorizationWhitelist.contains(context.getRpcCode())) { + if (this.authorizationWhiteSet.contains(context.getRpcCode())) { return; } try { diff --git a/auth/src/test/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProviderTest.java b/auth/src/test/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProviderTest.java new file mode 100644 index 00000000000..15ec8c32603 --- /dev/null +++ b/auth/src/test/java/org/apache/rocketmq/auth/authentication/provider/LocalAuthenticationMetadataProviderTest.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.auth.authentication.provider; + +import org.apache.rocketmq.auth.config.AuthConfig; +import org.apache.rocketmq.auth.helper.AuthTestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class LocalAuthenticationMetadataProviderTest { + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testShutdownReleasesCacheExecutor() throws Exception { + AuthConfig authConfig = AuthTestHelper.createDefaultConfig(); + authConfig.setAuthConfigPath(tempFolder.newFolder("auth-test").getAbsolutePath()); + + LocalAuthenticationMetadataProvider provider = new LocalAuthenticationMetadataProvider(); + // Initialize provider to create the internal cache refresh executor + provider.initialize(authConfig, () -> null); + + // After initialization, the executor should exist and not be shutdown + Assert.assertNotNull(provider.cacheRefreshExecutor); + Assert.assertFalse(provider.cacheRefreshExecutor.isShutdown()); + + // Shutdown provider should also shutdown its executor to release resources + provider.shutdown(); + + // Verify that the cache refresh executor has been shutdown + Assert.assertTrue(provider.cacheRefreshExecutor.isShutdown()); + } +} diff --git a/auth/src/test/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerTest.java b/auth/src/test/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerTest.java index b6bcfa74886..72504f1cb34 100644 --- a/auth/src/test/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerTest.java +++ b/auth/src/test/java/org/apache/rocketmq/auth/authorization/manager/AuthorizationMetadataManagerTest.java @@ -203,6 +203,21 @@ public void getAcl() { }); } + @Test + public void testGetAclWithNullSubject() { + if (MixAll.isMac()) { + return; + } + AuthorizationException authorizationException = Assert.assertThrows(AuthorizationException.class, () -> { + try { + this.authorizationMetadataManager.getAcl(null).join(); + } catch (Exception e) { + AuthTestHelper.handleException(e); + } + }); + Assert.assertEquals("The subject is null.", authorizationException.getMessage()); + } + @Test public void listAcl() { if (MixAll.isMac()) { diff --git a/auth/src/test/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProviderTest.java b/auth/src/test/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProviderTest.java new file mode 100644 index 00000000000..32771a4d80c --- /dev/null +++ b/auth/src/test/java/org/apache/rocketmq/auth/authorization/provider/LocalAuthorizationMetadataProviderTest.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.auth.authorization.provider; + +import org.apache.rocketmq.auth.config.AuthConfig; +import org.apache.rocketmq.auth.helper.AuthTestHelper; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class LocalAuthorizationMetadataProviderTest { + + @Rule + public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testShutdownReleasesCacheExecutor() throws Exception { + AuthConfig authConfig = AuthTestHelper.createDefaultConfig(); + authConfig.setAuthConfigPath(tempFolder.newFolder("auth-test").getAbsolutePath()); + + LocalAuthorizationMetadataProvider provider = new LocalAuthorizationMetadataProvider(); + // Initialize provider to create the internal cache refresh executor + provider.initialize(authConfig, () -> null); + + // After initialization, the executor should exist and not be shutdown + Assert.assertNotNull(provider.cacheRefreshExecutor); + Assert.assertFalse(provider.cacheRefreshExecutor.isShutdown()); + + // Shutdown provider should also shutdown its executor to release resources + provider.shutdown(); + + // Verify that the cache refresh executor has been shutdown + Assert.assertTrue(provider.cacheRefreshExecutor.isShutdown()); + } +} diff --git a/broker/BUILD.bazel b/broker/BUILD.bazel index fdf549d3e51..ffd2bea14c5 100644 --- a/broker/BUILD.bazel +++ b/broker/BUILD.bazel @@ -31,7 +31,6 @@ java_library( "//tieredstore", "@maven//:org_slf4j_slf4j_api", "@maven//:ch_qos_logback_logback_classic", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_github_luben_zstd_jni", "@maven//:com_google_guava_guava", @@ -83,7 +82,6 @@ java_library( "//remoting", "//store", "//tieredstore", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:org_slf4j_slf4j_api", "@maven//:com_google_guava_guava", diff --git a/broker/pom.xml b/broker/pom.xml index 17518526310..b94e596cc5c 100644 --- a/broker/pom.xml +++ b/broker/pom.xml @@ -66,10 +66,6 @@ commons-io commons-io - - com.alibaba - fastjson - org.javassist javassist 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 7b1701c61a0..c578f17aaeb 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/BrokerController.java @@ -18,6 +18,29 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; +import java.net.InetSocketAddress; +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.rocketmq.auth.authentication.factory.AuthenticationFactory; import org.apache.rocketmq.auth.authentication.manager.AuthenticationMetadataManager; import org.apache.rocketmq.auth.authorization.factory.AuthorizationFactory; @@ -50,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; @@ -58,21 +89,25 @@ import org.apache.rocketmq.broker.mqtrace.SendMessageHook; import org.apache.rocketmq.broker.offset.BroadcastOffsetManager; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; -import org.apache.rocketmq.broker.offset.ConsumerOrderInfoManager; import org.apache.rocketmq.broker.offset.LmqConsumerOffsetManager; import org.apache.rocketmq.broker.out.BrokerOuterAPI; import org.apache.rocketmq.broker.plugin.BrokerAttachedPlugin; import org.apache.rocketmq.broker.pop.PopConsumerService; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.pop.orderly.QueueLevelConsumerManager; import org.apache.rocketmq.broker.processor.AckMessageProcessor; import org.apache.rocketmq.broker.processor.AdminBrokerProcessor; import org.apache.rocketmq.broker.processor.ChangeInvisibleTimeProcessor; 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; @@ -93,11 +128,11 @@ import org.apache.rocketmq.broker.transaction.TransactionMetricsFlushService; import org.apache.rocketmq.broker.transaction.TransactionalMessageCheckService; import org.apache.rocketmq.broker.transaction.TransactionalMessageService; +import org.apache.rocketmq.broker.transaction.rocksdb.TransactionalMessageRocksDBService; import org.apache.rocketmq.broker.transaction.queue.DefaultTransactionalMessageCheckListener; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageBridge; import org.apache.rocketmq.broker.transaction.queue.TransactionalMessageServiceImpl; import org.apache.rocketmq.broker.util.HookUtils; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.BrokerIdentity; import org.apache.rocketmq.common.MixAll; @@ -155,30 +190,8 @@ import org.apache.rocketmq.store.timer.TimerCheckpoint; import org.apache.rocketmq.store.timer.TimerMessageStore; import org.apache.rocketmq.store.timer.TimerMetrics; - -import java.net.InetSocketAddress; -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Function; -import java.util.function.Supplier; -import java.util.stream.Collectors; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; public class BrokerController { protected static final Logger LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -204,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; @@ -270,6 +290,8 @@ public class BrokerController { private BrokerStats brokerStats; private InetSocketAddress storeHost; private TimerMessageStore timerMessageStore; + private TimerMessageRocksDBStore timerMessageRocksDBStore; + private TransMessageRocksDBStore transMessageRocksDBStore; private TimerCheckpoint timerCheckpoint; protected BrokerFastFailure brokerFastFailure; private Configuration configuration; @@ -278,6 +300,7 @@ public class BrokerController { protected TransactionalMessageCheckService transactionalMessageCheckService; protected TransactionalMessageService transactionalMessageService; protected AbstractTransactionalMessageCheckListener transactionalMessageCheckListener; + protected TransactionalMessageRocksDBService transactionalMessageRocksDBService; protected volatile boolean shutdown = false; protected ShutdownHook shutdownHook; private volatile boolean isScheduleServiceStart = false; @@ -371,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); @@ -382,12 +414,12 @@ 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); this.consumerFilterManager = new ConsumerFilterManager(this); - this.consumerOrderInfoManager = new ConsumerOrderInfoManager(this); + this.consumerOrderInfoManager = new QueueLevelConsumerManager(this); this.popInflightMessageCounter = new PopInflightMessageCounter(this); this.popConsumerService = brokerConfig.isPopConsumerKVServiceInit() ? new PopConsumerService(this) : null; this.clientHousekeepingService = new ClientHousekeepingService(this); @@ -461,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); } @@ -866,6 +896,14 @@ public boolean initializeMessageStore() { this.timerMessageStore = new TimerMessageStore(messageStore, messageStoreConfig, timerCheckpoint, timerMetrics, brokerStatsManager); this.timerMessageStore.registerEscapeBridgeHook(msg -> escapeBridge.putMessage(msg)); this.messageStore.setTimerMessageStore(this.timerMessageStore); + if (messageStoreConfig.isTimerRocksDBEnable()) { + this.timerMessageRocksDBStore = new TimerMessageRocksDBStore(messageStore, timerMetrics, brokerStatsManager); + this.messageStore.setTimerMessageRocksDBStore(timerMessageRocksDBStore); + } + } + if (messageStoreConfig.isTransRocksDBEnable()) { + this.transMessageRocksDBStore = new TransMessageRocksDBStore(messageStore, brokerStatsManager, new InetSocketAddress(this.getBrokerConfig().getBrokerIP1(), this.getNettyServerConfig().getListenPort())); + this.messageStore.setTransRocksDBStore(transMessageRocksDBStore); } } catch (Exception e) { result = false; @@ -905,6 +943,9 @@ public boolean recoverAndInitService() throws CloneNotSupportedException { if (messageStoreConfig.isTimerWheelEnable()) { result = result && this.timerMessageStore.load(); + if (messageStoreConfig.isTimerRocksDBEnable()) { + result = result && this.timerMessageRocksDBStore.load(); + } } //scheduleMessageService load after messageStore load success @@ -934,6 +975,8 @@ public boolean recoverAndInitService() throws CloneNotSupportedException { initialRequestPipeline(); + initLiteService(); + if (TlsSystemConfig.tlsMode != TlsMode.DISABLED) { // Register a listener to reload SslContext try { @@ -1029,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) { @@ -1061,6 +1119,10 @@ private void initialTransaction() { this.transactionMetricsFlushService = new TransactionMetricsFlushService(this); this.transactionMetricsFlushService.start(); + if (messageStoreConfig.isTransRocksDBEnable()) { + this.transactionalMessageRocksDBService = new TransactionalMessageRocksDBService(messageStore, this); + this.transactionalMessageRocksDBService.start(); + } } private void initialRpcHooks() { @@ -1091,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); @@ -1125,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 @@ -1176,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 @@ -1207,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 */ @@ -1389,6 +1476,10 @@ public PopMessageProcessor getPopMessageProcessor() { return popMessageProcessor; } + public PopLiteMessageProcessor getPopLiteMessageProcessor() { + return popLiteMessageProcessor; + } + public NotificationProcessor getNotificationProcessor() { return notificationProcessor; } @@ -1401,6 +1492,14 @@ public void setTimerMessageStore(TimerMessageStore timerMessageStore) { this.timerMessageStore = timerMessageStore; } + public TimerMessageRocksDBStore getTimerMessageRocksDBStore() { + return timerMessageRocksDBStore; + } + + public void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore) { + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + } + public AckMessageProcessor getAckMessageProcessor() { return ackMessageProcessor; } @@ -1409,6 +1508,14 @@ public ChangeInvisibleTimeProcessor getChangeInvisibleTimeProcessor() { return changeInvisibleTimeProcessor; } + public LiteSubscriptionRegistry getLiteSubscriptionRegistry() { + return liteSubscriptionRegistry; + } + + public AbstractLiteLifecycleManager getLiteLifecycleManager() { + return liteLifecycleManager; + } + protected void shutdownBasicService() { shutdown = true; @@ -1442,14 +1549,17 @@ protected void shutdownBasicService() { this.pullRequestHoldService.shutdown(); } - if (this.popConsumerService != null) { - this.popConsumerService.shutdown(); - } - if (this.popMessageProcessor.getPopLongPollingService() != null) { 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(); } @@ -1478,6 +1588,10 @@ protected void shutdownBasicService() { this.transactionMetricsFlushService.shutdown(); } + if (this.transactionalMessageRocksDBService != null) { + this.transactionalMessageRocksDBService.shutdown(); + } + if (this.notificationProcessor != null) { this.notificationProcessor.getPopLongPollingService().shutdown(); } @@ -1493,6 +1607,15 @@ protected void shutdownBasicService() { if (this.timerMessageStore != null) { this.timerMessageStore.shutdown(); } + + if (this.timerMessageRocksDBStore != null) { + this.timerMessageRocksDBStore.shutdown(); + } + + if (this.transMessageRocksDBStore != null) { + this.transMessageRocksDBStore.shutdown(); + } + if (this.fileWatchService != null) { this.fileWatchService.shutdown(); } @@ -1567,7 +1690,7 @@ protected void shutdownBasicService() { if (this.transactionalMessageCheckService != null) { this.transactionalMessageCheckService.shutdown(false); } - + if (this.loadBalanceExecutor != null) { this.loadBalanceExecutor.shutdown(); } @@ -1600,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); @@ -1641,6 +1776,10 @@ protected void shutdownBasicService() { } } + if (this.popConsumerService != null) { + this.popConsumerService.shutdown(); + } + if (this.messageStore != null) { this.messageStore.shutdown(); } @@ -1694,6 +1833,10 @@ protected void startBasicService() throws Exception { this.timerMessageStore.start(); } + if (this.timerMessageRocksDBStore != null && this.messageStoreConfig.isTimerRocksDBEnable()) { + this.timerMessageRocksDBStore.start(); + } + if (this.replicasManager != null) { this.replicasManager.start(); } @@ -1732,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(); @@ -1793,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 { @@ -1814,9 +1976,9 @@ public void start() throws Exception { this.registerBrokerAll(true, false, true); } - scheduledFutures.add(this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + scheduledFutures.add(this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { if (System.currentTimeMillis() < shouldStartTime) { BrokerController.LOG.info("Register to namesrv after {}", shouldStartTime); @@ -1836,9 +1998,9 @@ public void run0() { if (this.brokerConfig.isEnableSlaveActingMaster()) { scheduleSendHeartbeat(); - scheduledFutures.add(this.syncBrokerMemberGroupExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + scheduledFutures.add(this.syncBrokerMemberGroupExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { BrokerController.this.syncBrokerMemberGroup(); } catch (Throwable e) { @@ -1869,9 +2031,9 @@ public void run() { } protected void scheduleSendHeartbeat() { - scheduledFutures.add(this.brokerHeartbeatExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + scheduledFutures.add(this.brokerHeartbeatExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { if (isIsolated) { return; } @@ -2654,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/DefaultConsumerIdsChangeListener.java b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java index e0461769568..2946e03e1ae 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/client/DefaultConsumerIdsChangeListener.java @@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.rocketmq.broker.BrokerController; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.utils.ThreadUtils; import org.apache.rocketmq.logging.org.slf4j.Logger; @@ -48,9 +47,9 @@ public class DefaultConsumerIdsChangeListener implements ConsumerIdsChangeListen public DefaultConsumerIdsChangeListener(BrokerController brokerController) { this.brokerController = brokerController; - scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(brokerController.getBrokerConfig()) { + scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { notifyConsumerChange(); } catch (Exception e) { 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/RocksDBConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConfigManager.java index 4ebdce13157..e68cd20a9d2 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConfigManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBConfigManager.java @@ -16,10 +16,7 @@ */ package org.apache.rocketmq.broker.config.v1; -import com.alibaba.fastjson.JSON; -import java.nio.charset.Charset; -import java.nio.charset.StandardCharsets; -import java.util.function.BiConsumer; +import com.alibaba.fastjson2.JSON; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.config.ConfigRocksDBStorage; import org.apache.rocketmq.common.constant.LoggerName; @@ -33,12 +30,15 @@ import org.rocksdb.Statistics; import org.rocksdb.WriteBatch; +import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; +import java.util.function.BiConsumer; + public class RocksDBConfigManager { protected static final Logger BROKER_LOG = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); public static final Charset CHARSET = StandardCharsets.UTF_8; - public volatile boolean isStop = false; public ConfigRocksDBStorage configRocksDBStorage = null; private FlushOptions flushOptions = null; private volatile long lastFlushMemTableMicroSecond = 0; @@ -72,11 +72,14 @@ public RocksDBConfigManager(String filePath, long memTableFlushInterval, Compres } public boolean init(boolean readOnly) { - this.isStop = false; this.configRocksDBStorage = ConfigRocksDBStorage.getStore(filePath, readOnly, compressionType); return this.configRocksDBStorage.start(); } + public boolean isLoaded() { + return this.configRocksDBStorage != null && this.configRocksDBStorage.isLoaded(); + } + public boolean init() { return this.init(false); } @@ -113,7 +116,6 @@ public void start() { } public boolean stop() { - this.isStop = true; ConfigRocksDBStorage.shutdown(filePath); if (this.flushOptions != null) { this.flushOptions.close(); @@ -123,7 +125,7 @@ public boolean stop() { public void flushWAL() { try { - if (this.isStop) { + if (!isLoaded()) { return; } if (this.configRocksDBStorage != null) { @@ -183,4 +185,5 @@ public Statistics getStatistics() { return configRocksDBStorage.getStatistics(); } + } 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 4f635167777..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 @@ -16,16 +16,18 @@ */ package org.apache.rocketmq.broker.config.v1; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; +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; @@ -122,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); @@ -157,7 +159,12 @@ public String configFilePath() { @Override public synchronized void persist() { - if (!rocksDBConfigManager.isStop) { + 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()); @@ -175,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(); @@ -184,7 +220,7 @@ private void putWriteBatch(final WriteBatch writeBatch, final String topicGroupN byte[] keyBytes = topicGroupName.getBytes(DataConverter.CHARSET_UTF8); RocksDBOffsetSerializeWrapper wrapper = new RocksDBOffsetSerializeWrapper(); wrapper.setOffsetTable(offsetMap); - byte[] valueBytes = JSON.toJSONBytes(wrapper, SerializerFeature.BrowserCompatible); + byte[] valueBytes = JSON.toJSONBytes(wrapper, JSONWriter.Feature.BrowserCompatible); rocksDBConfigManager.writeBatchPutOperation(writeBatch, keyBytes, valueBytes); } 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/v1/RocksDBSubscriptionGroupManager.java b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBSubscriptionGroupManager.java index f6ae3a3e598..b4392212a66 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBSubscriptionGroupManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBSubscriptionGroupManager.java @@ -16,16 +16,9 @@ */ package org.apache.rocketmq.broker.config.v1; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; -import com.alibaba.fastjson.serializer.SerializerFeature; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.function.BiConsumer; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.BrokerPathConfigHelper; @@ -35,6 +28,14 @@ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; import org.rocksdb.CompressionType; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.BiConsumer; + public class RocksDBSubscriptionGroupManager extends SubscriptionGroupManager { protected transient RocksDBConfigManager rocksDBConfigManager; @@ -162,7 +163,7 @@ public SubscriptionGroupConfig putSubscriptionGroupConfig(SubscriptionGroupConfi try { byte[] keyBytes = groupName.getBytes(RocksDBConfigManager.CHARSET); - byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, SerializerFeature.BrowserCompatible); + byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, JSONWriter.Feature.BrowserCompatible); this.rocksDBConfigManager.put(keyBytes, valueBytes); } catch (Exception e) { log.error("kv put sub Failed, {}", subscriptionGroupConfig.toString()); @@ -177,7 +178,7 @@ protected SubscriptionGroupConfig putSubscriptionGroupConfigIfAbsent(Subscriptio if (oldConfig == null) { try { byte[] keyBytes = groupName.getBytes(RocksDBConfigManager.CHARSET); - byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, SerializerFeature.BrowserCompatible); + byte[] valueBytes = JSON.toJSONBytes(subscriptionGroupConfig, JSONWriter.Feature.BrowserCompatible); this.rocksDBConfigManager.put(keyBytes, valueBytes); } catch (Exception e) { log.error("kv put sub Failed, {}", subscriptionGroupConfig.toString()); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManager.java index 4a8d124e9bf..96f12e66813 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManager.java @@ -16,12 +16,8 @@ */ package org.apache.rocketmq.broker.config.v1; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Map; -import java.util.concurrent.ConcurrentMap; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.BrokerPathConfigHelper; @@ -32,6 +28,11 @@ import org.apache.rocketmq.remoting.protocol.DataVersion; import org.rocksdb.CompressionType; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; + public class RocksDBTopicConfigManager extends TopicConfigManager { private static final String VERSION_COLUMN_FAMILY = "topicVersion"; private static final String TOPIC_COLUMN_FAMILY = "topic"; @@ -142,7 +143,7 @@ public TopicConfig putTopicConfig(TopicConfig topicConfig) { TopicConfig oldTopicConfig = this.topicConfigTable.put(topicName, topicConfig); try { byte[] keyBytes = topicName.getBytes(DataConverter.CHARSET_UTF8); - byte[] valueBytes = JSON.toJSONBytes(topicConfig, SerializerFeature.BrowserCompatible); + byte[] valueBytes = JSON.toJSONBytes(topicConfig, JSONWriter.Feature.BrowserCompatible); this.rocksDBConfigManager.put(keyBytes, valueBytes); } catch (Exception e) { log.error("kv put topic Failed, {}", topicConfig.toString(), e); 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/controller/ReplicasManager.java b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java index f22f22a12bd..93d48de1dd9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/controller/ReplicasManager.java @@ -775,29 +775,33 @@ private void stopCheckSyncStateSet() { } private void scanAvailableControllerAddresses() { - if (controllerAddresses == null) { - LOGGER.warn("scanAvailableControllerAddresses addresses of controller is null!"); - return; - } + try { + if (controllerAddresses == null) { + LOGGER.warn("scanAvailableControllerAddresses addresses of controller is null!"); + return; + } - for (String address : availableControllerAddresses.keySet()) { - if (!controllerAddresses.contains(address)) { - LOGGER.warn("scanAvailableControllerAddresses remove invalid address {}", address); - availableControllerAddresses.remove(address); + for (String address : availableControllerAddresses.keySet()) { + if (!controllerAddresses.contains(address)) { + LOGGER.warn("scanAvailableControllerAddresses remove invalid address {}", address); + availableControllerAddresses.remove(address); + } } - } - for (String address : controllerAddresses) { - scanExecutor.submit(() -> { - if (brokerOuterAPI.checkAddressReachable(address)) { - availableControllerAddresses.putIfAbsent(address, true); - } else { - Boolean value = availableControllerAddresses.remove(address); - if (value != null) { - LOGGER.warn("scanAvailableControllerAddresses remove unconnected address {}", address); + for (String address : controllerAddresses) { + scanExecutor.submit(() -> { + if (brokerOuterAPI.checkAddressReachable(address)) { + availableControllerAddresses.putIfAbsent(address, true); + } else { + Boolean value = availableControllerAddresses.remove(address); + if (value != null) { + LOGGER.warn("scanAvailableControllerAddresses remove unconnected address {}", address); + } } - } - }); + }); + } + } catch (final Throwable t) { + LOGGER.error("scanAvailableControllerAddresses unexpected exception", t); } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java index ce8fdd88579..31bdb838a2c 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/latency/BrokerFastFailure.java @@ -23,7 +23,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import org.apache.rocketmq.broker.BrokerController; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.ThreadFactoryImpl; import org.apache.rocketmq.common.UtilAll; @@ -80,9 +79,9 @@ public static RequestTask castRunnable(final Runnable runnable) { } public void start() { - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.brokerController.getBrokerConfig()) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { if (brokerController.getBrokerConfig().isBrokerFastFailureEnable()) { cleanExpiredRequest(); } 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..d43c315efc9 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManager.java @@ -0,0 +1,205 @@ +/* + * 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 java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +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 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..2a19bfaca1b --- /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, boolean resetOffset); + + 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..275d60d2d03 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteEventDispatcher.java @@ -0,0 +1,583 @@ +/* + * 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 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; +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; + +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, boolean resetOffset) { + if (resetOffset) { + long consumerOffset = consumerOffsetManager.queryOffset(group, lmqName, 0); + LOGGER.info("unregister and reset offset. {}, {}, {}, {}", group, clientId, lmqName, consumerOffset); + if (consumerOffset > 0) { + consumerOffsetManager.assignResetOffset(lmqName, group, 0, 0); + consumerOrderInfoManager.remove(lmqName, group); + } + } + } + + /** + * 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..616db2ade20 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistry.java @@ -0,0 +1,54 @@ +/* + * 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; + +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, boolean exclusive); + + 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..7e694aa1b4a --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImpl.java @@ -0,0 +1,319 @@ +/* + * 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.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, + boolean exclusive) { + 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 (exclusive) { + excludeClientByLmqName(clientId, group, lmqName); + } + 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, resetOffset); + } + } + 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 parentTopic = 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, parentTopic, 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, parentTopic, 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()); + } + + 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/PopCommandCallback.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopCommandCallback.java index 2e190e20f92..ef541a06786 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopCommandCallback.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopCommandCallback.java @@ -17,33 +17,31 @@ package org.apache.rocketmq.broker.longpolling; +import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; -import java.util.function.Consumer; import org.apache.rocketmq.broker.metrics.ConsumerLagCalculator; import org.apache.rocketmq.remoting.CommandCallback; public class PopCommandCallback implements CommandCallback { private final BiConsumer> biConsumer; - + CompletableFuture> biConsumer; private final ConsumerLagCalculator.ProcessGroupInfo info; - private final Consumer lagRecorder; - + private final CompletableFuture future; public PopCommandCallback( BiConsumer> biConsumer, + CompletableFuture> biConsumer, ConsumerLagCalculator.ProcessGroupInfo info, - Consumer lagRecorder) { + CompletableFuture future) { this.biConsumer = biConsumer; this.info = info; - this.lagRecorder = lagRecorder; + this.future = future; } @Override public void accept() { - biConsumer.accept(info, lagRecorder); + biConsumer.accept(info, future); } } 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/longpolling/PopLongPollingService.java b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopLongPollingService.java index 7068793faee..c595178d193 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopLongPollingService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/longpolling/PopLongPollingService.java @@ -27,18 +27,22 @@ import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.common.KeyBuilder; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.PopAckConstants; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.CommandCallback; +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.NamespaceUtil; import org.apache.rocketmq.remoting.protocol.RemotingCommand; -import org.apache.rocketmq.remoting.netty.NettyRemotingAbstract; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; import org.apache.rocketmq.store.ConsumeQueueExt; import org.apache.rocketmq.store.MessageFilter; @@ -167,13 +171,31 @@ public void notifyMessageArrivingWithRetryTopic(final String topic, final int qu public void notifyMessageArrivingWithRetryTopic(final String topic, final int queueId, long offset, Long tagsCode, long msgStoreTime, byte[] filterBitMap, Map properties) { - String notifyTopic; - if (KeyBuilder.isPopRetryTopicV2(topic)) { - notifyTopic = KeyBuilder.parseNormalTopic(topic); + if (NamespaceUtil.isRetryTopic(topic)) { + notifyMessageArrivingFromRetry(topic, queueId, tagsCode, msgStoreTime, filterBitMap, properties); } else { - notifyTopic = topic; + notifyMessageArriving(topic, queueId, offset, tagsCode, msgStoreTime, filterBitMap, properties); + } + } + + private void notifyMessageArrivingFromRetry(String topic, int queueId, Long tagsCode, long msgStoreTime, byte[] filterBitMap, + Map properties) { + String prefix = MixAll.RETRY_GROUP_TOPIC_PREFIX; + String originGroup = properties.get(MessageConst.PROPERTY_ORIGIN_GROUP); + // In the case of pop consumption, there is no long polling hanging on the retry topic, so the wake-up is skipped. + if (StringUtils.isBlank(originGroup)) { + return; + } + // %RETRY%GROUP is used for pull mode, so the wake-up is skipped. + int originTopicStartIndex = prefix.length() + originGroup.length() + 1; + if (topic.length() <= originTopicStartIndex) { + return; + } + String originTopic = topic.substring(originTopicStartIndex); + if (queueId >= 0) { + notifyMessageArriving(originTopic, -1, originGroup, true, tagsCode, msgStoreTime, filterBitMap, properties); } - notifyMessageArriving(notifyTopic, queueId, offset, tagsCode, msgStoreTime, filterBitMap, properties); + notifyMessageArriving(originTopic, queueId, originGroup, true, tagsCode, msgStoreTime, filterBitMap, properties); } public void notifyMessageArriving(final String topic, final int queueId, long offset, 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 960c1dd2505..b262b6ae525 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,27 +675,27 @@ private void initLagAndDlqMetrics() { consumerLagMessages = brokerMeter.gaugeBuilder(GAUGE_CONSUMER_LAG_MESSAGES) .setDescription("Consumer lag messages") .ofLongs() - .buildWithCallback(measurement -> + .buildWithCallback(measurement -> { consumerLagCalculator.calculateLag(result -> { - // Note: 'record' method uses HashMap which may cause - // concurrent access issues when Pull thread executes Pop callbacks. - synchronized (this) { - measurement.record(result.lag, buildLagAttributes(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") @@ -717,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") @@ -770,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 35519c1d1cb..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 @@ -16,10 +16,16 @@ */ package org.apache.rocketmq.broker.metrics; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +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; @@ -114,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 { @@ -136,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; @@ -211,20 +227,43 @@ private void processAllGroup(Consumer consumer) { } public void calculateLag(Consumer lagRecorder) { + + List> futures = new ArrayList<>(); + + BiConsumer> biConsumer = + (info, future) -> calculate(info, future::complete); + processAllGroup(info -> { if (info.group == null || info.topic == null) { return; } - + CompletableFuture future = new CompletableFuture<>(); if (info.isPop && brokerConfig.isEnableNotifyBeforePopCalculateLag()) { if (popLongPollingService.notifyMessageArriving(info.topic, -1, info.group, - true, null, 0, null, null, new PopCommandCallback(this::calculate, info, lagRecorder))) { + true, null, 0, null, null, + new PopCommandCallback(biConsumer, info, future))) { + futures.add(future); return; } } - calculate(info, lagRecorder); }); + + // Set the maximum wait time to 10 seconds to avoid indefinite blocking + // in case of a fast fail that causes the future to not complete its execution. + try { + CompletableFuture.allOf(futures.toArray( + new CompletableFuture[0])).get(10, TimeUnit.SECONDS); + + futures.forEach(future -> { + if (future.isDone() && !future.isCompletedExceptionally()) { + lagRecorder.accept(future.join()); + } + }); + } catch (Exception e) { + LOGGER.error("Calculate lag timeout after 10 seconds", e); + } } public void calculate(ProcessGroupInfo info, Consumer lagRecorder) { 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/out/BrokerOuterAPI.java b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java index 21ba349c84c..ba4ba2ccf9e 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/out/BrokerOuterAPI.java @@ -46,7 +46,6 @@ import org.apache.rocketmq.client.impl.consumer.PullResultExt; import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.client.producer.SendStatus; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.BrokerIdentity; import org.apache.rocketmq.common.LockCallback; import org.apache.rocketmq.common.MixAll; @@ -356,18 +355,14 @@ public void sendHeartbeatViaDataVersion( requestHeader.setClusterName(clusterName); for (final String namesrvAddr : nameServerAddressList) { - brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) { - - @Override - public void run0() { - RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_DATA_VERSION, requestHeader); - request.setBody(dataVersion.encode()); + brokerOuterExecutor.execute(() -> { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_DATA_VERSION, requestHeader); + request.setBody(dataVersion.encode()); - try { - BrokerOuterAPI.this.remotingClient.invokeOneway(namesrvAddr, request, timeoutMillis); - } catch (Exception e) { - LOGGER.error("sendHeartbeat Exception " + namesrvAddr, e); - } + try { + BrokerOuterAPI.this.remotingClient.invokeOneway(namesrvAddr, request, timeoutMillis); + } catch (Exception e) { + LOGGER.error("sendHeartbeat Exception " + namesrvAddr, e); } }); } @@ -389,9 +384,9 @@ public void sendHeartbeat(final String clusterName, if (nameServerAddressList != null && nameServerAddressList.size() > 0) { for (final String namesrvAddr : nameServerAddressList) { - brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) { + brokerOuterExecutor.execute(new Runnable() { @Override - public void run0() { + public void run() { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.BROKER_HEARTBEAT, requestHeader); try { @@ -532,9 +527,9 @@ public List registerBrokerAll( requestHeader.setBodyCrc32(bodyCrc32); final CountDownLatch countDownLatch = new CountDownLatch(nameServerAddressList.size()); for (final String namesrvAddr : nameServerAddressList) { - brokerOuterExecutor.execute(new AbstractBrokerRunnable(brokerIdentity) { + brokerOuterExecutor.execute(new Runnable() { @Override - public void run0() { + public void run() { try { RegisterBrokerResult result = registerBroker(namesrvAddr, oneway, timeoutMills, requestHeader, body); if (result != null) { @@ -719,9 +714,9 @@ public List needRegister( if (nameServerAddressList != null && nameServerAddressList.size() > 0) { final CountDownLatch countDownLatch = new CountDownLatch(nameServerAddressList.size()); for (final String namesrvAddr : nameServerAddressList) { - brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) { + brokerOuterExecutor.execute(new Runnable() { @Override - public void run0() { + public void run() { try { QueryDataVersionRequestHeader requestHeader = new QueryDataVersionRequestHeader(); requestHeader.setBrokerAddr(brokerAddr); @@ -1501,9 +1496,9 @@ public void sendHeartbeatToController(final String controllerAddress, requestHeader.setHeartbeatTimeoutMills(controllerHeartBeatTimeoutMills); requestHeader.setElectionPriority(electionPriority); requestHeader.setBrokerId(brokerId); - brokerOuterExecutor.execute(new AbstractBrokerRunnable(new BrokerIdentity(clusterName, brokerName, brokerId, isInBrokerContainer)) { + brokerOuterExecutor.execute(new Runnable() { @Override - public void run0() { + public void run() { RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.BROKER_HEARTBEAT, requestHeader); try { 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/PopConsumerRecord.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerRecord.java index 1ee01fea1c8..661ace9bcb0 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerRecord.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerRecord.java @@ -16,9 +16,9 @@ */ package org.apache.rocketmq.broker.pop; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.annotation.JSONField; + import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -119,7 +119,7 @@ public byte[] getValueBytes() { } public static PopConsumerRecord decode(byte[] body) { - return JSONObject.parseObject(body, PopConsumerRecord.class); + return JSON.parseObject(body, PopConsumerRecord.class); } public long getPopTime() { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerService.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerService.java index 277a4999cf6..839c96e3900 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/PopConsumerService.java @@ -16,25 +16,9 @@ */ package org.apache.rocketmq.broker.pop; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Stopwatch; -import java.nio.ByteBuffer; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Objects; -import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Triple; import org.apache.rocketmq.broker.BrokerController; @@ -65,6 +49,23 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.ByteBuffer; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Objects; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + public class PopConsumerService extends ServiceThread { private static final Logger log = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME); @@ -167,7 +168,7 @@ public PopConsumerContext handleGetMessageResult(PopConsumerContext context, Get if (GetMessageStatus.FOUND.equals(result.getStatus()) && !result.getMessageQueueOffset().isEmpty()) { if (context.isFifo()) { - this.setFifoBlocked(context, context.getGroupId(), topicId, queueId, result.getMessageQueueOffset()); + this.setFifoBlocked(context, context.getGroupId(), topicId, queueId, result.getMessageQueueOffset(), result); } // build response header here context.addGetMessageResult(result, topicId, queueId, retryType, offset); @@ -275,10 +276,10 @@ public CompletableFuture getMessageAsync(String clientHost, * Fifo message does not have retry feature in broker */ public void setFifoBlocked(PopConsumerContext context, - String groupId, String topicId, int queueId, List queueOffsetList) { + String groupId, String topicId, int queueId, List queueOffsetList, GetMessageResult getMessageResult) { brokerController.getConsumerOrderInfoManager().update( context.getAttemptId(), false, topicId, groupId, queueId, - context.getPopTime(), context.getInvisibleTime(), queueOffsetList, context.getOrderCountInfoBuilder()); + context.getPopTime(), context.getInvisibleTime(), queueOffsetList, context.getOrderCountInfoBuilder(), getMessageResult); } public boolean isFifoBlocked(PopConsumerContext context, String groupId, String topicId, int queueId) { @@ -624,6 +625,7 @@ public boolean reviveRetry(PopConsumerRecord record, MessageExt messageExt) { msgInner.getProperties().get(MessageConst.PROPERTY_FIRST_POP_TIME) == null) { msgInner.getProperties().put(MessageConst.PROPERTY_FIRST_POP_TIME, String.valueOf(record.getPopTime())); } + msgInner.getProperties().put(MessageConst.PROPERTY_ORIGIN_GROUP, record.getGroupId()); msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); PutMessageResult putMessageResult = 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 new file mode 100644 index 00000000000..4dd19ff7aa4 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManager.java @@ -0,0 +1,156 @@ +/* + * 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.pop.orderly; + +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.rocketmq.common.OrderedConsumptionLevel; +import org.apache.rocketmq.store.GetMessageResult; + +/** + * + * Ordered Consumption Controller Interface + * This is the top-level interface that encapsulates complete ordered consumption management functionality, + * supporting different concurrency strategy implementations + *

+ * Design Goals: + * 1. Support queue-level ordered consumption (existing implementation) + * 2. Support message group-level ordered consumption (improve concurrency) + * 3. Support custom ordered consumption strategies + *

+ */ +public interface ConsumerOrderInfoManager { + + /** + * Update the reception status of message list + * Called by handleGetMessageResult when consumer POPs messages, used to record message status and build consumption information + * + * @param attemptId Distinguish different pop requests + * @param isRetry Whether it is a retry topic + * @param topic Topic name + * @param group Consumer group name + * @param queueId Queue ID + * @param popTime Time when messages are popped + * @param invisibleTime Message invisible time + * @param msgQueueOffsetList List of message queue offsets + * @param orderInfoBuilder String builder for constructing order information + * @param getMessageResult Return new result + */ + void update(String attemptId, boolean isRetry, String topic, String group, int queueId, + long popTime, long invisibleTime, List msgQueueOffsetList, + StringBuilder orderInfoBuilder, GetMessageResult getMessageResult); + + /** + * Check whether the current POP request needs to be blocked + * Used to ensure ordered consumption of ordered messages + * Called when consumer POPs messages + * + * @param attemptId Attempt ID + * @param topic Topic name + * @param group Consumer group name + * @param queueId Queue ID + * @param invisibleTime Invisible time + * @return true indicates blocking is needed, false indicates can proceed + */ + boolean checkBlock(String attemptId, String topic, String group, int queueId, long invisibleTime); + + /** + * Commit message and calculate next consumption offset + * Called when consumer ACKs messages + * + * @param topic Topic name + * @param group Consumer group name + * @param queueId Queue ID + * @param queueOffset Message queue offset + * @param popTime Pop time, used for validation + * @return -1: invalid, -2: no need to commit, >=0: offset that needs to be committed (indicates messages below this offset have been consumed) + */ + long commitAndNext(String topic, String group, int queueId, long queueOffset, long popTime); + + /** + * Update the next visible time of message + * Used for delayed message re-consumption + * + * @param topic Topic name + * @param group Consumer group name + * @param queueId Queue ID + * @param queueOffset Message offset + * @param popTime Pop time, used for validation + * @param nextVisibleTime Next visible time + */ + void updateNextVisibleTime(String topic, String group, int queueId, long queueOffset, + long popTime, long nextVisibleTime); + + /** + * Clear the blocking status of specified queue + * Usually called during consumer rebalancing or queue reassignment + * + * @param topic Topic name + * @param group Consumer group name + * @param queueId Queue ID + */ + void clearBlock(String topic, String group, int queueId); + + /** + * 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(); + + /** + * Get ordered consumption level + * Used to distinguish different implementation strategies + * + * @return Ordered consumption level, such as: QUEUE, MESSAGE_GROUP, etc. + */ + OrderedConsumptionLevel getOrderedConsumptionLevel(); + + /** + * Start the controller + * Initialize necessary resources, such as timers, thread pools, etc. + */ + void start(); + + /** + * Shutdown the controller + * Release resources, clean up scheduled tasks, etc. + */ + void shutdown(); + + /** + * Persist the controller + * Persist the controller's data + */ + void persist(); + + boolean load(); + + /** + * Get available message result + * Used to retrieve messages from cache + */ + 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/offset/ConsumerOrderInfoManager.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java similarity index 84% rename from broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManager.java rename to broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java index 120f5b104c7..becd29eb1ef 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerManager.java @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.broker.offset; +package org.apache.rocketmq.broker.pop.orderly; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import java.util.ArrayList; @@ -26,18 +26,21 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.BrokerPathConfigHelper; import org.apache.rocketmq.common.ConfigManager; +import org.apache.rocketmq.common.OrderedConsumptionLevel; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; import org.apache.rocketmq.remoting.protocol.header.ExtraInfoUtil; +import org.apache.rocketmq.store.GetMessageResult; -public class ConsumerOrderInfoManager extends ConfigManager { +public class QueueLevelConsumerManager extends ConfigManager implements ConsumerOrderInfoManager { private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); private static final String TOPIC_GROUP_SEPARATOR = "@"; @@ -46,15 +49,15 @@ public class ConsumerOrderInfoManager extends ConfigManager { private ConcurrentHashMap> table = new ConcurrentHashMap<>(128); - private transient ConsumerOrderInfoLockManager consumerOrderInfoLockManager; + private transient QueueLevelConsumerOrderInfoLockManager queueLevelConsumerOrderInfoLockManager; private transient BrokerController brokerController; - public ConsumerOrderInfoManager() { + public QueueLevelConsumerManager() { } - public ConsumerOrderInfoManager(BrokerController brokerController) { + public QueueLevelConsumerManager(BrokerController brokerController) { this.brokerController = brokerController; - this.consumerOrderInfoLockManager = new ConsumerOrderInfoLockManager(brokerController); + this.queueLevelConsumerOrderInfoLockManager = new QueueLevelConsumerOrderInfoLockManager(brokerController); } public ConcurrentHashMap> getTable() { @@ -73,9 +76,9 @@ protected static String[] decodeKey(String key) { return key.split(TOPIC_GROUP_SEPARATOR); } - private void updateLockFreeTimestamp(String topic, String group, int queueId, OrderInfo orderInfo) { - if (consumerOrderInfoLockManager != null) { - consumerOrderInfoLockManager.updateLockFreeTimestamp(topic, group, queueId, orderInfo); + protected void updateLockFreeTimestamp(String topic, String group, int queueId, OrderInfo orderInfo) { + if (queueLevelConsumerOrderInfoLockManager != null) { + queueLevelConsumerOrderInfoLockManager.updateLockFreeTimestamp(topic, group, queueId, orderInfo); } } @@ -91,7 +94,8 @@ private void updateLockFreeTimestamp(String topic, String group, int queueId, Or * @param msgQueueOffsetList the queue offsets of messages * @param orderInfoBuilder will append order info to this builder */ - public void update(String attemptId, boolean isRetry, String topic, String group, int queueId, long popTime, long invisibleTime, + public void update(String attemptId, boolean isRetry, String topic, String group, int queueId, long popTime, + long invisibleTime, List msgQueueOffsetList, StringBuilder orderInfoBuilder) { String key = buildKey(topic, group); ConcurrentHashMap qs = table.get(key); @@ -140,6 +144,14 @@ public void update(String attemptId, boolean isRetry, String topic, String group updateLockFreeTimestamp(topic, group, queueId, orderInfo); } + @Override + public void update(String attemptId, boolean isRetry, String topic, String group, int queueId, + long popTime, long invisibleTime, + List msgQueueOffsetList, StringBuilder orderInfoBuilder, GetMessageResult getMessageResult) { + update(attemptId, isRetry, topic, group, queueId, popTime, invisibleTime, msgQueueOffsetList, orderInfoBuilder); + } + + @Override public boolean checkBlock(String attemptId, String topic, String group, int queueId, long invisibleTime) { String key = buildKey(topic, group); ConcurrentHashMap qs = table.get(key); @@ -159,6 +171,7 @@ public boolean checkBlock(String attemptId, String topic, String group, int queu return orderInfo.needBlock(attemptId, invisibleTime); } + @Override public void clearBlock(String topic, String group, int queueId) { table.computeIfPresent(buildKey(topic, group), (key, val) -> { val.remove(queueId); @@ -166,6 +179,25 @@ 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; + } + + @Override + public void start() { + } + /** * mark message is consumed finished. return the consumer offset * @@ -175,6 +207,7 @@ public void clearBlock(String topic, String group, int queueId) { * @param queueOffset queue offset of message * @return -1 : illegal, -2 : no need commit, >= 0 : commit */ + @Override public long commitAndNext(String topic, String group, int queueId, long queueOffset, long popTime) { String key = buildKey(topic, group); ConcurrentHashMap qs = table.get(key); @@ -234,7 +267,9 @@ public long commitAndNext(String topic, String group, int queueId, long queueOff * @param queueOffset queue offset of message * @param nextVisibleTime nex visible time */ - public void updateNextVisibleTime(String topic, String group, int queueId, long queueOffset, long popTime, long nextVisibleTime) { + @Override + public void updateNextVisibleTime(String topic, String group, int queueId, long queueOffset, long popTime, + long nextVisibleTime) { String key = buildKey(topic, group); ConcurrentHashMap qs = table.get(key); @@ -256,6 +291,7 @@ public void updateNextVisibleTime(String topic, String group, int queueId, long updateLockFreeTimestamp(topic, group, queueId, orderInfo); } + @VisibleForTesting protected void autoClean() { if (brokerController == null) { return; @@ -328,11 +364,11 @@ public String configFilePath() { @Override public void decode(String jsonString) { if (jsonString != null) { - ConsumerOrderInfoManager obj = RemotingSerializable.fromJson(jsonString, ConsumerOrderInfoManager.class); + QueueLevelConsumerManager obj = RemotingSerializable.fromJson(jsonString, QueueLevelConsumerManager.class); if (obj != null) { this.table = obj.table; - if (this.consumerOrderInfoLockManager != null) { - this.consumerOrderInfoLockManager.recover(this.table); + if (this.queueLevelConsumerOrderInfoLockManager != null) { + this.queueLevelConsumerOrderInfoLockManager.recover(this.table); } } } @@ -345,14 +381,20 @@ public String encode(boolean prettyFormat) { } public void shutdown() { - if (this.consumerOrderInfoLockManager != null) { - this.consumerOrderInfoLockManager.shutdown(); + if (this.queueLevelConsumerOrderInfoLockManager != null) { + this.queueLevelConsumerOrderInfoLockManager.shutdown(); } } + @Override + public CompletableFuture getAvailableMessageResult(String attemptId, long popTime, long invisibleTime, + String groupId, String topicId, int queueId, int batchSize, StringBuilder orderCountInfoBuilder) { + return CompletableFuture.completedFuture(null); + } + @VisibleForTesting - protected ConsumerOrderInfoLockManager getConsumerOrderInfoLockManager() { - return consumerOrderInfoLockManager; + protected QueueLevelConsumerOrderInfoLockManager getConsumerOrderInfoLockManager() { + return queueLevelConsumerOrderInfoLockManager; } public static class OrderInfo { @@ -397,7 +439,8 @@ public static class OrderInfo { public OrderInfo() { } - public OrderInfo(String attemptId, long popTime, long invisibleTime, List queueOffsetList, long lastConsumeTimestamp, + public OrderInfo(String attemptId, long popTime, long invisibleTime, List queueOffsetList, + long lastConsumeTimestamp, long commitOffsetBit) { this.popTime = popTime; this.invisibleTime = invisibleTime; @@ -544,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) { @@ -600,7 +670,8 @@ public boolean isNotAck(int offsetIndex) { * @param prevOffsetConsumedCount the offset list of message */ @JSONField(serialize = false, deserialize = false) - public void mergeOffsetConsumedCount(String preAttemptId, List preOffsetList, Map prevOffsetConsumedCount) { + public void mergeOffsetConsumedCount(String preAttemptId, List preOffsetList, + Map prevOffsetConsumedCount) { Map offsetConsumedCount = new HashMap<>(); if (prevOffsetConsumedCount == null) { prevOffsetConsumedCount = new HashMap<>(); @@ -641,4 +712,4 @@ public String toString() { .toString(); } } -} +} \ No newline at end of file diff --git a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoLockManager.java b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java similarity index 85% rename from broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoLockManager.java rename to broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java index 37b3eed2302..08615416bde 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoLockManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/pop/orderly/QueueLevelConsumerOrderInfoLockManager.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.rocketmq.broker.offset; +package org.apache.rocketmq.broker.pop.orderly; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; @@ -29,17 +29,20 @@ 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; -public class ConsumerOrderInfoLockManager { +public class QueueLevelConsumerOrderInfoLockManager { private static final Logger POP_LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME); + private ConsumerOrderInfoManager consumerOrderInfoManager; + private final BrokerController brokerController; private final Map timeoutMap = new ConcurrentHashMap<>(); private final Timer timer; private static final int TIMER_TICK_MS = 100; - public ConsumerOrderInfoLockManager(BrokerController brokerController) { + public QueueLevelConsumerOrderInfoLockManager(BrokerController brokerController) { this.brokerController = brokerController; this.timer = new HashedWheelTimer( new ThreadFactoryImpl("ConsumerOrderInfoLockManager_"), @@ -47,22 +50,22 @@ public ConsumerOrderInfoLockManager(BrokerController brokerController) { } /** - * when ConsumerOrderInfoManager load from disk, recover data + * when QueueLevelConsumerManager load from disk, recover data */ - public void recover(Map> table) { + public void recover(Map> table) { if (!this.brokerController.getBrokerConfig().isEnableNotifyAfterPopOrderLockRelease()) { return; } - for (Map.Entry> entry : table.entrySet()) { + for (Map.Entry> entry : table.entrySet()) { String topicAtGroup = entry.getKey(); - ConcurrentHashMap qs = entry.getValue(); - String[] arrays = ConsumerOrderInfoManager.decodeKey(topicAtGroup); + ConcurrentHashMap qs = entry.getValue(); + String[] arrays = QueueLevelConsumerManager.decodeKey(topicAtGroup); if (arrays.length != 2) { continue; } String topic = arrays[0]; String group = arrays[1]; - for (Map.Entry qsEntry : qs.entrySet()) { + for (Map.Entry qsEntry : qs.entrySet()) { Long lockFreeTimestamp = qsEntry.getValue().getLockFreeTimestamp(); if (lockFreeTimestamp == null || lockFreeTimestamp <= System.currentTimeMillis()) { continue; @@ -72,7 +75,7 @@ public void recover(Map -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 298e2390864..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 @@ -16,8 +16,9 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.alibaba.fastjson2.JSONWriter; import com.google.common.collect.Sets; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; @@ -66,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; @@ -89,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; @@ -236,6 +238,7 @@ import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_INVOCATION_STATUS; import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_IS_SYSTEM; +import static org.apache.rocketmq.common.message.MessageConst.TIMER_ENGINE_TYPE; import static org.apache.rocketmq.remoting.protocol.RemotingCommand.buildErrorResponse; public class AdminBrokerProcessor implements NettyRequestProcessor { @@ -405,6 +408,8 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, return this.listAcl(ctx, request); case RequestCode.POP_ROLLBACK: return this.transferPopToFsStore(ctx, request); + case RequestCode.SWITCH_TIMER_ENGINE: + return this.switchTimerEngine(ctx, request); default: return getUnknownCmdResponse(ctx, request); } @@ -796,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); @@ -1177,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); @@ -1674,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()); } @@ -2785,7 +2812,7 @@ private RemotingCommand queryConsumeQueue(ChannelHandlerContext ctx, } else { ConsumerFilterData filterData = this.brokerController.getConsumerFilterManager() .get(requestHeader.getTopic(), requestHeader.getConsumerGroup()); - body.setFilterData(JSON.toJSONString(filterData, true)); + body.setFilterData(JSON.toJSONString(filterData, JSONWriter.Feature.PrettyFormat)); messageFilter = new ExpressionMessageFilter(subscriptionData, filterData, this.brokerController.getConsumerFilterManager()); @@ -2879,7 +2906,11 @@ private RemotingCommand resumeCheckHalfMessage(ChannelHandlerContext ctx, private MessageExtBrokerInner toMessageExtBrokerInner(MessageExt msgExt) { MessageExtBrokerInner inner = new MessageExtBrokerInner(); - inner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + if (brokerController.getMessageStoreConfig().isTransRocksDBEnable() && !brokerController.getMessageStoreConfig().isTransWriteOriginTransHalfEnable()) { + inner.setTopic(TransactionalMessageUtil.buildHalfTopicForRocksDB()); + } else { + inner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + } inner.setBody(msgExt.getBody()); inner.setFlag(msgExt.getFlag()); MessageAccessor.setProperties(inner, msgExt.getProperties()); @@ -3408,4 +3439,64 @@ private RemotingCommand transferPopToFsStore(ChannelHandlerContext ctx, Remoting } return response; } + + private synchronized RemotingCommand switchTimerEngine(ChannelHandlerContext ctx, RemotingCommand request) { + final RemotingCommand response = RemotingCommand.createResponseCommand(null); + if (!this.brokerController.getMessageStoreConfig().isTimerWheelEnable()) { + LOGGER.info("switchTimerEngine error, broker timerWheelEnable is false"); + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("broker timerWheelEnable is false"); + return response; + } + if (null == request.getExtFields()) { + LOGGER.info("switchTimerEngine extFields is null"); + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("param error, extFields is null"); + return response; + } + String engineType = request.getExtFields().get(TIMER_ENGINE_TYPE); + if (StringUtils.isEmpty(engineType) || !MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType) && !MessageConst.TIMER_ENGINE_FILE_TIME_WHEEL.equals(engineType)) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("param error"); + return response; + } + try { + Properties properties = new Properties(); + boolean result = false; + if (MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType)) { + if (this.brokerController.getTimerMessageRocksDBStore() == null) { + response.setCode(ResponseCode.INVALID_PARAMETER); + response.setRemark("timerUseRocksDB muse be configured true when broker start"); + return response; + } + result = this.brokerController.getTimerMessageRocksDBStore().restart(); + if (result) { + properties.put("timerStopEnqueue", Boolean.TRUE.toString()); + properties.put("timerUseRocksDB", Boolean.TRUE.toString()); + properties.put("timerRocksDBStopScan", Boolean.FALSE.toString()); + } + } else { + result = this.brokerController.getTimerMessageStore().restart(); + if (result) { + properties.put("timerRocksDBStopScan", Boolean.TRUE.toString()); + properties.put("timerStopEnqueue", Boolean.FALSE.toString()); + } + } + if (result) { + this.brokerController.getConfiguration().update(properties); + response.setCode(ResponseCode.SUCCESS); + response.setRemark("switch timer engine success"); + LOGGER.info("switchTimerEngine success"); + } else { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("switch timer engine error"); + LOGGER.info("switchTimerEngine error"); + } + } catch (Exception e) { + response.setCode(ResponseCode.SYSTEM_ERROR); + response.setRemark("switch timer engine error"); + LOGGER.error("switchTimerEngine error : {}", e.getMessage()); + } + return response; + } } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessor.java index e8e2a909952..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 @@ -16,20 +16,22 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; +import java.nio.charset.StandardCharsets; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; -import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; -import org.apache.rocketmq.broker.offset.ConsumerOrderInfoManager; import org.apache.rocketmq.broker.pop.PopConsumerLockService; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; import org.apache.rocketmq.common.PopAckConstants; 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; @@ -37,11 +39,11 @@ 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.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.ChangeInvisibleTimeRequestHeader; -import org.apache.rocketmq.remoting.netty.NettyRemotingAbstract; import org.apache.rocketmq.remoting.protocol.header.ChangeInvisibleTimeResponseHeader; import org.apache.rocketmq.remoting.protocol.header.ExtraInfoUtil; import org.apache.rocketmq.store.PutMessageStatus; @@ -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/EndTransactionProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java index 153ac24c1f6..7298e5da58a 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/EndTransactionProcessor.java @@ -30,6 +30,7 @@ import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.common.topic.TopicValidator; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.common.RemotingHelper; @@ -146,7 +147,7 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_TRANSACTION_PREPARED); RemotingCommand sendResult = sendFinalMessage(msgInner); if (sendResult.getCode() == ResponseCode.SUCCESS) { - this.brokerController.getTransactionalMessageService().deletePrepareMessage(result.getPrepareMessage()); + deletePrepareMessage(result); // successful committed, then total num of half-messages minus 1 this.brokerController.getTransactionalMessageService().getTransactionMetrics().addAndGet(msgInner.getTopic(), -1); this.brokerController.getBrokerMetricsManager().getCommitMessagesTotal().add(1, this.brokerController.getBrokerMetricsManager().newAttributesBuilder() @@ -173,7 +174,7 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand } RemotingCommand res = checkPrepareMessage(result.getPrepareMessage(), requestHeader); if (res.getCode() == ResponseCode.SUCCESS) { - this.brokerController.getTransactionalMessageService().deletePrepareMessage(result.getPrepareMessage()); + deletePrepareMessage(result); // roll back, then total num of half-messages minus 1 this.brokerController.getTransactionalMessageService().getTransactionMetrics().addAndGet(result.getPrepareMessage().getProperty(MessageConst.PROPERTY_REAL_TOPIC), -1); this.brokerController.getBrokerMetricsManager().getRollBackMessagesTotal().add(1, this.brokerController.getBrokerMetricsManager().newAttributesBuilder() @@ -188,6 +189,26 @@ public RemotingCommand processRequest(ChannelHandlerContext ctx, RemotingCommand return response; } + private void deletePrepareMessage(OperationResult result) { + if (null == result || null == result.getPrepareMessage()) { + LOGGER.error("deletePrepareMessage param error, result is null or prepareMessage is null"); + return; + } + MessageExt prepareMessage = result.getPrepareMessage(); + String halfTopic = prepareMessage.getTopic(); + if (StringUtils.isEmpty(halfTopic)) { + LOGGER.error("deletePrepareMessage halfTopic is empty, halfTopic: {}", halfTopic); + return; + } + if (TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC.equals(halfTopic)) { + this.brokerController.getTransactionalMessageService().deletePrepareMessage(prepareMessage); + } else if (this.brokerController.getMessageStoreConfig().isTransRocksDBEnable() && TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC.equals(halfTopic)) { + this.brokerController.getMessageStore().getTransRocksDBStore().deletePrepareMessage(prepareMessage); + } else { + LOGGER.warn("deletePrepareMessage error, topic of half message is: {}, transRocksDBEnable: {}", halfTopic, this.brokerController.getMessageStoreConfig().isTransRocksDBEnable()); + } + } + /** * If you specify a custom first check time CheckImmunityTimeInSeconds, * And the commit/rollback request whose validity period exceeds CheckImmunityTimeInSeconds and is not checked back will be processed and failed @@ -265,10 +286,17 @@ private MessageExtBrokerInner endMessageTransaction(MessageExt msgExt) { : TopicFilterType.SINGLE_TAG; long tagsCodeValue = MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags()); msgInner.setTagsCode(tagsCodeValue); - MessageAccessor.setProperties(msgInner, msgExt.getProperties()); - msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgExt.getProperties())); + String checkTimes = msgExt.getUserProperty(MessageConst.PROPERTY_TRANSACTION_CHECK_TIMES); + if (StringUtils.isEmpty(checkTimes) && this.brokerController.getMessageStoreConfig().isTransRocksDBEnable() && null != this.brokerController.getMessageStore().getTransRocksDBStore()) { + Integer checkTimesRocksDB = this.brokerController.getMessageStore().getTransRocksDBStore().getCheckTimes(msgInner.getTopic(), msgInner.getTransactionId(), msgExt.getCommitLogOffset()); + if (null != checkTimesRocksDB && checkTimesRocksDB >= 0) { + msgExt.putUserProperty(MessageConst.PROPERTY_TRANSACTION_CHECK_TIMES, String.valueOf(checkTimesRocksDB)); + } + } + MessageAccessor.setProperties(msgInner, MessageDecoder.string2messageProperties(MessageDecoder.messageProperties2String(msgExt.getProperties()))); MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_TOPIC); MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_QUEUE_ID); + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); return msgInner; } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteManagerProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteManagerProcessor.java new file mode 100644 index 00000000000..04c1a6748da --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteManagerProcessor.java @@ -0,0 +1,381 @@ +/* + * 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.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +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; + +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..66b0f4e7b5f --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/LiteSubscriptionCtlProcessor.java @@ -0,0 +1,137 @@ +/* + * 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()); + boolean isExclusive = LiteMetadataUtil.isSubLiteExclusive(group, brokerController); + this.liteSubscriptionRegistry.addPartialSubscription(clientId, group, topic, lmqNameSet, isExclusive); + 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/NotificationProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/NotificationProcessor.java index 640d77c298c..4563132fe48 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/NotificationProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/NotificationProcessor.java @@ -19,7 +19,6 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; import java.util.Map; -import java.util.Objects; import java.util.Random; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.longpolling.PollingHeader; @@ -79,16 +78,17 @@ public void notifyMessageArriving(final String topic, final int queueId) { @Override public RemotingCommand processRequest(final ChannelHandlerContext ctx, RemotingCommand request) throws RemotingCommandException { - 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())); - } Channel channel = ctx.channel(); RemotingCommand response = RemotingCommand.createResponseCommand(NotificationResponseHeader.class); final NotificationResponseHeader responseHeader = (NotificationResponseHeader) response.readCustomHeader(); final NotificationRequestHeader requestHeader = - (NotificationRequestHeader) request.decodeCommandCustomHeader(NotificationRequestHeader.class); + request.decodeCommandCustomHeader(NotificationRequestHeader.class, true); + if (requestHeader.getBornTime() == 0) { + final long beginTimeMills = this.brokerController.getMessageStore().now(); + request.addExtField(BORN_TIME, String.valueOf(beginTimeMills)); + requestHeader.setBornTime(beginTimeMills); + } response.setOpaque(request.getOpaque()); @@ -135,9 +135,16 @@ public RemotingCommand processRequest(final ChannelHandlerContext ctx, } int randomQ = random.nextInt(100); boolean hasMsg = false; - boolean needRetry = randomQ % 5 == 0; BrokerConfig brokerConfig = brokerController.getBrokerConfig(); - if (needRetry) { + if (requestHeader.getQueueId() < 0) { + // read all queue + hasMsg = hasMsgFromTopic(topicConfig, randomQ, requestHeader); + } else { + int queueId = requestHeader.getQueueId(); + hasMsg = hasMsgFromQueue(topicConfig.getTopicName(), requestHeader, queueId); + } + // if it doesn't have message, fetch retry + if (!hasMsg) { String retryTopic = KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup(), brokerConfig.isEnableRetryTopicV2()); hasMsg = hasMsgFromTopic(retryTopic, randomQ, requestHeader); if (!hasMsg && brokerConfig.isEnableRetryTopicV2() && brokerConfig.isRetrieveMessageFromPopRetryTopicV1()) { @@ -145,24 +152,6 @@ public RemotingCommand processRequest(final ChannelHandlerContext ctx, hasMsg = hasMsgFromTopic(retryTopicConfigV1, randomQ, requestHeader); } } - if (!hasMsg) { - if (requestHeader.getQueueId() < 0) { - // read all queue - hasMsg = hasMsgFromTopic(topicConfig, randomQ, requestHeader); - } else { - int queueId = requestHeader.getQueueId(); - hasMsg = hasMsgFromQueue(topicConfig.getTopicName(), requestHeader, queueId); - } - // if it doesn't have message, fetch retry again - if (!needRetry && !hasMsg) { - String retryTopic = KeyBuilder.buildPopRetryTopic(requestHeader.getTopic(), requestHeader.getConsumerGroup(), brokerConfig.isEnableRetryTopicV2()); - hasMsg = hasMsgFromTopic(retryTopic, randomQ, requestHeader); - if (!hasMsg && brokerConfig.isEnableRetryTopicV2() && brokerConfig.isRetrieveMessageFromPopRetryTopicV1()) { - String retryTopicConfigV1 = KeyBuilder.buildPopRetryTopicV1(requestHeader.getTopic(), requestHeader.getConsumerGroup()); - hasMsg = hasMsgFromTopic(retryTopicConfigV1, randomQ, requestHeader); - } - } - } if (!hasMsg) { PollingResult pollingResult = popLongPollingService.polling(ctx, request, new PollingHeader(requestHeader)); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java index 657adaa34d5..06d89e047d9 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopBufferMergeService.java @@ -16,15 +16,7 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.atomic.AtomicInteger; +import com.alibaba.fastjson2.JSON; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.common.KeyBuilder; import org.apache.rocketmq.common.PopAckConstants; @@ -43,6 +35,15 @@ import org.apache.rocketmq.store.pop.BatchAckMsg; import org.apache.rocketmq.store.pop.PopCheckPoint; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.atomic.AtomicInteger; + public class PopBufferMergeService extends ServiceThread { private static final Logger POP_LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_POP_LOGGER_NAME); ConcurrentHashMap 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..f90d05ea058 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessor.java @@ -0,0 +1,479 @@ +/* + * 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 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 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 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/PopMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java index 411fb064574..3144eb973ad 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopMessageProcessor.java @@ -16,27 +16,13 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import com.github.benmanes.caffeine.cache.Cache; import io.netty.channel.Channel; import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.FileRegion; import io.opentelemetry.api.common.Attributes; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; -import java.util.Random; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.filter.ConsumerFilterData; import org.apache.rocketmq.broker.filter.ConsumerFilterManager; @@ -45,7 +31,6 @@ import org.apache.rocketmq.broker.longpolling.PollingResult; import org.apache.rocketmq.broker.longpolling.PopLongPollingService; import org.apache.rocketmq.broker.longpolling.PopRequest; - import org.apache.rocketmq.broker.pagecache.ManyMessageTransfer; import org.apache.rocketmq.broker.pop.PopConsumerContext; import org.apache.rocketmq.common.BrokerConfig; @@ -91,6 +76,20 @@ import org.apache.rocketmq.store.pop.BatchAckMsg; import org.apache.rocketmq.store.pop.PopCheckPoint; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Random; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +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; @@ -228,18 +227,16 @@ public RemotingCommand processRequest(final ChannelHandlerContext ctx, RemotingC final long beginTimeMills = this.brokerController.getMessageStore().now(); - // fill bron time to properties if not exist, why we need this? - 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())); - } - Channel channel = ctx.channel(); RemotingCommand response = RemotingCommand.createResponseCommand(PopMessageResponseHeader.class); response.setOpaque(request.getOpaque()); final PopMessageRequestHeader requestHeader = request.decodeCommandCustomHeader(PopMessageRequestHeader.class, true); + if (requestHeader.getBornTime() == 0) { + request.addExtField(BORN_TIME, String.valueOf(beginTimeMills)); + requestHeader.setBornTime(beginTimeMills); + } final PopMessageResponseHeader responseHeader = (PopMessageResponseHeader) response.readCustomHeader(); // Pop mode only supports consumption in cluster load balancing mode @@ -809,7 +806,7 @@ private CompletableFuture popMsgFromQueue(String topic, String attemptId, this.brokerController.getConsumerOrderInfoManager().update(requestHeader.getAttemptId(), isRetry, topic, requestHeader.getConsumerGroup(), queueId, popTime, requestHeader.getInvisibleTime(), result.getMessageQueueOffset(), - orderCountInfo); + orderCountInfo, result); this.brokerController.getConsumerOffsetManager().commitOffset(channel.remoteAddress().toString(), requestHeader.getConsumerGroup(), topic, queueId, finalOffset); } else { diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java index 7bf3595be89..434812883e1 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/PopReviveService.java @@ -16,32 +16,21 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import io.opentelemetry.api.common.Attributes; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.TreeMap; -import java.util.concurrent.CompletableFuture; import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Triple; import org.apache.rocketmq.broker.BrokerController; - import org.apache.rocketmq.client.consumer.PullResult; import org.apache.rocketmq.client.consumer.PullStatus; import org.apache.rocketmq.common.KeyBuilder; import org.apache.rocketmq.common.MixAll; -import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.PopAckConstants; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.TopicFilterType; +import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.message.MessageAccessor; import org.apache.rocketmq.common.message.MessageConst; @@ -60,6 +49,17 @@ import org.apache.rocketmq.store.pop.BatchAckMsg; import org.apache.rocketmq.store.pop.PopCheckPoint; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; + import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_CONSUMER_GROUP; import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_IS_SYSTEM; import static org.apache.rocketmq.broker.metrics.BrokerMetricsConstant.LABEL_TOPIC; @@ -128,6 +128,7 @@ private boolean reviveRetry(PopCheckPoint popCheckPoint, MessageExt messageExt) if (messageExt.getReconsumeTimes() == 0 || msgInner.getProperties().get(MessageConst.PROPERTY_FIRST_POP_TIME) == null) { msgInner.getProperties().put(MessageConst.PROPERTY_FIRST_POP_TIME, String.valueOf(popCheckPoint.getPopTime())); } + msgInner.getProperties().put(MessageConst.PROPERTY_ORIGIN_GROUP, popCheckPoint.getCId()); msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); addRetryTopicIfNotExist(msgInner.getTopic(), popCheckPoint.getCId()); PutMessageResult putMessageResult = brokerController.getEscapeBridge().putMessageToSpecificQueue(msgInner); 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/QueryMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java index fc46cb53186..f29cd2b8a14 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/processor/QueryMessageProcessor.java @@ -21,11 +21,13 @@ import io.netty.channel.FileRegion; import io.opentelemetry.api.common.Attributes; import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.pagecache.OneMessageTransfer; import org.apache.rocketmq.broker.pagecache.QueryMessageTransfer; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.common.RemotingHelper; @@ -84,16 +86,19 @@ public RemotingCommand queryMessage(ChannelHandlerContext ctx, RemotingCommand r .decodeCommandCustomHeader(QueryMessageRequestHeader.class); response.setOpaque(request.getOpaque()); - - String isUniqueKey = request.getExtFields().get(MixAll.UNIQUE_MSG_QUERY_FLAG); - if (isUniqueKey != null && isUniqueKey.equals("true")) { + String indexType = requestHeader.getIndexType(); + String lastKey = requestHeader.getLastKey(); + String isUniqueKey = null; + if (null != request.getExtFields()) { + isUniqueKey = request.getExtFields().get(MixAll.UNIQUE_MSG_QUERY_FLAG); + } + if (!StringUtils.isEmpty(isUniqueKey) && Boolean.parseBoolean(isUniqueKey)) { requestHeader.setMaxNum(this.brokerController.getMessageStoreConfig().getDefaultQueryMaxNum()); + indexType = MessageConst.INDEX_UNIQUE_TYPE; + } else if (StringUtils.isEmpty(indexType)) { + indexType = MessageConst.INDEX_KEY_TYPE; } - - final QueryMessageResult queryMessageResult = - this.brokerController.getMessageStore().queryMessage(requestHeader.getTopic(), - requestHeader.getKey(), requestHeader.getMaxNum(), requestHeader.getBeginTimestamp(), - requestHeader.getEndTimestamp()); + final QueryMessageResult queryMessageResult = this.brokerController.getMessageStore().queryMessage(requestHeader.getTopic(), requestHeader.getKey(), requestHeader.getMaxNum(), requestHeader.getBeginTimestamp(), requestHeader.getEndTimestamp(), indexType, lastKey); assert queryMessageResult != null; responseHeader.setIndexLastUpdatePhyoffset(queryMessageResult.getIndexLastUpdatePhyoffset()); diff --git a/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java b/broker/src/main/java/org/apache/rocketmq/broker/processor/SendMessageProcessor.java index eefdb85ccf4..9faedb1125c 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); CleanupPolicy cleanupPolicy = CleanupPolicyUtils.getDeletePolicy(Optional.of(topicConfig)); @@ -435,8 +443,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/topic/TopicConfigManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java index f59651fc8dd..51b5f5492ad 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicConfigManager.java @@ -214,6 +214,26 @@ protected void init() { putTopicConfig(topicConfig); } + { + // TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC + String topic = TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC; + TopicConfig topicConfig = new TopicConfig(topic); + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); + putTopicConfig(topicConfig); + } + + { + // TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC + String topic = TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC; + TopicConfig topicConfig = new TopicConfig(topic); + TopicValidator.addSystemTopic(topic); + topicConfig.setReadQueueNums(1); + topicConfig.setWriteQueueNums(1); + putTopicConfig(topicConfig); + } + { if (this.brokerController.getMessageStoreConfig().isTimerWheelEnable()) { String topic = TimerMessageStore.TIMER_TOPIC; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManager.java b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManager.java index 4b0714decb6..9e20ecd9b6a 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManager.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManager.java @@ -16,7 +16,8 @@ */ package org.apache.rocketmq.broker.topic; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import com.google.common.collect.Maps; import java.util.List; import java.util.Map; @@ -156,7 +157,10 @@ public String encode(boolean pretty) { TopicQueueMappingSerializeWrapper wrapper = new TopicQueueMappingSerializeWrapper(); wrapper.setTopicQueueMappingInfoMap(topicQueueMappingTable); wrapper.setDataVersion(this.dataVersion); - return JSON.toJSONString(wrapper, pretty); + if (pretty) { + return JSON.toJSONString(wrapper, JSONWriter.Feature.PrettyFormat); + } + return JSON.toJSONString(wrapper); } @Override diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionMetrics.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionMetrics.java index 8a18218f533..17b0ac67746 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionMetrics.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/TransactionMetrics.java @@ -16,9 +16,19 @@ */ package org.apache.rocketmq.broker.transaction; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; +import org.apache.rocketmq.common.ConfigManager; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.constant.LoggerName; +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.protocol.DataVersion; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + import java.io.File; +import java.io.IOException; import java.io.RandomAccessFile; import java.io.StringWriter; import java.io.Writer; @@ -32,14 +42,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; -import org.apache.rocketmq.common.ConfigManager; -import org.apache.rocketmq.common.MixAll; -import org.apache.rocketmq.common.constant.LoggerName; -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.protocol.DataVersion; -import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + public class TransactionMetrics extends ConfigManager { private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -90,11 +93,11 @@ public void setTransactionCounts(ConcurrentMap transactionCounts this.transactionCounts = transactionCounts; } - protected void write0(Writer writer) { + protected void write0(Writer writer) throws IOException { TransactionMetricsSerializeWrapper wrapper = new TransactionMetricsSerializeWrapper(); wrapper.setTransactionCount(transactionCounts); wrapper.setDataVersion(dataVersion); - JSON.writeJSONString(writer, wrapper, SerializerFeature.BrowserCompatible); + writer.write(JSON.toJSONString(wrapper, JSONWriter.Feature.BrowserCompatible)); } @Override diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java index 5fcc1f56b78..47e453946d7 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageBridge.java @@ -226,7 +226,11 @@ private MessageExtBrokerInner parseHalfMessageInner(MessageExtBrokerInner msgInn String.valueOf(msgInner.getQueueId())); msgInner.setSysFlag( MessageSysFlag.resetTransactionValue(msgInner.getSysFlag(), MessageSysFlag.TRANSACTION_NOT_TYPE)); - msgInner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + if (null != store.getMessageStoreConfig() && store.getMessageStoreConfig().isTransRocksDBEnable() && !store.getMessageStoreConfig().isTransWriteOriginTransHalfEnable()) { + msgInner.setTopic(TransactionalMessageUtil.buildHalfTopicForRocksDB()); + } else { + msgInner.setTopic(TransactionalMessageUtil.buildHalfTopic()); + } msgInner.setQueueId(0); msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); return msgInner; diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java index 555ae4d2940..7edbc57b385 100644 --- a/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/queue/TransactionalMessageUtil.java @@ -38,10 +38,18 @@ public static String buildOpTopic() { return TopicValidator.RMQ_SYS_TRANS_OP_HALF_TOPIC; } + public static String buildOpTopicForRocksDB() { + return TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC; + } + public static String buildHalfTopic() { return TopicValidator.RMQ_SYS_TRANS_HALF_TOPIC; } + public static String buildHalfTopicForRocksDB() { + return TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC; + } + public static String buildConsumerGroup() { return MixAll.CID_SYS_RMQ_TRANS; } diff --git a/broker/src/main/java/org/apache/rocketmq/broker/transaction/rocksdb/TransactionalMessageRocksDBService.java b/broker/src/main/java/org/apache/rocketmq/broker/transaction/rocksdb/TransactionalMessageRocksDBService.java new file mode 100644 index 00000000000..389c75e4267 --- /dev/null +++ b/broker/src/main/java/org/apache/rocketmq/broker/transaction/rocksdb/TransactionalMessageRocksDBService.java @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.broker.transaction.rocksdb; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy; +import java.util.concurrent.TimeUnit; +import io.netty.channel.Channel; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageClientIDSetter; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.utils.ThreadUtils; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.protocol.header.CheckTransactionStateRequestHeader; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.transaction.TransRocksDBRecord; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TRANS_COLUMN_FAMILY; + +public class TransactionalMessageRocksDBService { + private static final Logger log = LoggerFactory.getLogger(LoggerName.TRANSACTION_LOGGER_NAME); + private static final int MAX_BATCH_SIZE_FROM_ROCKSDB = 2000; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + + private final MessageRocksDBStorage messageRocksDBStorage; + private final TransMessageRocksDBStore transMessageRocksDBStore; + private final MessageStore messageStore; + private final BrokerController brokerController; + + private TransStatusCheckService transStatusService; + private ExecutorService checkTranStatusTaskExecutor; + + public TransactionalMessageRocksDBService(final MessageStore messageStore, final BrokerController brokerController) { + this.messageStore = messageStore; + this.transMessageRocksDBStore = messageStore.getTransRocksDBStore(); + this.messageRocksDBStorage = transMessageRocksDBStore.getMessageRocksDBStorage(); + this.brokerController = brokerController; + } + + public void start() { + if (this.state == RUNNING) { + return; + } + initService(); + this.transStatusService.start(); + this.state = RUNNING; + log.info("TransactionalMessageRocksDBService start success"); + } + + private void initService() { + this.transStatusService = new TransStatusCheckService(); + this.checkTranStatusTaskExecutor = ThreadUtils.newThreadPoolExecutor( + 2, + 5, + 100, + TimeUnit.SECONDS, + new ArrayBlockingQueue<>(2000), + new ThreadFactoryImpl("Transaction-rocksdb-msg-check-thread", brokerController.getBrokerIdentity()), + new CallerRunsPolicy()); + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.transStatusService) { + this.transStatusService.shutdown(); + } + if (null != this.checkTranStatusTaskExecutor) { + this.checkTranStatusTaskExecutor.shutdown(); + } + this.state = SHUTDOWN; + log.info("TransactionalMessageRocksDBService shutdown success"); + } + + private void checkTransStatus() { + long count = 0; + byte[] lastKey = null; + while (true) { + try { + List trs = messageRocksDBStorage.scanRecordsForTrans(TRANS_COLUMN_FAMILY, MAX_BATCH_SIZE_FROM_ROCKSDB, lastKey); + if (CollectionUtils.isEmpty(trs)) { + log.info("TransactionalMessageRocksDBService checkTransStatus trs is empty"); + break; + } + count += trs.size(); + checkTransRecordsStatus(trs); + lastKey = trs.size() >= MAX_BATCH_SIZE_FROM_ROCKSDB ? trs.get(trs.size() - 1).getKeyBytes() : null; + if (null == lastKey) { + break; + } + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService checkTransStatus error, error: {}, count: {}", e.getMessage(), count); + break; + } + } + log.info("TransactionalMessageRocksDBService checkTransStatus count: {}", count); + } + + private void checkTransRecordsStatus(List trs) { + if (CollectionUtils.isEmpty(trs)) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus, trs is empty"); + return; + } + try { + List updateList = new ArrayList<>(); + for (TransRocksDBRecord halfRecord : trs) { + if (null == halfRecord) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus, halfRecord is null"); + continue; + } + try { + if (halfRecord.getCheckTimes() > brokerController.getBrokerConfig().getTransactionCheckMax()) { + halfRecord.setDelete(true); + updateList.add(halfRecord); + log.info("TransactionalMessageRocksDBService checkTransRecordsStatus checkTimes > {}, need delete, checkTimes: {}, msgId: {}", brokerController.getBrokerConfig().getTransactionCheckMax(), halfRecord.getCheckTimes(), halfRecord.getUniqKey()); + continue; + } + MessageExt msgExt = transMessageRocksDBStore.getMessage(halfRecord.getOffsetPy(), halfRecord.getSizePy()); + if (null == msgExt) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus, msgExt is null, offsetPy: {}, sizePy: {}", halfRecord.getOffsetPy(), halfRecord.getSizePy()); + halfRecord.setDelete(true); + updateList.add(halfRecord); + continue; + } + if (!isImmunityTimeExpired(msgExt)) { + continue; + } + resolveHalfMsg(msgExt); + halfRecord.setCheckTimes(halfRecord.getCheckTimes() + 1); + if (halfRecord.getCheckTimes() > brokerController.getBrokerConfig().getTransactionCheckMax()) { + halfRecord.setDelete(true); + log.info("TransactionalMessageRocksDBService checkTransRecordsStatus checkTimes > {}, need delete, checkTimes: {}, msgId: {}", brokerController.getBrokerConfig().getTransactionCheckMax(), halfRecord.getCheckTimes(), halfRecord.getUniqKey()); + } + updateList.add(halfRecord); + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus error : {}", e.getMessage()); + } + } + if (!CollectionUtils.isEmpty(updateList)) { + messageRocksDBStorage.updateRecordsForTrans(TRANS_COLUMN_FAMILY, updateList); + } + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService checkTransRecordsStatus error: {}", e.getMessage()); + } + } + + private boolean isImmunityTimeExpired(MessageExt msgExt) { + String immunityTimeStr = msgExt.getUserProperty(MessageConst.PROPERTY_CHECK_IMMUNITY_TIME_IN_SECONDS); + long immunityTime = brokerController.getBrokerConfig().getTransactionTimeOut(); + if (!StringUtils.isEmpty(immunityTimeStr)) { + try { + immunityTime = Long.parseLong(immunityTimeStr); + immunityTime *= 1000; + } catch (Exception e) { + log.error("parse immunityTimesStr error: {}, msgId: {}", e.getMessage(), msgExt.getMsgId()); + } + } + if ((System.currentTimeMillis() - msgExt.getBornTimestamp()) < immunityTime) { + return false; + } + return true; + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (TransactionalMessageRocksDBService.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) TransactionalMessageRocksDBService.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private void resolveHalfMsg(final MessageExt msgExt) { + if (checkTranStatusTaskExecutor != null) { + checkTranStatusTaskExecutor.execute(new Runnable() { + @Override + public void run() { + try { + sendCheckMessage(msgExt); + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService Send check message error: {}, msgId: {}", e.getMessage(), msgExt.getMsgId()); + } + } + }); + } else { + log.error("TransactionalMessageRocksDBService checkTranStatusTaskExecutor not init, msgId: {}", msgExt.getMsgId()); + } + } + + private void sendCheckMessage(MessageExt msgExt) { + if (null == msgExt) { + log.info("TransactionalMessageRocksDBService sendCheckMessage msgExt is null"); + return; + } + try { + CheckTransactionStateRequestHeader checkTransactionStateRequestHeader = new CheckTransactionStateRequestHeader(); + checkTransactionStateRequestHeader.setTopic(msgExt.getTopic()); + checkTransactionStateRequestHeader.setCommitLogOffset(msgExt.getCommitLogOffset()); + checkTransactionStateRequestHeader.setOffsetMsgId(msgExt.getMsgId()); + checkTransactionStateRequestHeader.setMsgId(MessageClientIDSetter.getUniqID(msgExt)); + checkTransactionStateRequestHeader.setTransactionId(checkTransactionStateRequestHeader.getMsgId()); + checkTransactionStateRequestHeader.setTranStateTableOffset(msgExt.getQueueOffset()); + checkTransactionStateRequestHeader.setBrokerName(brokerController.getBrokerConfig().getBrokerName()); + msgExt.setTopic(msgExt.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC)); + msgExt.setQueueId(Integer.parseInt(msgExt.getUserProperty(MessageConst.PROPERTY_REAL_QUEUE_ID))); + msgExt.setStoreSize(0); + String groupId = msgExt.getProperty(MessageConst.PROPERTY_PRODUCER_GROUP); + Channel channel = brokerController.getProducerManager().getAvailableChannel(groupId); + if (channel != null) { + brokerController.getBroker2Client().checkProducerTransactionState(groupId, channel, checkTransactionStateRequestHeader, msgExt); + } else { + log.warn("TransactionalMessageRocksDBService checkProducerTransactionState failed, channel is null. groupId: {}, msgId: {}", groupId, msgExt.getMsgId()); + } + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService sendCheckMessage error: {}, msgId: {}", e.getMessage(), msgExt.getMsgId()); + } + } + + private class TransStatusCheckService extends ServiceThread { + private final Logger log = TransactionalMessageRocksDBService.log; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped()) { + try { + long begin = System.currentTimeMillis(); + checkTransStatus(); + log.info("TransactionalMessageRocksDBService ScanTransAndStatusCheckService check trans status, check cost: {}", System.currentTimeMillis() - begin); + waitForRunning(brokerController.getBrokerConfig().getTransactionCheckInterval()); + } catch (Exception e) { + log.error("TransactionalMessageRocksDBService ScanTransAndStatusCheckService error: {}", e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + } +} diff --git a/broker/src/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/RocksDBConfigManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/RocksDBConfigManagerTest.java new file mode 100644 index 00000000000..1832902a768 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/RocksDBConfigManagerTest.java @@ -0,0 +1,67 @@ +/* + * 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; + +import org.apache.rocketmq.broker.config.v1.RocksDBConfigManager; +import org.apache.rocketmq.common.config.ConfigRocksDBStorage; +import org.apache.rocketmq.remoting.protocol.DataVersion; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.powermock.api.mockito.PowerMockito.mock; + +public class RocksDBConfigManagerTest { + + private ConfigRocksDBStorage configRocksDBStorage; + + private RocksDBConfigManager rocksDBConfigManager; + + @Before + public void setUp() throws IllegalAccessException { + configRocksDBStorage = mock(ConfigRocksDBStorage.class); + rocksDBConfigManager = spy(new RocksDBConfigManager("testPath", 1000L, null)); + rocksDBConfigManager.configRocksDBStorage = configRocksDBStorage; + } + + @Test + public void testLoadDataVersion() throws Exception { + DataVersion expected = new DataVersion(); + expected.nextVersion(); + + when(rocksDBConfigManager.getKvDataVersion()).thenReturn(expected); + + boolean result = rocksDBConfigManager.loadDataVersion(); + + assertTrue(result); + assertEquals(expected.getCounter().get(), rocksDBConfigManager.getKvDataVersion().getCounter().get()); + assertEquals(expected.getTimestamp(), rocksDBConfigManager.getKvDataVersion().getTimestamp()); + } + + @Test + public void testUpdateKvDataVersion() throws Exception { + rocksDBConfigManager.updateKvDataVersion(); + + verify(rocksDBConfigManager, times(1)).updateKvDataVersion(); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/client/ConsumerManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/client/ConsumerManagerTest.java index 1b8293159d3..5f2e96015db 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/client/ConsumerManagerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/client/ConsumerManagerTest.java @@ -20,6 +20,7 @@ import com.google.common.collect.ImmutableSet; import io.netty.channel.Channel; import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.broker.client.net.Broker2Client; import org.apache.rocketmq.broker.filter.ConsumerFilterManager; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; @@ -214,6 +215,9 @@ public void removeExpireConsumerGroupInfo() { @Test public void testRegisterConsumerWithoutSub() { + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + Broker2Client broker2Client = mock(Broker2Client.class); + when(brokerController.getBroker2Client()).thenReturn(broker2Client); ConsumerGroupInfo groupInfo = new ConsumerGroupInfo(GROUP, CONSUME_PASSIVELY, MessageModel.CLUSTERING, ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); SubscriptionData subscriptionData = new SubscriptionData(TOPIC, "*"); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/config/v1/RocksDBSubscriptionGroupManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/config/v1/RocksDBSubscriptionGroupManagerTest.java new file mode 100644 index 00000000000..98d0e342191 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/config/v1/RocksDBSubscriptionGroupManagerTest.java @@ -0,0 +1,120 @@ +/* + * 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.config.v1; + +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ConcurrentMap; + +import static org.junit.Assert.assertTrue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class RocksDBSubscriptionGroupManagerTest { + + @Mock + private BrokerController brokerController; + + @Mock + private RocksDBConfigManager rocksDBConfigManager; + + private RocksDBSubscriptionGroupManager rocksDBSubscriptionGroupManager; + + @Mock + private MessageStoreConfig messageStoreConfig; + + @Before + public void init() throws IllegalAccessException { + when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); + when(messageStoreConfig.getMemTableFlushIntervalMs()).thenReturn(1000L); + when(messageStoreConfig.getRocksdbCompressionType()).thenReturn("LZ4_COMPRESSION"); + when(messageStoreConfig.getStorePathRootDir()).thenReturn("/"); + BrokerConfig brokerConfig = mock(BrokerConfig.class); + when(brokerConfig.isUseSingleRocksDBForAllConfigs()).thenReturn(true); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + rocksDBSubscriptionGroupManager = new RocksDBSubscriptionGroupManager(brokerController); + FieldUtils.writeDeclaredField(rocksDBSubscriptionGroupManager, "rocksDBConfigManager", rocksDBConfigManager, true); + } + + @Test + public void testPutSubscriptionGroupConfig() { + SubscriptionGroupConfig newConfig = new SubscriptionGroupConfig(); + newConfig.setGroupName("group"); + SubscriptionGroupConfig oldConfig = new SubscriptionGroupConfig(); + oldConfig.setGroupName("group"); + rocksDBSubscriptionGroupManager.getSubscriptionGroupTable().put("group", oldConfig); + + assertEquals(oldConfig, rocksDBSubscriptionGroupManager.putSubscriptionGroupConfig(newConfig)); + assertEquals(newConfig, rocksDBSubscriptionGroupManager.getSubscriptionGroupTable().get("group")); + } + + @Test + public void testPutSubscriptionGroupConfigIfAbsent() { + SubscriptionGroupConfig newConfig = new SubscriptionGroupConfig(); + newConfig.setGroupName("group"); + SubscriptionGroupConfig oldConfig = new SubscriptionGroupConfig(); + oldConfig.setGroupName("group"); + + assertNull(rocksDBSubscriptionGroupManager.putSubscriptionGroupConfigIfAbsent(newConfig)); + assertEquals(newConfig, rocksDBSubscriptionGroupManager.getSubscriptionGroupTable().get("group")); + } + + @Test + public void testDecodeForbidden() { + String forbiddenGroupName = "group"; + String bodyJson = "{\"topic1\":1,\"topic2\":2}"; + byte[] key = forbiddenGroupName.getBytes(StandardCharsets.UTF_8); + byte[] body = bodyJson.getBytes(StandardCharsets.UTF_8); + + rocksDBSubscriptionGroupManager.decodeForbidden(key, body); + ConcurrentMap> forbiddenTable = rocksDBSubscriptionGroupManager.getForbiddenTable(); + assertTrue(forbiddenTable.containsKey(forbiddenGroupName)); + + ConcurrentMap forbiddenGroup = forbiddenTable.get(forbiddenGroupName); + assertEquals(2, forbiddenGroup.size()); + assertEquals(Integer.valueOf(1), forbiddenGroup.get("topic1")); + assertEquals(Integer.valueOf(2), forbiddenGroup.get("topic2")); + } + + @Test + public void testDecodeSubscriptionGroup() { + String groupName = "group"; + String bodyJson = "{\"groupName\":\"group\",\"consumeEnable\":true}"; + byte[] key = groupName.getBytes(StandardCharsets.UTF_8); + byte[] body = bodyJson.getBytes(StandardCharsets.UTF_8); + + rocksDBSubscriptionGroupManager.decodeSubscriptionGroup(key, body); + ConcurrentMap subscriptionGroupTable = rocksDBSubscriptionGroupManager.getSubscriptionGroupTable(); + assertEquals(1, subscriptionGroupTable.size()); + SubscriptionGroupConfig config = subscriptionGroupTable.get(groupName); + assertEquals(groupName, config.getGroupName()); + assertTrue(config.isConsumeEnable()); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManagerTest.java new file mode 100644 index 00000000000..b2b742fb2f6 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/config/v1/RocksDBTopicConfigManagerTest.java @@ -0,0 +1,97 @@ +/* + * 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.config.v1; + +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.broker.BrokerController; +import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.nio.charset.StandardCharsets; +import java.util.concurrent.ConcurrentMap; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class RocksDBTopicConfigManagerTest { + + @Mock + private BrokerController brokerController; + + @Mock + private MessageStoreConfig messageStoreConfig; + + @Mock + private RocksDBConfigManager rocksDBConfigManager; + + private RocksDBTopicConfigManager rocksDBTopicConfigManager; + + @Before + public void init() throws Exception { + when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); + when(messageStoreConfig.getMemTableFlushIntervalMs()).thenReturn(1000L); + when(messageStoreConfig.getRocksdbCompressionType()).thenReturn("LZ4_COMPRESSION"); + when(messageStoreConfig.getStorePathRootDir()).thenReturn("/"); + BrokerConfig brokerConfig = mock(BrokerConfig.class); + when(brokerConfig.isUseSingleRocksDBForAllConfigs()).thenReturn(true); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + rocksDBTopicConfigManager = new RocksDBTopicConfigManager(brokerController); + FieldUtils.writeDeclaredField(rocksDBTopicConfigManager, "rocksDBConfigManager", rocksDBConfigManager, true); + } + + @Test + public void testDecodeTopicConfig() { + String topicName = "testTopic"; + String topicConfigJson = "{\"topicName\":\"testTopic\",\"readQueueNums\":10,\"writeQueueNums\":10}"; + byte[] key = topicName.getBytes(StandardCharsets.UTF_8); + byte[] body = topicConfigJson.getBytes(StandardCharsets.UTF_8); + + rocksDBTopicConfigManager.decodeTopicConfig(key, body); + + ConcurrentMap topicConfigTable = rocksDBTopicConfigManager.getTopicConfigTable(); + assertNotNull(topicConfigTable); + assertEquals(1, topicConfigTable.size()); + TopicConfig topicConfig = topicConfigTable.get(topicName); + assertNotNull(topicConfig); + assertEquals(topicName, topicConfig.getTopicName()); + assertEquals(10, topicConfig.getReadQueueNums()); + assertEquals(10, topicConfig.getWriteQueueNums()); + } + + @Test + public void testPutTopicConfig() throws Exception { + TopicConfig newTopicConfig = new TopicConfig("newTopic"); + newTopicConfig.setReadQueueNums(10); + newTopicConfig.setWriteQueueNums(10); + + assertNull(rocksDBTopicConfigManager.putTopicConfig(newTopicConfig)); + verify(rocksDBConfigManager, times(1)).put(any(byte[].class), any(byte[].class)); + } +} 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..f1168394584 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/AbstractLiteLifecycleManagerTest.java @@ -0,0 +1,280 @@ +/* + * 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.anyInt; +import static org.mockito.Mockito.atLeastOnce; +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 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..3f86ca1502c --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteEventDispatcherTest.java @@ -0,0 +1,580 @@ +/* + * 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 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 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 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..ce177f14537 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteLifecycleManagerTest.java @@ -0,0 +1,204 @@ +/* + * 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.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)); + } + } + + @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..9fda97e45fc --- /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.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class LiteShardingImplTest { + + @Mock + private BrokerController brokerController; + + @Mock + private TopicRouteInfoManager topicRouteInfoManager; + + private LiteShardingImpl liteSharding; + + @BeforeEach + void setUp() { + liteSharding = new LiteShardingImpl(brokerController, topicRouteInfoManager); + } + + /** + * Test normal case: multiple MessageQueues, verify consistent hash selects correct brokerName + */ + @Test + 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 + 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 + 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..8e6c199158e --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/lite/LiteSubscriptionRegistryImplTest.java @@ -0,0 +1,774 @@ +/* + * 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.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.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +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.anyBoolean; +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; + +class LiteSubscriptionRegistryImplTest { + + private LiteSubscriptionRegistryImpl registry; + private LiteCtlListener mockListener; + private AbstractLiteLifecycleManager mockLifecycleManager; + private BrokerConfig mockBrokerConfig; + private SubscriptionGroupManager mockSubscriptionGroupManager; + + @BeforeEach + public void setUp() { + BrokerController mockBrokerController = mock(BrokerController.class); + mockLifecycleManager = mock(AbstractLiteLifecycleManager.class); + mockBrokerConfig = mock(BrokerConfig.class); + mockSubscriptionGroupManager = mock(SubscriptionGroupManager.class); + + when(mockBrokerController.getBrokerConfig()).thenReturn(mockBrokerConfig); + when(mockBrokerController.getSubscriptionGroupManager()).thenReturn(mockSubscriptionGroupManager); + 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, false); + + 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); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group)).thenReturn(subscriptionGroupConfig); + + // Add existing client + registry.addPartialSubscription(existingClientId, group, topic, liteTopicSet, false); + + // 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, true); + + // 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, false); + } + + @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, false); + + // Add new client in non-exclusive mode + Set newLiteTopicSet = new HashSet<>(); + newLiteTopicSet.add(liteTopic); + registry.addPartialSubscription(newClientId, group, topic, newLiteTopicSet, false); + + // 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(), anyBoolean()); + } + + @Test + public void testAddPartialSubscription_WithEmptyLiteTopicSet() { + String clientId = "client1"; + String group = "group1"; + String topic = "topic1"; + Set liteTopicSet = new HashSet<>(); + + registry.addPartialSubscription(clientId, group, topic, liteTopicSet, false); + + 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, false); + + 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); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group1)).thenReturn(subscriptionGroupConfig1); + + SubscriptionGroupConfig subscriptionGroupConfig2 = new SubscriptionGroupConfig(); + subscriptionGroupConfig2.setGroupName(group2); + when(mockSubscriptionGroupManager.findSubscriptionGroupConfig(group2)).thenReturn(subscriptionGroupConfig2); + + // Add first client + registry.addPartialSubscription(client1, group1, topic, liteTopicSet, false); + + // Add second client + registry.addPartialSubscription(client2, group2, topic, liteTopicSet, false); + + // 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, true); + + // 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, false); // Same group client1 removed + verify(mockListener, never()).onUnregister(client2, group2, liteTopic, false); // 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, false); + + // 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, false); + }); + } + + // 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, false); + + // 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, false); + verify(mockListener, never()).onUnregister(clientId, group, liteTopic2, false); + } + + // 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, false); + + // 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, false); + + // 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, false); + } + + // 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, false); + + // 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, false); + 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, false); + + 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, false); + assertEquals(1, registry.getActiveSubscriptionNum()); + + // Add second client + registry.addPartialSubscription(clientId2, group, topic, liteTopicSet2, false); + 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"), eq(false)); + verify(mockListener, times(1)).onUnregister(eq(clientId), eq(group), eq("lmq2"), eq(false)); + 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(), anyBoolean()); + } + + // 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()); + } +} \ 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/longpolling/PopLongPollingServiceTest.java b/broker/src/test/java/org/apache/rocketmq/broker/longpolling/PopLongPollingServiceTest.java index 3547687a6de..23dcf5c2fda 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/longpolling/PopLongPollingServiceTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/longpolling/PopLongPollingServiceTest.java @@ -20,11 +20,18 @@ import com.github.benmanes.caffeine.cache.Caffeine; import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.KeyBuilder; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.remoting.netty.NettyRequestProcessor; import org.apache.rocketmq.remoting.protocol.RemotingCommand; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; @@ -36,11 +43,6 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentSkipListSet; -import java.util.concurrent.ExecutorService; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -87,6 +89,27 @@ public void testNotifyMessageArrivingWithRetryTopic() { verify(popLongPollingService, times(1)).notifyMessageArrivingWithRetryTopic(defaultTopic, queueId, -1L, null, 0L, null, null); } + @Test + public void testNotifyMessageArrivingFromRetry() { + int queueId = -1; + String group = "group"; + String pullRetryTopic = MixAll.getRetryTopic(group); + String popRetryTopicV1 = KeyBuilder.buildPopRetryTopic(defaultTopic, group, false); + String popRetryTopicV2 = KeyBuilder.buildPopRetryTopic(defaultTopic, group, true); + + Map properties = new HashMap<>(); + properties.putIfAbsent(MessageConst.PROPERTY_ORIGIN_GROUP, group); + // pull retry + popLongPollingService.notifyMessageArrivingWithRetryTopic(pullRetryTopic, queueId, queueId, -1L, 0L, null, properties); + verify(popLongPollingService, times(0)).notifyMessageArriving(defaultTopic, queueId, group, true, -1L, 0L, null, properties, null); + // pop retry v1 + popLongPollingService.notifyMessageArrivingWithRetryTopic(popRetryTopicV1, queueId, queueId, -1L, 0L, null, properties); + verify(popLongPollingService, times(1)).notifyMessageArriving(defaultTopic, queueId, group, true, -1L, 0L, null, properties, null); + // pop retry v2 + popLongPollingService.notifyMessageArrivingWithRetryTopic(popRetryTopicV2, queueId, queueId, -1L, 0L, null, properties); + verify(popLongPollingService, times(2)).notifyMessageArriving(defaultTopic, queueId, group, true, -1L, 0L, null, properties, null); + } + @Test public void testNotifyMessageArriving() { int queueId = 0; 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..c860bc445cb 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,39 +17,48 @@ 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(); @@ -64,37 +73,27 @@ public void init() { @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); } } 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/pagecache/QueryMessageTransferTest.java b/broker/src/test/java/org/apache/rocketmq/broker/pagecache/QueryMessageTransferTest.java new file mode 100644 index 00000000000..a10dd4efca0 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/pagecache/QueryMessageTransferTest.java @@ -0,0 +1,139 @@ +/* + * 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.pagecache; + +import org.apache.rocketmq.store.QueryMessageResult; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; +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 QueryMessageTransferTest { + + @Mock + private WritableByteChannel writableByteChannel; + + @Mock + private QueryMessageResult queryMessageResult; + + private QueryMessageTransfer queryMessageTransfer; + + private ByteBuffer byteBufferHeader; + + private ByteBuffer bb1; + + private ByteBuffer bb2; + + @Before + public void init() { + byteBufferHeader = ByteBuffer.allocate(4); + byteBufferHeader.putInt(1); + byteBufferHeader.flip(); + + bb1 = ByteBuffer.allocate(4); + bb1.putInt(2); + bb1.flip(); + + bb2 = ByteBuffer.allocate(4); + bb2.putInt(3); + bb2.flip(); + + when(queryMessageResult.getMessageBufferList()).thenReturn(Arrays.asList(bb1, bb2)); + + queryMessageTransfer = new QueryMessageTransfer(byteBufferHeader, queryMessageResult); + } + + @Test + public void testPosition_WithHeaderAndMessageBuffers() { + byteBufferHeader.position(2); + bb1.position(1); + bb2.position(3); + + long actual = queryMessageTransfer.position(); + + long expected = byteBufferHeader.position() + bb1.position() + bb2.position(); + assertEquals(expected, actual); + } + + @Test + public void testPosition_WithHeaderOnly() { + byteBufferHeader.position(2); + + when(queryMessageResult.getMessageBufferList()).thenReturn(new ArrayList<>()); + + long actual = queryMessageTransfer.position(); + + long expected = byteBufferHeader.position(); + assertEquals(expected, actual); + } + + @Test + public void testPosition_WithMessageBuffersOnly() { + byteBufferHeader.clear(); + byteBufferHeader.flip(); + + bb1.position(1); + bb2.position(3); + + long actual = queryMessageTransfer.position(); + + long expected = bb1.position() + bb2.position(); + assertEquals(expected, actual); + } + + @Test + public void testTransferTo_OnlyHeaderData() throws Exception { + bb1.clear(); + bb2.clear(); + + when(writableByteChannel.write(byteBufferHeader)).thenReturn(4); + + long actual = queryMessageTransfer.transferTo(writableByteChannel, 0); + + assertEquals(4, actual); + verify(writableByteChannel, times(1)).write(byteBufferHeader); + verify(writableByteChannel, never()).write(bb1); + verify(writableByteChannel, never()).write(bb2); + } + + @Test + public void testTransferTo_OnlyMessageBuffersData() throws Exception { + byteBufferHeader.clear(); + byteBufferHeader.flip(); + + when(writableByteChannel.write(bb1)).thenReturn(4); + + long actual = queryMessageTransfer.transferTo(writableByteChannel, 0); + + assertEquals(4, actual); + verify(writableByteChannel, never()).write(byteBufferHeader); + verify(writableByteChannel, times(1)).write(bb1); + } +} diff --git a/broker/src/test/java/org/apache/rocketmq/broker/pop/PopConsumerServiceTest.java b/broker/src/test/java/org/apache/rocketmq/broker/pop/PopConsumerServiceTest.java index 9c23a8625eb..db5f60fb17a 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/pop/PopConsumerServiceTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/pop/PopConsumerServiceTest.java @@ -35,7 +35,7 @@ import org.apache.rocketmq.broker.failover.EscapeBridge; import org.apache.rocketmq.broker.longpolling.PopLongPollingService; import org.apache.rocketmq.broker.offset.ConsumerOffsetManager; -import org.apache.rocketmq.broker.offset.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.pop.orderly.ConsumerOrderInfoManager; import org.apache.rocketmq.broker.processor.PopMessageProcessor; import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.BrokerConfig; @@ -233,7 +233,7 @@ public void getMessageAsyncTest() throws Exception { // fifo block PopConsumerContext context = new PopConsumerContext( clientHost, System.currentTimeMillis(), 20000, groupId, false, ConsumeInitMode.MIN, attemptId); - consumerService.setFifoBlocked(context, groupId, topicId, queueId, Collections.singletonList(100L)); + consumerService.setFifoBlocked(context, groupId, topicId, queueId, Collections.singletonList(100L), resetGetMessageResult); Mockito.when(brokerController.getConsumerOrderInfoManager() .checkBlock(anyString(), anyString(), anyString(), anyInt(), anyLong())).thenReturn(true); Assert.assertTrue(consumerService.isFifoBlocked(context, groupId, topicId, queueId)); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManagerLockFreeNotifyTest.java b/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerLockFreeNotifyTest.java similarity index 95% rename from broker/src/test/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManagerLockFreeNotifyTest.java rename to broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerLockFreeNotifyTest.java index 1fdf454d5e0..d3c0df987c6 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManagerLockFreeNotifyTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerLockFreeNotifyTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.rocketmq.broker.offset; +package org.apache.rocketmq.broker.pop.orderly; import java.time.Duration; import java.util.concurrent.atomic.AtomicBoolean; @@ -43,7 +43,7 @@ public class ConsumerOrderInfoManagerLockFreeNotifyTest { private static final int QUEUE_ID_0 = 0; private long popTime; - private ConsumerOrderInfoManager consumerOrderInfoManager; + private QueueLevelConsumerManager consumerOrderInfoManager; private AtomicBoolean notified; private final BrokerConfig brokerConfig = new BrokerConfig(); @@ -61,7 +61,7 @@ public void before() throws ConsumeQueueException { return null; }).when(popMessageProcessor).notifyLongPollingRequestIfNeed(anyString(), anyString(), anyInt()); - consumerOrderInfoManager = new ConsumerOrderInfoManager(brokerController); + consumerOrderInfoManager = new QueueLevelConsumerManager(brokerController); popTime = System.currentTimeMillis(); } @@ -158,7 +158,7 @@ public void testConsumeTheChangeInvisibleShorter() { @Test public void testRecover() { - ConsumerOrderInfoManager savedConsumerOrderInfoManager = new ConsumerOrderInfoManager(); + QueueLevelConsumerManager savedConsumerOrderInfoManager = new QueueLevelConsumerManager(); savedConsumerOrderInfoManager.update( null, false, diff --git a/broker/src/test/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java similarity index 84% rename from broker/src/test/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManagerTest.java rename to broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java index 4414eda54e9..a5a5dfc2357 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/offset/ConsumerOrderInfoManagerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/pop/orderly/ConsumerOrderInfoManagerTest.java @@ -15,9 +15,13 @@ * limitations under the License. */ -package org.apache.rocketmq.broker.offset; +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; @@ -51,11 +55,11 @@ public class ConsumerOrderInfoManagerTest { private static final int QUEUE_ID_1 = 1; private long popTime; - private ConsumerOrderInfoManager consumerOrderInfoManager; + private QueueLevelConsumerManager consumerOrderInfoManager; @Before public void before() { - consumerOrderInfoManager = new ConsumerOrderInfoManager(); + consumerOrderInfoManager = new QueueLevelConsumerManager(); popTime = System.currentTimeMillis(); } @@ -387,7 +391,7 @@ public void testAutoCleanAndEncode() { TopicConfig topicConfig = new TopicConfig(TOPIC); when(topicConfigManager.selectTopicConfig(eq(TOPIC))).thenReturn(topicConfig); - ConsumerOrderInfoManager consumerOrderInfoManager = new ConsumerOrderInfoManager(brokerController); + QueueLevelConsumerManager consumerOrderInfoManager = new QueueLevelConsumerManager(brokerController); { consumerOrderInfoManager.update(null, false, @@ -444,7 +448,7 @@ public void testAutoCleanAndEncode() { consumerOrderInfoManager.autoClean(); assertEquals(1, consumerOrderInfoManager.getTable().size()); - for (ConcurrentHashMap orderInfoMap : consumerOrderInfoManager.getTable().values()) { + for (ConcurrentHashMap orderInfoMap : consumerOrderInfoManager.getTable().values()) { assertEquals(1, orderInfoMap.size()); assertNotNull(orderInfoMap.get(QUEUE_ID_0)); break; @@ -453,13 +457,13 @@ public void testAutoCleanAndEncode() { } private void assertEncodeAndDecode() { - ConsumerOrderInfoManager.OrderInfo prevOrderInfo = consumerOrderInfoManager.getTable().values().stream().findFirst() + QueueLevelConsumerManager.OrderInfo prevOrderInfo = consumerOrderInfoManager.getTable().values().stream().findFirst() .get().get(QUEUE_ID_0); String dataEncoded = consumerOrderInfoManager.encode(); consumerOrderInfoManager.decode(dataEncoded); - ConsumerOrderInfoManager.OrderInfo newOrderInfo = consumerOrderInfoManager.getTable().values().stream().findFirst() + QueueLevelConsumerManager.OrderInfo newOrderInfo = consumerOrderInfoManager.getTable().values().stream().findFirst() .get().get(QUEUE_ID_0); assertNotSame(prevOrderInfo, newOrderInfo); @@ -482,7 +486,7 @@ public void testLoadFromOldVersionOrderInfoData() { 1, Lists.newArrayList(2L, 3L, 4L), new StringBuilder()); - ConsumerOrderInfoManager.OrderInfo orderInfo = consumerOrderInfoManager.getTable().values().stream().findFirst() + QueueLevelConsumerManager.OrderInfo orderInfo = consumerOrderInfoManager.getTable().values().stream().findFirst() .get().get(QUEUE_ID_0); orderInfo.setInvisibleTime(null); @@ -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 1bf99eadfba..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 @@ -16,7 +16,8 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -37,11 +38,12 @@ import org.apache.rocketmq.broker.client.ConsumerGroupInfo; import org.apache.rocketmq.broker.client.ConsumerManager; import org.apache.rocketmq.broker.client.net.Broker2Client; +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.config.v1.RocksDBSubscriptionGroupManager; -import org.apache.rocketmq.broker.config.v1.RocksDBTopicConfigManager; import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.BrokerConfig; @@ -56,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; @@ -80,6 +83,7 @@ import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper; import org.apache.rocketmq.remoting.protocol.body.UnlockBatchRequestBody; import org.apache.rocketmq.remoting.protocol.body.UserInfo; +import org.apache.rocketmq.remoting.protocol.header.CheckRocksdbCqWriteProgressRequestHeader; import org.apache.rocketmq.remoting.protocol.header.CreateAclRequestHeader; import org.apache.rocketmq.remoting.protocol.header.CreateTopicRequestHeader; import org.apache.rocketmq.remoting.protocol.header.CreateUserRequestHeader; @@ -97,11 +101,13 @@ import org.apache.rocketmq.remoting.protocol.header.GetEarliestMsgStoretimeRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMaxOffsetRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetMinOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.GetSubscriptionGroupConfigRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetTopicConfigRequestHeader; import org.apache.rocketmq.remoting.protocol.header.GetUserRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ListAclsRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ListUsersRequestHeader; import org.apache.rocketmq.remoting.protocol.header.NotifyMinBrokerIdChangeRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.QueryConsumeQueueRequestHeader; import org.apache.rocketmq.remoting.protocol.header.QueryCorrectionOffsetHeader; import org.apache.rocketmq.remoting.protocol.header.QuerySubscriptionByConsumerRequestHeader; import org.apache.rocketmq.remoting.protocol.header.QueryTopicConsumeByWhoRequestHeader; @@ -110,10 +116,13 @@ 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; import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel; +import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.statictopic.TopicConfigAndQueueMapping; import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.store.CommitLog; import org.apache.rocketmq.store.DefaultMessageStore; @@ -121,6 +130,7 @@ import org.apache.rocketmq.store.SelectMappedBufferResult; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.logfile.DefaultMappedFile; +import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.stats.BrokerStats; import org.apache.rocketmq.store.timer.TimerCheckpoint; import org.apache.rocketmq.store.timer.TimerMessageStore; @@ -159,6 +169,8 @@ import java.util.concurrent.atomic.LongAdder; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -703,13 +715,55 @@ 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 public void testGetMaxOffset() throws Exception { messageStore = mock(MessageStore.class); @@ -889,7 +943,7 @@ public void testGetAllConsumerOffset() throws RemotingCommandException { consumerOffsetManager = mock(ConsumerOffsetManager.class); when(brokerController.getConsumerOffsetManager()).thenReturn(consumerOffsetManager); ConsumerOffsetManager consumerOffset = new ConsumerOffsetManager(); - when(consumerOffsetManager.encode()).thenReturn(JSON.toJSONString(consumerOffset, false)); + when(consumerOffsetManager.encode()).thenReturn(JSON.toJSONString(consumerOffset)); RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ALL_CONSUMER_OFFSET, null); RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); @@ -1500,6 +1554,73 @@ public void testResetMasterFlushOffset() throws RemotingCommandException { assertThat(response.getCode()).isEqualTo(ResponseCode.SUCCESS); } + @Test + public void testGetSubscriptionGroup() throws RemotingCommandException { + brokerController.getSubscriptionGroupManager().getSubscriptionGroupTable().put("group", new SubscriptionGroupConfig()); + GetSubscriptionGroupConfigRequestHeader requestHeader = new GetSubscriptionGroupConfigRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_SUBSCRIPTIONGROUP_CONFIG, requestHeader); + requestHeader.setGroup("group"); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + } + + @Test + public void testCheckRocksdbCqWriteProgress() throws RemotingCommandException { + CheckRocksdbCqWriteProgressRequestHeader requestHeader = new CheckRocksdbCqWriteProgressRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.CHECK_ROCKSDB_CQ_WRITE_PROGRESS, requestHeader); + requestHeader.setTopic("topic"); + request.makeCustomHeaderToNet(); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + } + + @Test + public void testQueryConsumeQueue() throws RemotingCommandException { + messageStore = mock(MessageStore.class); + ConsumeQueueInterface consumeQueue = mock(ConsumeQueueInterface.class); + when(consumeQueue.getMinOffsetInQueue()).thenReturn(0L); + when(consumeQueue.getMaxOffsetInQueue()).thenReturn(1L); + when(messageStore.getConsumeQueue(anyString(), anyInt())).thenReturn(consumeQueue); + when(brokerController.getMessageStore()).thenReturn(messageStore); + QueryConsumeQueueRequestHeader requestHeader = new QueryConsumeQueueRequestHeader(); + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.QUERY_CONSUME_QUEUE, requestHeader); + requestHeader.setTopic("topic"); + requestHeader.setQueueId(0); + requestHeader.setConsumerGroup("testGroup"); + request.makeCustomHeaderToNet(); + SubscriptionData subscriptionData = mock(SubscriptionData.class); + when(brokerController.getConsumerManager()).thenReturn(consumerManager); + when(consumerManager.findSubscriptionData(any(), any())).thenReturn(subscriptionData); + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + } + + @Test + public void testProcessRequest_GetTopicConfig() throws Exception { + GetTopicConfigRequestHeader requestHeader = new GetTopicConfigRequestHeader(); + requestHeader.setTopic("testTopic"); + + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_TOPIC_CONFIG, requestHeader); + request.makeCustomHeaderToNet(); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setTopicName("testTopic"); + TopicConfigManager topicConfigManager = mock(TopicConfigManager.class); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(topicConfigManager.selectTopicConfig("testTopic")) + .thenReturn(topicConfig); + + RemotingCommand response = adminBrokerProcessor.processRequest(handlerContext, request); + + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + + String responseBody = new String(response.getBody(), StandardCharsets.UTF_8); + TopicConfigAndQueueMapping result = JSONObject.parseObject(responseBody, TopicConfigAndQueueMapping.class); + assertEquals("testTopic", result.getTopicName()); + } + private ResetOffsetRequestHeader createRequestHeader(String topic,String group,long timestamp,boolean force,long offset,int queueId) { ResetOffsetRequestHeader requestHeader = new ResetOffsetRequestHeader(); requestHeader.setTopic(topic); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessorTest.java index ca2529e3190..7afd338dcaa 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/ChangeInvisibleTimeProcessorTest.java @@ -18,18 +18,15 @@ import io.netty.channel.Channel; import io.netty.channel.ChannelHandlerContext; -import java.lang.reflect.Field; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.net.Broker2Client; import org.apache.rocketmq.broker.failover.EscapeBridge; import org.apache.rocketmq.broker.metrics.BrokerMetricsManager; -import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.apache.rocketmq.broker.metrics.PopMetricsManager; +import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.TopicConfig; -import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.remoting.exception.RemotingCommandException; @@ -41,13 +38,16 @@ import org.apache.rocketmq.remoting.protocol.ResponseCode; import org.apache.rocketmq.remoting.protocol.header.ChangeInvisibleTimeRequestHeader; import org.apache.rocketmq.remoting.protocol.header.ExtraInfoUtil; +import org.apache.rocketmq.remoting.protocol.heartbeat.ConsumerData; import org.apache.rocketmq.store.AppendMessageResult; import org.apache.rocketmq.store.AppendMessageStatus; import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.exception.ConsumeQueueException; +import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -55,10 +55,18 @@ import org.mockito.Spy; import org.mockito.junit.MockitoJUnitRunner; +import java.lang.reflect.Field; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.rocketmq.broker.processor.PullMessageProcessorTest.createConsumerData; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -118,6 +126,17 @@ public void init() throws IllegalAccessException, NoSuchFieldException { PopBufferMergeService popBufferMergeService = mock(PopBufferMergeService.class); when(brokerController.getPopMessageProcessor()).thenReturn(popMessageProcessor); when(popMessageProcessor.getPopBufferMergeService()).thenReturn(popBufferMergeService); + + ConsumerData consumerData = createConsumerData(group, topic); + clientInfo = new ClientChannelInfo(channel, "127.0.0.1", LanguageCode.JAVA, 0); + brokerController.getConsumerManager().registerConsumer( + consumerData.getGroupName(), + clientInfo, + consumerData.getConsumeType(), + consumerData.getMessageModel(), + consumerData.getConsumeFromWhere(), + consumerData.getSubscriptionDataSet(), + false); clientInfo = new ClientChannelInfo(channel, "127.0.0.1", LanguageCode.JAVA, 0); changeInvisibleTimeProcessor = new ChangeInvisibleTimeProcessor(brokerController); @@ -177,4 +196,58 @@ public void testProcessRequest_NoMessage() throws RemotingCommandException, Cons assertThat(responseToReturn.getCode()).isEqualTo(ResponseCode.NO_MESSAGE); assertThat(responseToReturn.getOpaque()).isEqualTo(request.getOpaque()); } + + @Test + public void testProcessRequestAsync_JsonParsing() throws Exception { + Channel mockChannel = mock(Channel.class); + RemotingCommand mockRequest = mock(RemotingCommand.class); + BrokerController mockBrokerController = mock(BrokerController.class); + TopicConfigManager mockTopicConfigManager = mock(TopicConfigManager.class); + MessageStore mockMessageStore = mock(MessageStore.class); + BrokerConfig mockBrokerConfig = mock(BrokerConfig.class); + BrokerStatsManager mockBrokerStatsManager = mock(BrokerStatsManager.class); + PopMessageProcessor mockPopMessageProcessor = mock(PopMessageProcessor.class); + PopBufferMergeService mockPopBufferMergeService = mock(PopBufferMergeService.class); + BrokerMetricsManager brokerMetricsManager = mock(BrokerMetricsManager.class); + PopMetricsManager popMetricsManager = mock(PopMetricsManager.class); + + when(brokerMetricsManager.getPopMetricsManager()).thenReturn(popMetricsManager); + when(mockBrokerController.getBrokerMetricsManager()).thenReturn(brokerMetricsManager); + doNothing().when(popMetricsManager).incPopReviveCkPutCount(any(), any()); + when(brokerMetricsManager.getPopMetricsManager()).thenReturn(popMetricsManager); + when(mockBrokerController.getTopicConfigManager()).thenReturn(mockTopicConfigManager); + when(mockBrokerController.getMessageStore()).thenReturn(mockMessageStore); + when(mockBrokerController.getBrokerConfig()).thenReturn(mockBrokerConfig); + when(mockBrokerController.getBrokerStatsManager()).thenReturn(mockBrokerStatsManager); + when(mockBrokerController.getPopMessageProcessor()).thenReturn(mockPopMessageProcessor); + when(mockPopMessageProcessor.getPopBufferMergeService()).thenReturn(mockPopBufferMergeService); + when(mockPopBufferMergeService.addAk(anyInt(), any())).thenReturn(false); + when(mockBrokerController.getEscapeBridge()).thenReturn(escapeBridge); + PutMessageResult mockPutMessageResult = new PutMessageResult(PutMessageStatus.PUT_OK, null, true); + when(mockBrokerController.getEscapeBridge().asyncPutMessageToSpecificQueue(any())) + .thenReturn(CompletableFuture.completedFuture(mockPutMessageResult)); + + TopicConfig topicConfig = new TopicConfig(); + topicConfig.setReadQueueNums(4); + when(mockTopicConfigManager.selectTopicConfig(anyString())).thenReturn(topicConfig); + when(mockMessageStore.getMinOffsetInQueue(anyString(), anyInt())).thenReturn(0L); + when(mockMessageStore.getMaxOffsetInQueue(anyString(), anyInt())).thenReturn(10L); + when(mockBrokerConfig.isPopConsumerKVServiceEnable()).thenReturn(false); + + ChangeInvisibleTimeRequestHeader requestHeader = new ChangeInvisibleTimeRequestHeader(); + requestHeader.setTopic("TestTopic"); + requestHeader.setQueueId(1); + requestHeader.setOffset(5L); + requestHeader.setConsumerGroup("TestGroup"); + requestHeader.setExtraInfo("0 10000 10000 0 TestBroker 1"); + requestHeader.setInvisibleTime(60000L); + when(mockRequest.decodeCommandCustomHeader(ChangeInvisibleTimeRequestHeader.class)).thenReturn(requestHeader); + + ChangeInvisibleTimeProcessor processor = new ChangeInvisibleTimeProcessor(mockBrokerController); + CompletableFuture futureResponse = processor.processRequestAsync(mockChannel, mockRequest, true); + + RemotingCommand response = futureResponse.get(); + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + } } 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..e4dbd9913e2 --- /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.pop.orderly.ConsumerOrderInfoManager; +import org.apache.rocketmq.broker.pop.orderly.QueueLevelConsumerManager; +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 QueueLevelConsumerManager(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..cb584f99a74 --- /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.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anySet; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +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 + void testProcessRequest_BodyIsNull() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(0, null); + RemotingCommand response = processor.processRequest(ctx, request); + assertEquals(ResponseCode.ILLEGAL_OPERATION, response.getCode()); + } + + @Test + 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 + 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(), anyBoolean()); + } + + @Test + 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 + 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 + 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 + 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/PopBufferMergeServiceTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopBufferMergeServiceTest.java index acc7a3da74a..6cbbd9cfd92 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopBufferMergeServiceTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopBufferMergeServiceTest.java @@ -16,19 +16,22 @@ */ package org.apache.rocketmq.broker.processor; -import io.netty.channel.Channel; -import io.netty.channel.ChannelHandlerContext; +import com.alibaba.fastjson2.JSON; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.rocketmq.broker.BrokerController; -import org.apache.rocketmq.broker.client.ClientChannelInfo; +import org.apache.rocketmq.broker.client.ConsumerManager; +import org.apache.rocketmq.broker.failover.EscapeBridge; +import org.apache.rocketmq.broker.metrics.BrokerMetricsManager; +import org.apache.rocketmq.broker.metrics.PopMetricsManager; import org.apache.rocketmq.broker.schedule.ScheduleMessageService; +import org.apache.rocketmq.broker.topic.TopicConfigManager; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.TopicConfig; -import org.apache.rocketmq.remoting.netty.NettyClientConfig; -import org.apache.rocketmq.remoting.netty.NettyServerConfig; -import org.apache.rocketmq.remoting.protocol.heartbeat.ConsumerData; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.pop.AckMsg; import org.apache.rocketmq.store.pop.PopCheckPoint; @@ -37,56 +40,83 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.Spy; import org.mockito.junit.MockitoJUnitRunner; -import static org.apache.rocketmq.broker.processor.PullMessageProcessorTest.createConsumerData; +import java.lang.reflect.Method; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; + import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.Silent.class) public class PopBufferMergeServiceTest { - @Spy - private BrokerController brokerController = new BrokerController(new BrokerConfig(), new NettyServerConfig(), new NettyClientConfig(), new MessageStoreConfig()); + @Mock + private BrokerController brokerController; + private PopMessageProcessor popMessageProcessor; + + @Mock + private ScheduleMessageService scheduleMessageService; + @Mock - private ChannelHandlerContext handlerContext; + private TopicConfigManager topicConfigManager; + + @Mock + private ConsumerManager consumerManager; + @Mock private DefaultMessageStore messageStore; - private ScheduleMessageService scheduleMessageService; - private ClientChannelInfo clientChannelInfo; - private String group = "FooBarGroup"; - private String topic = "FooBar"; + + @Mock + private MessageStoreConfig messageStoreConfig; + + private String defaultGroup = "defaultGroup"; + + private String defaultTopic = "defaultTopic"; + + private PopBufferMergeService popBufferMergeService; + + @Mock + private BrokerConfig brokerConfig; + + @Mock + private EscapeBridge escapeBridge; @Before public void init() throws Exception { - FieldUtils.writeField(brokerController.getBrokerConfig(), "enablePopBufferMerge", true, true); - brokerController.setMessageStore(messageStore); + when(brokerConfig.getBrokerIP1()).thenReturn("127.0.0.1"); + when(brokerConfig.isEnablePopBufferMerge()).thenReturn(true); + when(brokerConfig.getPopCkStayBufferTime()).thenReturn(10 * 1000); + when(brokerController.getBrokerConfig()).thenReturn(brokerConfig); + when(brokerController.getEscapeBridge()).thenReturn(escapeBridge); + when(brokerController.getMessageStore()).thenReturn(messageStore); + when(brokerController.getTopicConfigManager()).thenReturn(topicConfigManager); + when(brokerController.getScheduleMessageService()).thenReturn(scheduleMessageService); + when(brokerController.getConsumerManager()).thenReturn(consumerManager); + when(brokerController.getMessageStoreConfig()).thenReturn(messageStoreConfig); popMessageProcessor = new PopMessageProcessor(brokerController); - scheduleMessageService = new ScheduleMessageService(brokerController); - scheduleMessageService.parseDelayLevel(); - Channel mockChannel = mock(Channel.class); - brokerController.getTopicConfigManager().getTopicConfigTable().put(topic, new TopicConfig()); - clientChannelInfo = new ClientChannelInfo(mockChannel); - ConsumerData consumerData = createConsumerData(group, topic); - brokerController.getConsumerManager().registerConsumer( - consumerData.getGroupName(), - clientChannelInfo, - consumerData.getConsumeType(), - consumerData.getMessageModel(), - consumerData.getConsumeFromWhere(), - consumerData.getSubscriptionDataSet(), - false); + popBufferMergeService = new PopBufferMergeService(brokerController, popMessageProcessor); + FieldUtils.writeDeclaredField(popBufferMergeService, "brokerController", brokerController, true); + ConcurrentMap topicConfigTable = new ConcurrentHashMap<>(); + topicConfigTable.put(defaultTopic, new TopicConfig()); + when(topicConfigManager.getTopicConfigTable()).thenReturn(topicConfigTable); } - @Test(timeout = 10_000) + @Test(timeout = 15_000) public void testBasic() throws Exception { // This test case fails on Windows in CI pipeline // Disable it for later fix Assume.assumeFalse(MixAll.isWindows()); - PopBufferMergeService popBufferMergeService = new PopBufferMergeService(brokerController, popMessageProcessor); - popBufferMergeService.start(); PopCheckPoint ck = new PopCheckPoint(); ck.setBitMap(0); int msgCnt = 1; @@ -97,8 +127,8 @@ public void testBasic() throws Exception { ck.setInvisibleTime(invisibleTime); int offset = 100; ck.setStartOffset(offset); - ck.setCId(group); - ck.setTopic(topic); + ck.setCId(defaultGroup); + ck.setTopic(defaultTopic); int queueId = 0; ck.setQueueId(queueId); @@ -108,18 +138,100 @@ public void testBasic() throws Exception { AckMsg ackMsg = new AckMsg(); ackMsg.setAckOffset(ackOffset); ackMsg.setStartOffset(offset); - ackMsg.setConsumerGroup(group); - ackMsg.setTopic(topic); + ackMsg.setConsumerGroup(defaultGroup); + ackMsg.setTopic(defaultTopic); ackMsg.setQueueId(queueId); ackMsg.setPopTime(popTime); try { assertThat(popBufferMergeService.addCk(ck, reviveQid, ackOffset, nextBeginOffset)).isTrue(); - assertThat(popBufferMergeService.getLatestOffset(topic, group, queueId)).isEqualTo(nextBeginOffset); + assertThat(popBufferMergeService.getLatestOffset(defaultTopic, defaultGroup, queueId)).isEqualTo(nextBeginOffset); Thread.sleep(1000); // wait background threads of PopBufferMergeService run for some time assertThat(popBufferMergeService.addAk(reviveQid, ackMsg)).isTrue(); - assertThat(popBufferMergeService.getLatestOffset(topic, group, queueId)).isEqualTo(nextBeginOffset); + assertThat(popBufferMergeService.getLatestOffset(defaultTopic, defaultGroup, queueId)).isEqualTo(nextBeginOffset); } finally { popBufferMergeService.shutdown(true); } } + + @Test + public void testAddCkJustOffset_MergeKeyConflict() { + PopCheckPoint point = mock(PopCheckPoint.class); + String mergeKey = "testMergeKey"; + when(point.getTopic()).thenReturn(mergeKey); + when(point.getCId()).thenReturn(""); + when(point.getQueueId()).thenReturn(0); + when(point.getStartOffset()).thenReturn(0L); + when(point.getPopTime()).thenReturn(0L); + when(point.getBrokerName()).thenReturn(""); + popBufferMergeService.buffer.put(mergeKey + "000", mock(PopBufferMergeService.PopCheckPointWrapper.class)); + + assertFalse(popBufferMergeService.addCkJustOffset(point, 0, 0, 0)); + } + + @Test + public void testAddCkMock() { + int queueId = 0; + long startOffset = 100L; + long invisibleTime = 30_000L; + long popTime = System.currentTimeMillis(); + int reviveQueueId = 0; + long nextBeginOffset = 101L; + String brokerName = "brokerName"; + popBufferMergeService.addCkMock(defaultGroup, defaultTopic, queueId, startOffset, invisibleTime, popTime, reviveQueueId, nextBeginOffset, brokerName); + verify(brokerConfig, times(1)).isEnablePopLog(); + } + + @Test + public void testPutAckToStore() throws Exception { + PopCheckPoint point = new PopCheckPoint(); + point.setStartOffset(100L); + point.setCId("testGroup"); + point.setTopic("testTopic"); + point.setQueueId(1); + point.setPopTime(System.currentTimeMillis()); + point.setBrokerName("testBroker"); + + PopBufferMergeService.PopCheckPointWrapper pointWrapper = mock(PopBufferMergeService.PopCheckPointWrapper.class); + when(pointWrapper.getCk()).thenReturn(point); + when(pointWrapper.getReviveQueueId()).thenReturn(0); + + AtomicInteger toStoreBits = new AtomicInteger(0); + when(pointWrapper.getToStoreBits()).thenReturn(toStoreBits); + + byte msgIndex = 0; + AtomicInteger count = new AtomicInteger(0); + + EscapeBridge escapeBridge = mock(EscapeBridge.class); + when(brokerController.getEscapeBridge()).thenReturn(escapeBridge); + when(brokerController.getBrokerConfig().isAppendAckAsync()).thenReturn(false); + BrokerMetricsManager brokerMetricsManager = mock(BrokerMetricsManager.class); + PopMetricsManager popMetricsManager = mock(PopMetricsManager.class); + + when(brokerMetricsManager.getPopMetricsManager()).thenReturn(popMetricsManager); + when(brokerController.getBrokerMetricsManager()).thenReturn(brokerMetricsManager); + doNothing().when(popMetricsManager).incPopReviveCkPutCount(any(), any()); + when(brokerMetricsManager.getPopMetricsManager()).thenReturn(popMetricsManager); + + when(escapeBridge.putMessageToSpecificQueue(any())).thenAnswer(invocation -> { + MessageExtBrokerInner capturedMessage = invocation.getArgument(0); + AckMsg ackMsg = JSON.parseObject(capturedMessage.getBody(), AckMsg.class); + + assertEquals(point.ackOffsetByIndex(msgIndex), ackMsg.getAckOffset()); + assertEquals(point.getStartOffset(), ackMsg.getStartOffset()); + assertEquals(point.getCId(), ackMsg.getConsumerGroup()); + assertEquals(point.getTopic(), ackMsg.getTopic()); + assertEquals(point.getQueueId(), ackMsg.getQueueId()); + assertEquals(point.getPopTime(), ackMsg.getPopTime()); + assertEquals(point.getBrokerName(), ackMsg.getBrokerName()); + + PutMessageResult result = mock(PutMessageResult.class); + when(result.getPutMessageStatus()).thenReturn(PutMessageStatus.PUT_OK); + return result; + }); + + Method method = PopBufferMergeService.class.getDeclaredMethod("putAckToStore", PopBufferMergeService.PopCheckPointWrapper.class, byte.class, AtomicInteger.class); + method.setAccessible(true); + method.invoke(popBufferMergeService, pointWrapper, msgIndex, count); + verify(escapeBridge, times(1)).putMessageToSpecificQueue(any(MessageExtBrokerInner.class)); + } } 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..4ad1eb77075 --- /dev/null +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopLiteMessageProcessorTest.java @@ -0,0 +1,489 @@ +/* + * 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.Iterator; +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 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.doReturn; +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 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/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java index 59559d3cfd0..5d7b97f2296 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopMessageProcessorTest.java @@ -16,10 +16,9 @@ */ package org.apache.rocketmq.broker.processor; +import com.alibaba.fastjson2.JSON; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.embedded.EmbeddedChannel; -import java.nio.ByteBuffer; -import java.util.concurrent.CompletableFuture; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.metrics.BrokerMetricsManager; @@ -28,6 +27,7 @@ import org.apache.rocketmq.common.TopicConfig; import org.apache.rocketmq.common.constant.ConsumeInitMode; import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; import org.apache.rocketmq.remoting.exception.RemotingCommandException; import org.apache.rocketmq.remoting.netty.NettyClientConfig; import org.apache.rocketmq.remoting.netty.NettyServerConfig; @@ -43,7 +43,7 @@ import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.exception.ConsumeQueueException; import org.apache.rocketmq.store.logfile.DefaultMappedFile; -import org.junit.Assert; +import org.apache.rocketmq.store.pop.PopCheckPoint; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -51,8 +51,13 @@ import org.mockito.Spy; import org.mockito.junit.MockitoJUnitRunner; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.CompletableFuture; + import static org.apache.rocketmq.broker.processor.PullMessageProcessorTest.createConsumerData; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertEquals; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; @@ -171,17 +176,17 @@ public void testGetInitOffset_retryTopic() throws RemotingCommandException { .thenReturn(CompletableFuture.completedFuture(getMessageResult)); long offset = brokerController.getConsumerOffsetManager().queryOffset(newGroup, retryTopic, 0); - Assert.assertEquals(-1, offset); + assertEquals(-1, offset); RemotingCommand request = createPopMsgCommand(newGroup, topic, 0, ConsumeInitMode.MAX); popMessageProcessor.processRequest(handlerContext, request); offset = brokerController.getConsumerOffsetManager().queryOffset(newGroup, retryTopic, 0); - Assert.assertEquals(minOffset, offset); + assertEquals(minOffset, offset); when(messageStore.getMinOffsetInQueue(retryTopic, 0)).thenReturn(minOffset * 2); popMessageProcessor.processRequest(handlerContext, request); offset = brokerController.getConsumerOffsetManager().queryOffset(newGroup, retryTopic, 0); - Assert.assertEquals(minOffset, offset); // will not entry getInitOffset() again + assertEquals(minOffset, offset); // will not entry getInitOffset() again messageStore.getMinOffsetInQueue(retryTopic, 0); // prevent UnnecessaryStubbingException } @@ -196,20 +201,47 @@ public void testGetInitOffset_normalTopic() throws RemotingCommandException, Con .thenReturn(CompletableFuture.completedFuture(getMessageResult)); long offset = brokerController.getConsumerOffsetManager().queryOffset(newGroup, topic, 0); - Assert.assertEquals(-1, offset); + assertEquals(-1, offset); RemotingCommand request = createPopMsgCommand(newGroup, topic, 0, ConsumeInitMode.MAX); popMessageProcessor.processRequest(handlerContext, request); offset = brokerController.getConsumerOffsetManager().queryOffset(newGroup, topic, 0); - Assert.assertEquals(maxOffset - 1, offset); // checkInMem return false + assertEquals(maxOffset - 1, offset); // checkInMem return false when(messageStore.getMaxOffsetInQueue(topic, 0)).thenReturn(maxOffset * 2); popMessageProcessor.processRequest(handlerContext, request); offset = brokerController.getConsumerOffsetManager().queryOffset(newGroup, topic, 0); - Assert.assertEquals(maxOffset - 1, offset); // will not entry getInitOffset() again + assertEquals(maxOffset - 1, offset); // will not entry getInitOffset() again messageStore.getMaxOffsetInQueue(topic, 0); // prevent UnnecessaryStubbingException } + @Test + public void testBuildCkMsgJsonParsing() { + PopCheckPoint ck = new PopCheckPoint(); + ck.setTopic("TestTopic"); + ck.setQueueId(1); + ck.setStartOffset(100L); + ck.setCId("TestConsumer"); + ck.setPopTime(System.currentTimeMillis()); + ck.setBrokerName("TestBroker"); + + int reviveQid = 0; + PopMessageProcessor processor = new PopMessageProcessor(brokerController); + + MessageExtBrokerInner result = processor.buildCkMsg(ck, reviveQid); + + String jsonBody = new String(result.getBody(), StandardCharsets.UTF_8); + PopCheckPoint actual = JSON.parseObject(jsonBody, PopCheckPoint.class); + + assertEquals(ck.getTopic(), actual.getTopic()); + assertEquals(ck.getQueueId(), actual.getQueueId()); + assertEquals(ck.getStartOffset(), actual.getStartOffset()); + assertEquals(ck.getCId(), actual.getCId()); + assertEquals(ck.getPopTime(), actual.getPopTime()); + assertEquals(ck.getBrokerName(), actual.getBrokerName()); + assertEquals(ck.getReviveTime(), actual.getReviveTime()); + } + private RemotingCommand createPopMsgCommand() { return createPopMsgCommand(group, topic, -1, ConsumeInitMode.MAX); } diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java index 6e77e7c557b..fa7e9982e1f 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/PopReviveServiceTest.java @@ -16,13 +16,7 @@ */ package org.apache.rocketmq.broker.processor; -import com.alibaba.fastjson.JSON; -import java.net.SocketAddress; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; +import com.alibaba.fastjson2.JSON; import org.apache.commons.lang3.tuple.Triple; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.failover.EscapeBridge; @@ -42,12 +36,13 @@ import org.apache.rocketmq.common.utils.DataConverter; import org.apache.rocketmq.common.utils.NetworkUtil; import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.store.AppendMessageResult; +import org.apache.rocketmq.store.AppendMessageStatus; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; import org.apache.rocketmq.store.PutMessageStatus; -import org.apache.rocketmq.store.AppendMessageResult; -import org.apache.rocketmq.store.AppendMessageStatus; import org.apache.rocketmq.store.pop.AckMsg; +import org.apache.rocketmq.store.pop.BatchAckMsg; import org.apache.rocketmq.store.pop.PopCheckPoint; import org.apache.rocketmq.store.timer.TimerMessageStore; import org.junit.Assert; @@ -58,18 +53,27 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import java.net.SocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + import static org.junit.Assert.assertEquals; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.any; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.any; import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.Silent.class) public class PopReviveServiceTest { @@ -139,7 +143,7 @@ public void testWhenAckMoreThanCk() throws Throwable { long maxReviveOffset = 4; when(consumerOffsetManager.queryOffset(PopAckConstants.REVIVE_GROUP, REVIVE_TOPIC, REVIVE_QUEUE_ID)) - .thenReturn(0L); + .thenReturn(0L); List reviveMessageExtList = new ArrayList<>(); long basePopTime = System.currentTimeMillis(); { @@ -176,7 +180,7 @@ public void testSkipLongWaiteAck() throws Throwable { long maxReviveOffset = 4; when(consumerOffsetManager.queryOffset(PopAckConstants.REVIVE_GROUP, REVIVE_TOPIC, REVIVE_QUEUE_ID)) - .thenReturn(0L); + .thenReturn(0L); List reviveMessageExtList = new ArrayList<>(); long basePopTime = System.currentTimeMillis() - brokerConfig.getReviveAckWaitMs() * 2; { @@ -213,7 +217,7 @@ public void testSkipLongWaiteAckWithSameAck() throws Throwable { long maxReviveOffset = 4; when(consumerOffsetManager.queryOffset(PopAckConstants.REVIVE_GROUP, REVIVE_TOPIC, REVIVE_QUEUE_ID)) - .thenReturn(0L); + .thenReturn(0L); List reviveMessageExtList = new ArrayList<>(); long basePopTime = System.currentTimeMillis() - brokerConfig.getReviveAckWaitMs() * 2; { @@ -419,6 +423,59 @@ public void testReviveMsgFromCk_messageNotFound_needRetry_noEnd() throws Throwab verify(messageStore, times(1)).putMessage(any(MessageExtBrokerInner.class)); // rewrite CK } + @Test + public void testReviveMsgFromBatchAck() throws Throwable { + brokerConfig.setEnableSkipLongAwaitingAck(true); + when(consumerOffsetManager.queryOffset(PopAckConstants.REVIVE_GROUP, REVIVE_TOPIC, REVIVE_QUEUE_ID)).thenReturn(0L); + List reviveMessageExtList = new ArrayList<>(); + long basePopTime = System.currentTimeMillis(); + reviveMessageExtList.add(buildBatchAckMsg(buildBatchAckMsg(Arrays.asList(1L, 2L, 3L), basePopTime), 1, 1, basePopTime)); + doReturn(reviveMessageExtList, new ArrayList<>()).when(popReviveService).getReviveMessage(anyLong(), anyInt()); + + PopReviveService.ConsumeReviveObj consumeReviveObj = new PopReviveService.ConsumeReviveObj(); + popReviveService.consumeReviveMessage(consumeReviveObj); + assertEquals(1, consumeReviveObj.map.size()); + + ArgumentCaptor commitOffsetCaptor = ArgumentCaptor.forClass(Long.class); + doNothing().when(consumerOffsetManager).commitOffset(anyString(), anyString(), anyString(), anyInt(), commitOffsetCaptor.capture()); + popReviveService.mergeAndRevive(consumeReviveObj); + assertEquals(1, commitOffsetCaptor.getValue().longValue()); + } + + public static MessageExtBrokerInner buildBatchAckMsg(BatchAckMsg batchAckMsg, long deliverMs, long reviveOffset, long deliverTime) { + MessageExtBrokerInner result = buildBatchAckInnerMessage(REVIVE_TOPIC, batchAckMsg, REVIVE_QUEUE_ID, STORE_HOST, deliverMs, PopMessageProcessor.genAckUniqueId(batchAckMsg)); + result.setQueueOffset(reviveOffset); + result.setDeliverTimeMs(deliverMs); + result.setStoreTimestamp(deliverTime); + return result; + } + + public static BatchAckMsg buildBatchAckMsg(Collection offsets, long popTime) { + BatchAckMsg result = new BatchAckMsg(); + result.setConsumerGroup(GROUP); + result.setTopic(TOPIC); + result.setQueueId(0); + result.setPopTime(popTime); + result.setBrokerName("broker-a"); + result.getAckOffsetList().addAll(offsets); + return result; + } + + public static MessageExtBrokerInner buildBatchAckInnerMessage(String reviveTopic, AckMsg ackMsg, int reviveQid, SocketAddress host, long deliverMs, String ackUniqueId) { + MessageExtBrokerInner result = new MessageExtBrokerInner(); + result.setTopic(reviveTopic); + result.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.CHARSET_UTF8)); + result.setQueueId(reviveQid); + result.setTags(PopAckConstants.BATCH_ACK_TAG); + result.setBornTimestamp(System.currentTimeMillis()); + result.setBornHost(host); + result.setStoreHost(host); + result.setDeliverTimeMs(deliverMs); + result.getProperties().put(MessageConst.PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX, ackUniqueId); + result.setPropertiesString(MessageDecoder.messageProperties2String(result.getProperties())); + return result; + } + public static PopCheckPoint buildPopCheckPoint(long startOffset, long popTime, long reviveOffset) { PopCheckPoint ck = new PopCheckPoint(); ck.setStartOffset(startOffset); @@ -467,14 +524,14 @@ public static MessageExtBrokerInner buildCkMsg(PopCheckPoint ck) { } public static MessageExtBrokerInner buildAckMsg(AckMsg ackMsg, long deliverMs, long reviveOffset, - long deliverTime) { + long deliverTime) { MessageExtBrokerInner messageExtBrokerInner = buildAckInnerMessage( - REVIVE_TOPIC, - ackMsg, - REVIVE_QUEUE_ID, - STORE_HOST, - deliverMs, - PopMessageProcessor.genAckUniqueId(ackMsg) + REVIVE_TOPIC, + ackMsg, + REVIVE_QUEUE_ID, + STORE_HOST, + deliverMs, + PopMessageProcessor.genAckUniqueId(ackMsg) ); messageExtBrokerInner.setQueueOffset(reviveOffset); messageExtBrokerInner.setDeliverTimeMs(deliverMs); @@ -483,7 +540,7 @@ public static MessageExtBrokerInner buildAckMsg(AckMsg ackMsg, long deliverMs, l } public static MessageExtBrokerInner buildAckInnerMessage(String reviveTopic, AckMsg ackMsg, int reviveQid, - SocketAddress host, long deliverMs, String ackUniqueId) { + SocketAddress host, long deliverMs, String ackUniqueId) { MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); msgInner.setTopic(reviveTopic); msgInner.setBody(JSON.toJSONString(ackMsg).getBytes(DataConverter.CHARSET_UTF8)); diff --git a/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java b/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java index 0fd54df7d8a..3656c5be2bd 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/processor/QueryMessageProcessorTest.java @@ -83,14 +83,13 @@ public void testQueryMessage() throws RemotingCommandException { result.setIndexLastUpdatePhyoffset(0); result.addMessage(new SelectMappedBufferResult(0, null, 0, null)); - when(messageStore.queryMessage(anyString(),anyString(),anyInt(),anyLong(),anyLong())).thenReturn(result); + when(messageStore.queryMessage(anyString(),anyString(),anyInt(),anyLong(),anyLong(),any(),any())).thenReturn(result); RemotingCommand request = createQueryMessageRequest("topic", "msgKey", 1, 100, 200,"false"); request.makeCustomHeaderToNet(); RemotingCommand response = queryMessageProcessor.processRequest(handlerContext, request); Assert.assertEquals(response.getCode(), ResponseCode.QUERY_NOT_FOUND); result.addMessage(new SelectMappedBufferResult(0, null, 1, null)); - when(messageStore.queryMessage(anyString(),anyString(),anyInt(),anyLong(),anyLong())).thenReturn(result); response = queryMessageProcessor.processRequest(handlerContext, request); Assert.assertNull(response); } diff --git a/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java index 3c975a599bc..bc34e26bf52 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/subscription/SubscriptionGroupManagerTest.java @@ -47,7 +47,6 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @RunWith(MockitoJUnitRunner.class) @@ -101,13 +100,13 @@ public void updateSubscriptionGroupConfig() { subscriptionGroupConfig.setGroupName(group); Map attr = ImmutableMap.of("+test", "true"); subscriptionGroupConfig.setAttributes(attr); + SubscriptionGroupManager subscriptionGroupManager = new SubscriptionGroupManager(brokerControllerMock); subscriptionGroupManager.updateSubscriptionGroupConfig(subscriptionGroupConfig); SubscriptionGroupConfig result = subscriptionGroupManager.getSubscriptionGroupTable().get(group); assertThat(result).isNotNull(); assertThat(result.getGroupName()).isEqualTo(group); assertThat(result.getAttributes().get("test")).isEqualTo("true"); - SubscriptionGroupConfig subscriptionGroupConfig1 = new SubscriptionGroupConfig(); subscriptionGroupConfig1.setGroupName(group); Map attrRemove = ImmutableMap.of("-test", ""); @@ -157,14 +156,11 @@ public void testUpdateSubscriptionGroupConfigList_ValidConfigList() { groupNames.add(groupName); } + SubscriptionGroupManager subscriptionGroupManager = new SubscriptionGroupManager(brokerControllerMock); subscriptionGroupManager.updateSubscriptionGroupConfigList(configList); - // Verifying that persist() is called once - verify(subscriptionGroupManager, times(1)).persist(); - groupNames.forEach(groupName -> assertThat(subscriptionGroupManager.getSubscriptionGroupTable().get(groupName)).isNotNull()); - } @Test diff --git a/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManagerTest.java b/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManagerTest.java index b74e57ab936..9b25e0134c2 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManagerTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/topic/TopicQueueMappingManagerTest.java @@ -17,15 +17,11 @@ package org.apache.rocketmq.broker.topic; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import java.util.UUID; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.common.BrokerConfig; +import org.apache.rocketmq.remoting.protocol.body.TopicQueueMappingSerializeWrapper; import org.apache.rocketmq.remoting.protocol.statictopic.TopicQueueMappingDetail; import org.apache.rocketmq.remoting.protocol.statictopic.TopicQueueMappingUtils; import org.apache.rocketmq.remoting.protocol.statictopic.TopicRemappingDetailWrapper; @@ -37,6 +33,16 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; + +import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.when; @RunWith(MockitoJUnitRunner.class) @@ -79,9 +85,9 @@ public void testEncodeDecode() throws Exception { String topic = UUID.randomUUID().toString(); int queueNum = 10; TopicRemappingDetailWrapper topicRemappingDetailWrapper = TopicQueueMappingUtils.createTopicConfigMapping(topic, queueNum, brokers, new HashMap<>()); - Assert.assertEquals(1, topicRemappingDetailWrapper.getBrokerConfigMap().size()); + assertEquals(1, topicRemappingDetailWrapper.getBrokerConfigMap().size()); TopicQueueMappingDetail topicQueueMappingDetail = topicRemappingDetailWrapper.getBrokerConfigMap().values().iterator().next().getMappingDetail(); - Assert.assertEquals(queueNum, topicQueueMappingDetail.getHostedQueues().size()); + assertEquals(queueNum, topicQueueMappingDetail.getHostedQueues().size()); mappingDetailMap.put(topic, topicQueueMappingDetail); } } @@ -89,7 +95,7 @@ public void testEncodeDecode() throws Exception { { topicQueueMappingManager = new TopicQueueMappingManager(brokerController); Assert.assertTrue(topicQueueMappingManager.load()); - Assert.assertEquals(0, topicQueueMappingManager.getTopicQueueMappingTable().size()); + assertEquals(0, topicQueueMappingManager.getTopicQueueMappingTable().size()); for (TopicQueueMappingDetail mappingDetail : mappingDetailMap.values()) { for (int i = 0; i < 10; i++) { topicQueueMappingManager.updateTopicQueueMapping(mappingDetail, false, false, true); @@ -101,11 +107,49 @@ public void testEncodeDecode() throws Exception { { topicQueueMappingManager = new TopicQueueMappingManager(brokerController); Assert.assertTrue(topicQueueMappingManager.load()); - Assert.assertEquals(mappingDetailMap.size(), topicQueueMappingManager.getTopicQueueMappingTable().size()); + assertEquals(mappingDetailMap.size(), topicQueueMappingManager.getTopicQueueMappingTable().size()); for (TopicQueueMappingDetail topicQueueMappingDetail: topicQueueMappingManager.getTopicQueueMappingTable().values()) { - Assert.assertEquals(topicQueueMappingDetail, mappingDetailMap.get(topicQueueMappingDetail.getTopic())); + assertEquals(topicQueueMappingDetail, mappingDetailMap.get(topicQueueMappingDetail.getTopic())); } } delete(topicQueueMappingManager); } + + @Test + public void testEncodePretty() { + TopicQueueMappingManager topicQueueMappingManager = new TopicQueueMappingManager(null); + TopicQueueMappingDetail detail = new TopicQueueMappingDetail(); + detail.setTopic("testTopic"); + detail.setBname("testBroker"); + + topicQueueMappingManager.getTopicQueueMappingTable().put("testTopic", detail); + topicQueueMappingManager.getDataVersion().nextVersion(); + + String actual = topicQueueMappingManager.encode(true); + TopicQueueMappingSerializeWrapper expectedWrapper = new TopicQueueMappingSerializeWrapper(); + expectedWrapper.setTopicQueueMappingInfoMap(new ConcurrentHashMap<>(topicQueueMappingManager.getTopicQueueMappingTable())); + expectedWrapper.setDataVersion(topicQueueMappingManager.getDataVersion()); + String expected = JSON.toJSONString(expectedWrapper, JSONWriter.Feature.PrettyFormat); + + assertEquals(expected, actual); + } + + @Test + public void testEncodeNonPretty() { + TopicQueueMappingManager topicQueueMappingManager = new TopicQueueMappingManager(null); + TopicQueueMappingDetail detail = new TopicQueueMappingDetail(); + detail.setTopic("testTopic"); + detail.setBname("testBroker"); + + topicQueueMappingManager.getTopicQueueMappingTable().put("testTopic", detail); + topicQueueMappingManager.getDataVersion().nextVersion(); + + String actual = topicQueueMappingManager.encode(false); + TopicQueueMappingSerializeWrapper expectedWrapper = new TopicQueueMappingSerializeWrapper(); + expectedWrapper.setTopicQueueMappingInfoMap(new ConcurrentHashMap<>(topicQueueMappingManager.getTopicQueueMappingTable())); + expectedWrapper.setDataVersion(topicQueueMappingManager.getDataVersion()); + String expected = JSON.toJSONString(expectedWrapper); + + assertEquals(expected, actual); + } } diff --git a/broker/src/test/java/org/apache/rocketmq/broker/transaction/queue/TransactionMetricsTest.java b/broker/src/test/java/org/apache/rocketmq/broker/transaction/queue/TransactionMetricsTest.java index 690b4eabb57..62a6ad8b5b9 100644 --- a/broker/src/test/java/org/apache/rocketmq/broker/transaction/queue/TransactionMetricsTest.java +++ b/broker/src/test/java/org/apache/rocketmq/broker/transaction/queue/TransactionMetricsTest.java @@ -19,23 +19,40 @@ import org.apache.rocketmq.broker.transaction.TransactionMetrics; import org.apache.rocketmq.broker.transaction.TransactionMetrics.Metric; +import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.junit.MockitoJUnitRunner; +import java.io.File; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.Collections; +import java.util.UUID; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; @RunWith(MockitoJUnitRunner.class) public class TransactionMetricsTest { private TransactionMetrics transactionMetrics; private String configPath; + private Path path; @Before - public void setUp() throws Exception { - configPath = "configPath"; - transactionMetrics = new TransactionMetrics(configPath); + public void before() throws Exception { + configPath = createBaseDir(); + path = Paths.get(configPath); + transactionMetrics = spy(new TransactionMetrics(configPath)); + } + + @After + public void after() throws Exception { + deleteFile(configPath); + assertFalse(path.toFile().exists()); } /** @@ -80,4 +97,40 @@ public void testCleanMetrics() { transactionMetrics.cleanMetrics(Collections.singleton(topic)); assert transactionMetrics.getTransactionCount(topic) == 0; } + + @Test + public void testPersist() { + assertFalse(path.toFile().exists()); + transactionMetrics.persist(); + assertTrue(path.toFile().exists()); + verify(transactionMetrics).persist(); + } + + private String createBaseDir() { + String baseDir = System.getProperty("java.io.tmpdir") + File.separator + "unitteststore-" + UUID.randomUUID(); + final File file = new File(baseDir); + if (file.exists()) { + System.exit(1); + } + return baseDir; + } + + private void deleteFile(String fileName) { + deleteFile(new File(fileName)); + } + + private void deleteFile(File file) { + if (!file.exists()) { + return; + } + if (file.isFile()) { + file.delete(); + } else if (file.isDirectory()) { + File[] files = file.listFiles(); + for (File file1 : files) { + deleteFile(file1); + } + file.delete(); + } + } } diff --git a/client/BUILD.bazel b/client/BUILD.bazel index 31681f10299..3bd84606a29 100644 --- a/client/BUILD.bazel +++ b/client/BUILD.bazel @@ -27,7 +27,6 @@ java_library( "@maven//:commons_validator_commons_validator", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:io_netty_netty_all", "@maven//:io_opentracing_opentracing_api", diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java index 20857f14e08..0840354d796 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumer.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.client.consumer; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -33,11 +34,13 @@ import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.protocol.NamespaceUtil; +import org.apache.rocketmq.remoting.protocol.filter.FilterAPI; import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel; -import org.apache.rocketmq.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; import static org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData.SUB_ALL; @@ -171,6 +174,8 @@ public class DefaultLitePullConsumer extends ClientConfig implements LitePullCon private RPCHook rpcHook; + private final Set subscriptionsForHeartbeat = new HashSet<>(); + /** * Default constructor. */ @@ -618,4 +623,17 @@ public boolean isEnableMsgTrace() { public void setEnableMsgTrace(boolean enableMsgTrace) { this.enableTrace = enableMsgTrace; } + + public Set getSubscriptionsForHeartbeat() { + return this.subscriptionsForHeartbeat; + } + + public synchronized void buildSubscriptionsForHeartbeat(Map messageSelectorMap) throws Exception { + this.subscriptionsForHeartbeat.clear(); + for (Map.Entry entry : messageSelectorMap.entrySet()) { + SubscriptionData subscriptionData = FilterAPI.build(entry.getKey(), + entry.getValue().getExpression(), entry.getValue().getExpressionType()); + this.subscriptionsForHeartbeat.add(subscriptionData); + } + } } diff --git a/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java b/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java index 6c6a5970a60..d16a7c95353 100644 --- a/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java +++ b/client/src/main/java/org/apache/rocketmq/client/consumer/LitePullConsumer.java @@ -16,14 +16,13 @@ */ package org.apache.rocketmq.client.consumer; -import org.apache.rocketmq.client.exception.MQClientException; -import org.apache.rocketmq.common.message.MessageExt; -import org.apache.rocketmq.common.message.MessageQueue; - import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; public interface LitePullConsumer { @@ -107,6 +106,8 @@ public interface LitePullConsumer { */ void setSubExpressionForAssign(final String topic, final String subExpression); + void buildSubscriptionsForHeartbeat(Map subExpressionMap) throws Exception; + /** * Fetch data for the topics or partitions specified using assign API * diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java index f98d9e5818d..b6fd72ad013 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/MQAdminImpl.java @@ -279,20 +279,18 @@ public MessageExt viewMessage(String topic, String msgId) public QueryResult queryMessage(String topic, String key, int maxNum, long begin, long end) throws MQClientException, InterruptedException { - return queryMessage(null, topic, key, maxNum, begin, end, false); + return queryMessage(null, topic, key, maxNum, begin, end, false, MessageConst.INDEX_KEY_TYPE, null); } public QueryResult queryMessageByUniqKey(String topic, String uniqKey, int maxNum, long begin, long end) throws MQClientException, InterruptedException { - - return queryMessage(null, topic, uniqKey, maxNum, begin, end, true); + return queryMessage(null, topic, uniqKey, maxNum, begin, end, true, MessageConst.INDEX_UNIQUE_TYPE, null); } public QueryResult queryMessageByUniqKey(String clusterName, String topic, String uniqKey, int maxNum, long begin, long end) throws MQClientException, InterruptedException { - - return queryMessage(clusterName, topic, uniqKey, maxNum, begin, end, true); + return queryMessage(clusterName, topic, uniqKey, maxNum, begin, end, true, MessageConst.INDEX_UNIQUE_TYPE, null); } public MessageExt queryMessageByUniqKey(String topic, @@ -312,7 +310,7 @@ public MessageExt queryMessageByUniqKey(String topic, public MessageExt queryMessageByUniqKey(String clusterName, String topic, String uniqKey, long begin, long end) throws InterruptedException, MQClientException { - QueryResult qr = this.queryMessage(clusterName, topic, uniqKey, 32, begin, end, true); + QueryResult qr = this.queryMessage(clusterName, topic, uniqKey, 32, begin, end, true, MessageConst.INDEX_UNIQUE_TYPE, null); if (qr != null && qr.getMessageList() != null && qr.getMessageList().size() > 0) { return qr.getMessageList().get(0); } else { @@ -320,8 +318,12 @@ public MessageExt queryMessageByUniqKey(String clusterName, String topic, } } - public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, - boolean isUniqKey) throws MQClientException, + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, boolean isUniqKey) throws MQClientException, + InterruptedException { + return queryMessage(clusterName, topic, key, maxNum, begin, end, isUniqKey, null, null); + } + + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, boolean isUniqKey, String indexType, String lastKey) throws MQClientException, InterruptedException { boolean isLmq = MixAll.isLmq(topic); @@ -369,6 +371,8 @@ public QueryResult queryMessage(String clusterName, String topic, String key, in requestHeader.setMaxNum(maxNum); requestHeader.setBeginTimestamp(begin); requestHeader.setEndTimestamp(end); + requestHeader.setIndexType(indexType); + requestHeader.setLastKey(lastKey); this.mQClientFactory.getMQClientAPIImpl().queryMessage(addr, requestHeader, timeoutMillis * 3, new InvokeCallback() { @@ -445,7 +449,7 @@ public void operationFail(Throwable throwable) { } else { log.warn("queryMessage by uniqKey, find message key not matched, maybe hash duplicate {}", msgExt.toString()); } - } else { + } else if (!StringUtils.isEmpty(indexType) && MessageConst.INDEX_KEY_TYPE.equals(indexType)) { String keys = msgExt.getKeys(); String msgTopic = msgExt.getTopic(); if (keys != null) { @@ -465,6 +469,20 @@ public void operationFail(Throwable throwable) { log.warn("queryMessage, find message key not matched, maybe hash duplicate {}", msgExt.toString()); } } + } else if (!StringUtils.isEmpty(indexType) && MessageConst.INDEX_TAG_TYPE.equals(indexType)) { + String tags = msgExt.getTags(); + String msgTopic = msgExt.getTopic(); + boolean matched = false; + if (tags != null) { + if (Objects.equals(key, tags) && (isLmq || Objects.equals(topic, msgTopic))) { + matched = true; + } + } + if (matched) { + messageList.add(msgExt); + } else { + log.warn("queryMessage, find message key not matched, maybe hash duplicate {}", msgExt.toString()); + } } } } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/MQClientAPIImpl.java index 95bb0e8a96f..f0c59b4e1c9 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 @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.client.impl; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.ClientConfig; @@ -108,7 +108,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 +170,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 +188,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 +214,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,7 +248,7 @@ import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; import org.apache.rocketmq.remoting.rpchook.DynamicalExtFieldRPCHook; import org.apache.rocketmq.remoting.rpchook.StreamTypeRPCHook; - +import static org.apache.rocketmq.common.message.MessageConst.TIMER_ENGINE_TYPE; import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -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 * @@ -3649,4 +3756,96 @@ public void exportPopRecord(String brokerAddr, long timeout) throws RemotingConn } throw new MQBrokerException(response.getCode(), response.getRemark()); } + + public void switchTimerEngine(String brokerAddr, String engineType, long timeoutMillis) throws RemotingConnectException, + RemotingSendRequestException, RemotingTimeoutException, InterruptedException, MQBrokerException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.SWITCH_TIMER_ENGINE, null); + request.addExtField(TIMER_ENGINE_TYPE, engineType); + RemotingCommand response = this.remotingClient.invokeSync(brokerAddr, request, timeoutMillis); + assert response != null; + if (response.getCode() == SUCCESS) { + return; + } + throw new MQBrokerException(response.getCode(), response.getRemark()); + } + + 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/consumer/DefaultLitePullConsumerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java index f85dcc7b459..6ce8b2d1cd4 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/consumer/DefaultLitePullConsumerImpl.java @@ -63,6 +63,8 @@ import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.common.sysflag.PullSysFlag; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.exception.RemotingException; import org.apache.rocketmq.remoting.protocol.NamespaceUtil; @@ -73,8 +75,6 @@ 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.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; public class DefaultLitePullConsumerImpl implements MQConsumerInner { @@ -1122,7 +1122,8 @@ public Set subscriptions() { Set subSet = new HashSet<>(); subSet.addAll(this.rebalanceImpl.getSubscriptionInner().values()); - + subSet.addAll(this.defaultLitePullConsumer.getSubscriptionsForHeartbeat()); + return subSet; } diff --git a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java index 3055f2cdee1..bb838a62650 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/factory/MQClientInstance.java @@ -16,26 +16,8 @@ */ package org.apache.rocketmq.client.impl.factory; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import io.netty.channel.Channel; -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.Random; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadFactory; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.admin.MQAdminExtInner; @@ -86,6 +68,25 @@ import org.apache.rocketmq.remoting.protocol.route.QueueData; import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +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.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + import static org.apache.rocketmq.remoting.rpc.ClientMetadata.topicRouteData2EndpointsForStaticTopic; public class MQClientInstance { 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/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java index b54091e3ced..b6af66b67b2 100644 --- a/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java +++ b/client/src/main/java/org/apache/rocketmq/client/impl/producer/DefaultMQProducerImpl.java @@ -809,7 +809,9 @@ private SendResult sendDefaultImpl( endTimestamp = System.currentTimeMillis(); this.updateFaultItem(mq.getBrokerName(), endTimestamp - beginTimestampPrev, false, true); log.warn("sendKernelImpl exception, resend at once, InvokeID: {}, RT: {}ms, Broker: {}", invokeID, endTimestamp - beginTimestampPrev, mq, e); - log.warn(msg.toString()); + if (log.isDebugEnabled()) { + log.debug(msg.toString()); + } exception = e; continue; } catch (RemotingException e) { diff --git a/client/src/main/java/org/apache/rocketmq/client/producer/SendResult.java b/client/src/main/java/org/apache/rocketmq/client/producer/SendResult.java index d160eb4eae9..54052753983 100644 --- a/client/src/main/java/org/apache/rocketmq/client/producer/SendResult.java +++ b/client/src/main/java/org/apache/rocketmq/client/producer/SendResult.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.client.producer; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import org.apache.rocketmq.common.message.MessageQueue; public class SendResult { diff --git a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java index 592c247057b..f57b3f80fe4 100644 --- a/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/consumer/DefaultLitePullConsumerTest.java @@ -17,17 +17,6 @@ package org.apache.rocketmq.client.consumer; -import java.io.ByteArrayOutputStream; -import java.lang.reflect.Field; -import java.net.InetSocketAddress; -import java.time.Duration; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentMap; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.consumer.store.OffsetStore; @@ -65,6 +54,18 @@ import org.mockito.junit.MockitoJUnitRunner; import org.mockito.stubbing.Answer; +import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; + import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Fail.failBecauseExceptionWasNotThrown; import static org.awaitility.Awaitility.await; @@ -75,6 +76,7 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -898,5 +900,6 @@ private void suppressUpdateTopicRouteInfoFromNameServer( ConcurrentMap factoryTable = (ConcurrentMap) FieldUtils.readDeclaredField(MQClientManager.getInstance(), "factoryTable", true); factoryTable.put(litePullConsumer.buildMQClientId(), mQClientFactory); doReturn(false).when(mQClientFactory).updateTopicRouteInfoFromNameServer(anyString()); + doNothing().when(mQClientFactory).updateTopicRouteInfoFromNameServer(); } } 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 c12b23cb0db..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 @@ -16,6 +16,7 @@ */ package org.apache.rocketmq.client.impl; +import com.alibaba.fastjson2.JSON; import org.apache.rocketmq.client.ClientConfig; import org.apache.rocketmq.client.consumer.AckCallback; import org.apache.rocketmq.client.consumer.AckResult; @@ -34,6 +35,7 @@ import org.apache.rocketmq.client.producer.SendCallback; import org.apache.rocketmq.client.producer.SendResult; import org.apache.rocketmq.client.producer.SendStatus; +import org.apache.rocketmq.common.CheckRocksdbCqWriteResult; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ObjectCreator; import org.apache.rocketmq.common.Pair; @@ -539,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); @@ -2031,6 +2033,22 @@ public NettyClientConfig getNettyClientConfig() { } } + @Test + public void testCheckRocksdbCqWriteProgress() throws Exception { + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success"); + CheckRocksdbCqWriteResult expectedResult = new CheckRocksdbCqWriteResult(); + expectedResult.setCheckStatus(CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue()); + response.setBody(JSON.toJSONString(expectedResult).getBytes()); + + when(remotingClient.invokeSync(any(String.class), any(RemotingCommand.class), any(Long.class))) + .thenReturn(response); + + CheckRocksdbCqWriteResult result = mqClientAPI.checkRocksdbCqWriteProgress( + "brokerAddr", "testTopic", 12345L, 3000L); + + assertEquals(CheckRocksdbCqWriteResult.CheckStatus.CHECK_OK.getValue(), result.getCheckStatus()); + } + private Properties createProperties() { Properties result = new Properties(); result.put("key", "value"); diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/admin/MqClientAdminImplTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/admin/MqClientAdminImplTest.java index 71682fb52c0..520f4da5f2a 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/admin/MqClientAdminImplTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/admin/MqClientAdminImplTest.java @@ -206,7 +206,7 @@ public void assertUpdateOrCreateTopicWithError() { @Test public void assertUpdateOrCreateSubscriptionGroupWithSuccess() throws Exception { setResponseSuccess(null); - SubscriptionGroupConfig config = mock(SubscriptionGroupConfig.class); + SubscriptionGroupConfig config = new SubscriptionGroupConfig(); CompletableFuture actual = mqClientAdminImpl.updateOrCreateSubscriptionGroup(defaultBrokerAddr, config, defaultTimeout); assertNull(actual.get()); } @@ -214,7 +214,7 @@ public void assertUpdateOrCreateSubscriptionGroupWithSuccess() throws Exception @Test public void assertUpdateOrCreateSubscriptionGroupWithError() { setResponseError(); - SubscriptionGroupConfig config = mock(SubscriptionGroupConfig.class); + SubscriptionGroupConfig config = new SubscriptionGroupConfig(); CompletableFuture actual = mqClientAdminImpl.updateOrCreateSubscriptionGroup(defaultBrokerAddr, config, defaultTimeout); Throwable thrown = assertThrows(ExecutionException.class, actual::get); assertTrue(thrown.getCause() instanceof MQClientException); diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/PopProcessQueueTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/PopProcessQueueTest.java new file mode 100644 index 00000000000..0633342c598 --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/PopProcessQueueTest.java @@ -0,0 +1,56 @@ +/* + * 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.client.impl.consumer; + +import org.apache.rocketmq.remoting.protocol.body.PopProcessQueueInfo; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.junit.MockitoJUnitRunner; + +import static org.junit.Assert.assertEquals; + +@RunWith(MockitoJUnitRunner.class) +public class PopProcessQueueTest { + + private final PopProcessQueueInfo popProcessQueueInfo = new PopProcessQueueInfo(); + + @Test + public void testPopProcessQueue() { + long currentTime = System.currentTimeMillis(); + PopProcessQueue popRequest1 = createPopProcessQueue(currentTime); + PopProcessQueue popRequest2 = createPopProcessQueue(currentTime); + assertEquals(popRequest1.getLastPopTimestamp(), popRequest2.getLastPopTimestamp()); + assertEquals(popRequest1.toString(), popRequest2.toString()); + assertEquals(popRequest1.getWaiAckMsgCount(), popRequest2.getWaiAckMsgCount()); + assertEquals(popRequest1.ack(), popRequest2.ack()); + assertEquals(popRequest1.isPullExpired(), popRequest2.isPullExpired()); + assertEquals(popProcessQueueInfo.getLastPopTimestamp(), popRequest1.getLastPopTimestamp()); + assertEquals(popProcessQueueInfo.isDroped(), popRequest1.isDropped()); + assertEquals(popProcessQueueInfo.getWaitAckCount(), popRequest1.getWaiAckMsgCount() + popRequest2.getWaiAckMsgCount()); + } + + private PopProcessQueue createPopProcessQueue(final long currentTime) { + PopProcessQueue result = new PopProcessQueue(); + long curTime = System.currentTimeMillis(); + result.setLastPopTimestamp(curTime); + result.incFoundMsg(1); + result.decFoundMsg(1); + result.setLastPopTimestamp(currentTime); + result.fillPopProcessQueueInfo(popProcessQueueInfo); + return result; + } +} diff --git a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java index dd7ffa757f8..a12633be1bd 100644 --- a/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java +++ b/client/src/test/java/org/apache/rocketmq/client/impl/consumer/ProcessQueueTest.java @@ -16,11 +16,6 @@ */ package org.apache.rocketmq.client.impl.consumer; -import java.lang.reflect.Field; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.TreeMap; import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.exception.MQBrokerException; @@ -34,6 +29,12 @@ import org.junit.runner.RunWith; import org.mockito.junit.MockitoJUnitRunner; +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.TreeMap; + import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -164,6 +165,7 @@ private ProcessQueue createProcessQueue() { ProcessQueue result = new ProcessQueue(); result.setMsgAccCnt(1); result.incTryUnlockTimes(); + result.setLastPullTimestamp(10000L); return result; } diff --git a/client/src/test/java/org/apache/rocketmq/client/producer/SendResultTest.java b/client/src/test/java/org/apache/rocketmq/client/producer/SendResultTest.java new file mode 100644 index 00000000000..1ae8371a96b --- /dev/null +++ b/client/src/test/java/org/apache/rocketmq/client/producer/SendResultTest.java @@ -0,0 +1,59 @@ +/* + * 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.client.producer; + +import com.alibaba.fastjson2.JSON; +import org.apache.rocketmq.common.message.MessageQueue; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class SendResultTest { + + @Test + public void testEncoderSendResultToJson() { + SendResult sendResult = new SendResult(); + sendResult.setSendStatus(SendStatus.SEND_OK); + sendResult.setMsgId("12345"); + sendResult.setQueueOffset(100L); + MessageQueue messageQueue = new MessageQueue("TestTopic", "BrokerA", 1); + sendResult.setMessageQueue(messageQueue); + + String json = SendResult.encoderSendResultToJson(sendResult); + + SendResult decodedResult = JSON.parseObject(json, SendResult.class); + assertEquals(sendResult.getSendStatus(), decodedResult.getSendStatus()); + assertEquals(sendResult.getMsgId(), decodedResult.getMsgId()); + assertEquals(sendResult.getQueueOffset(), decodedResult.getQueueOffset()); + assertEquals(sendResult.getMessageQueue(), decodedResult.getMessageQueue()); + } + + @Test + public void testDecoderSendResultFromJson() { + String json = "{\"sendStatus\":\"SEND_OK\",\"msgId\":\"12345\",\"queueOffset\":100,\"messageQueue\":{\"topic\":\"TestTopic\",\"brokerName\":\"BrokerA\",\"queueId\":1}}"; + + SendResult sendResult = SendResult.decoderSendResultFromJson(json); + + assertEquals(SendStatus.SEND_OK, sendResult.getSendStatus()); + assertEquals("12345", sendResult.getMsgId()); + assertEquals(100L, sendResult.getQueueOffset()); + assertEquals("TestTopic", sendResult.getMessageQueue().getTopic()); + assertEquals("BrokerA", sendResult.getMessageQueue().getBrokerName()); + assertEquals(1, sendResult.getMessageQueue().getQueueId()); + } +} diff --git a/common/BUILD.bazel b/common/BUILD.bazel index 10c5d19fbe8..8aeeb2f24fc 100644 --- a/common/BUILD.bazel +++ b/common/BUILD.bazel @@ -21,7 +21,6 @@ java_library( srcs = glob(["src/main/java/**/*.java"]), visibility = ["//visibility:public"], deps = [ - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_github_luben_zstd_jni", "@maven//:com_google_guava_guava", @@ -55,7 +54,6 @@ java_library( ":common", "//:test_deps", "@maven//:com_google_guava_guava", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:commons_codec_commons_codec", "@maven//:io_netty_netty_all", 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 a46435543a9..10f64c68345 100644 --- a/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/BrokerConfig.java @@ -485,6 +485,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; } @@ -2177,4 +2210,124 @@ public int getSplitMetadataSize() { public void setSplitMetadataSize(int splitMetadataSize) { this.splitMetadataSize = splitMetadataSize; } + + 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/MQVersion.java b/common/src/main/java/org/apache/rocketmq/common/MQVersion.java index 20218f51964..9b1a9526727 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MQVersion.java +++ b/common/src/main/java/org/apache/rocketmq/common/MQVersion.java @@ -18,7 +18,7 @@ public class MQVersion { - public static final int CURRENT_VERSION = Version.V5_3_3.ordinal(); + public static final int CURRENT_VERSION = Version.V5_3_4.ordinal(); public static String getVersionDesc(int value) { int length = Version.values().length; diff --git a/common/src/main/java/org/apache/rocketmq/common/MixAll.java b/common/src/main/java/org/apache/rocketmq/common/MixAll.java index 00006ac7a8e..efde29a891b 100644 --- a/common/src/main/java/org/apache/rocketmq/common/MixAll.java +++ b/common/src/main/java/org/apache/rocketmq/common/MixAll.java @@ -45,6 +45,7 @@ import java.util.Properties; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Predicate; @@ -128,6 +129,7 @@ public class MixAll { public static final String MULTI_PATH_SPLITTER = System.getProperty("rocketmq.broker.multiPathSplitter", ","); private static final String OS = System.getProperty("os.name").toLowerCase(); + public static final long MILLS_FOR_HOUR = TimeUnit.HOURS.toMillis(1); private static final Set PREDEFINE_GROUP_SET = ImmutableSet.of( DEFAULT_CONSUMER_GROUP, @@ -572,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); } @@ -584,4 +587,59 @@ public static String adjustConfigForPlatform(String config) { } return config; } + + public static long dealTimeToHourStamps(long timeStamp) { + if (timeStamp <= 0L) { + return timeStamp; + } + return (timeStamp / MILLS_FOR_HOUR) * MILLS_FOR_HOUR; + } + + public static boolean isHourTime(Long timeStamp) { + if (null == timeStamp) { + return false; + } + if (timeStamp <= 0L) { + return false; + } + return timeStamp % MILLS_FOR_HOUR == 0; + } + + public static List getHours(long startTimeMillis, long endTimeMillis) { + if (startTimeMillis > endTimeMillis || startTimeMillis <= 0L || endTimeMillis <= 0L) { + return null; + } + List result = new ArrayList<>(); + long startHour = dealTimeToHourStamps(startTimeMillis); + long endHour = dealTimeToHourStamps(endTimeMillis); + long current = startHour; + while (current <= endHour) { + result.add(current); + //protect system self 30 * 24 + if (result.size() >= 720) { + return result; + } + current += MILLS_FOR_HOUR; + } + return result; + } + + public static boolean isByteArrayEqual(byte[] array1, int offset1, int length1, byte[] array2, int offset2, int length2) { + if (null == array1 || null == array2) { + return false; + } + if (length1 != length2) { + return false; + } + if (offset1 < 0 || offset1 + length1 > array1.length || + offset2 < 0 || offset2 + length2 > array2.length) { + throw new ArrayIndexOutOfBoundsException("Invalid array index"); + } + for (int i = 0; i < length1; i++) { + if (array1[offset1 + i] != array2[offset2 + i]) { + return false; + } + } + return true; + } } diff --git a/common/src/main/java/org/apache/rocketmq/common/AbstractBrokerRunnable.java b/common/src/main/java/org/apache/rocketmq/common/OrderedConsumptionLevel.java similarity index 53% rename from common/src/main/java/org/apache/rocketmq/common/AbstractBrokerRunnable.java rename to common/src/main/java/org/apache/rocketmq/common/OrderedConsumptionLevel.java index 34aabc5772c..bdf972abda8 100644 --- a/common/src/main/java/org/apache/rocketmq/common/AbstractBrokerRunnable.java +++ b/common/src/main/java/org/apache/rocketmq/common/OrderedConsumptionLevel.java @@ -14,35 +14,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.rocketmq.common; -import java.io.File; -import org.apache.rocketmq.logging.org.slf4j.MDC; +public enum OrderedConsumptionLevel { + QUEUE(0), + SHARDING_KEY(1); -public abstract class AbstractBrokerRunnable implements Runnable { - protected final BrokerIdentity brokerIdentity; + private final int value; - public AbstractBrokerRunnable(BrokerIdentity brokerIdentity) { - this.brokerIdentity = brokerIdentity; + OrderedConsumptionLevel(int value) { + this.value = value; } - private static final String MDC_BROKER_CONTAINER_LOG_DIR = "brokerContainerLogDir"; - - /** - * real logic for running - */ - public abstract void run0(); + public int getValue() { + return value; + } - @Override - public void run() { - try { - if (brokerIdentity.isInBrokerContainer()) { - MDC.put(MDC_BROKER_CONTAINER_LOG_DIR, File.separator + brokerIdentity.getCanonicalName()); - } - run0(); - } finally { - MDC.clear(); + public static OrderedConsumptionLevel valueOf(int value) { + if (value == 1) { + return SHARDING_KEY; } + return QUEUE; } } 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 5b0072401c5..0c1b8c5b734 100644 --- a/common/src/main/java/org/apache/rocketmq/common/SubscriptionGroupAttributes.java +++ b/common/src/main/java/org/apache/rocketmq/common/SubscriptionGroupAttributes.java @@ -16,14 +16,71 @@ */ 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 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 + ); + public static final Map ALL; static { ALL = new HashMap<>(); + 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); } -} +} \ No newline at end of file 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 0bf64905a03..18389b58191 100644 --- a/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java +++ b/common/src/main/java/org/apache/rocketmq/common/TopicConfig.java @@ -16,15 +16,17 @@ */ package org.apache.rocketmq.common; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.TypeReference; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.TypeReference; +import com.alibaba.fastjson2.annotation.JSONField; 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 { @@ -215,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 5e581a34eec..5cd418465b3 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 @@ -28,6 +28,7 @@ public enum TopicMessageType { FIFO("FIFO"), DELAY("DELAY"), TRANSACTION("TRANSACTION"), + LITE("LITE"), MIXED("MIXED"); private final String value; @@ -36,7 +37,8 @@ public enum TopicMessageType { } public static Set topicMessageTypeSet() { - return Sets.newHashSet(UNSPECIFIED.value, NORMAL.value, FIFO.value, DELAY.value, TRANSACTION.value, MIXED.value); + return Sets.newHashSet(UNSPECIFIED.value, NORMAL.value, FIFO.value, DELAY.value, TRANSACTION.value, + LITE.value, MIXED.value); } public String getValue() { @@ -55,6 +57,8 @@ public static TopicMessageType parseFromMessageProperty(Map mess return TopicMessageType.DELAY; } else if (messageProperty.get(MessageConst.PROPERTY_SHARDING_KEY) != null) { return TopicMessageType.FIFO; + } 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/compression/ZlibCompressor.java b/common/src/main/java/org/apache/rocketmq/common/compression/ZlibCompressor.java index e64db9b62a6..f6dcebdee92 100644 --- a/common/src/main/java/org/apache/rocketmq/common/compression/ZlibCompressor.java +++ b/common/src/main/java/org/apache/rocketmq/common/compression/ZlibCompressor.java @@ -41,7 +41,6 @@ public byte[] compress(byte[] src, int level) throws IOException { deflaterOutputStream.close(); result = byteArrayOutputStream.toByteArray(); } catch (IOException e) { - defeater.end(); throw e; } finally { try { diff --git a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java index 08a103bb270..d3f41930b9b 100644 --- a/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java +++ b/common/src/main/java/org/apache/rocketmq/common/config/AbstractRocksDBStorage.java @@ -85,6 +85,8 @@ public abstract class AbstractRocksDBStorage { protected CompactionOptions compactionOptions; protected CompactRangeOptions compactRangeOptions; + protected FlushOptions flushOptions; + protected ColumnFamilyHandle defaultCFHandle; protected final List cfOptions = new ArrayList<>(); protected final List cfHandles = new ArrayList<>(); @@ -116,6 +118,7 @@ protected void initOptions() { initTotalOrderReadOptions(); initCompactRangeOptions(); initCompactionOptions(); + initFlushOptions(); } /** @@ -168,6 +171,10 @@ protected void initCompactionOptions() { this.compactionOptions.setOutputFileSizeLimit(4 * 1024 * 1024 * 1024L); } + protected void initFlushOptions() { + this.flushOptions = new FlushOptions(); + } + public boolean hold() { if (!this.loaded || this.db == null || this.closed) { LOGGER.error("hold rocksdb Failed. {}", this.dbPath); @@ -481,6 +488,10 @@ public synchronized boolean start() { */ protected abstract void preShutdown(); + public boolean isLoaded() { + return loaded; + } + public synchronized boolean shutdown() { try { if (!this.loaded) { @@ -528,6 +539,9 @@ public synchronized boolean shutdown() { if (this.totalOrderReadOptions != null) { this.totalOrderReadOptions.close(); } + if (this.flushOptions != null) { + this.flushOptions.close(); + } //4. close db. if (db != null && !this.readOnly) { try { @@ -556,6 +570,7 @@ public synchronized boolean shutdown() { this.db = null; this.readOptions = null; this.totalOrderReadOptions = null; + this.flushOptions = null; this.writeOptions = null; this.ableWalWriteOptions = null; this.options = null; diff --git a/common/src/main/java/org/apache/rocketmq/common/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/fastjson/GenericMapSuperclassDeserializer.java b/common/src/main/java/org/apache/rocketmq/common/fastjson/GenericMapSuperclassDeserializer.java index 80a1554d123..f2639b51c1d 100644 --- a/common/src/main/java/org/apache/rocketmq/common/fastjson/GenericMapSuperclassDeserializer.java +++ b/common/src/main/java/org/apache/rocketmq/common/fastjson/GenericMapSuperclassDeserializer.java @@ -17,11 +17,10 @@ package org.apache.rocketmq.common.fastjson; -import com.alibaba.fastjson.JSONException; -import com.alibaba.fastjson.parser.DefaultJSONParser; -import com.alibaba.fastjson.parser.JSONToken; -import com.alibaba.fastjson.parser.deserializer.MapDeserializer; -import com.alibaba.fastjson.parser.deserializer.ObjectDeserializer; +import com.alibaba.fastjson2.JSONException; +import com.alibaba.fastjson2.JSONReader; +import com.alibaba.fastjson2.reader.ObjectReader; + import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; import java.util.Map; @@ -29,32 +28,40 @@ /** * workaround https://github.com/alibaba/fastjson/issues/3730 */ -public class GenericMapSuperclassDeserializer implements ObjectDeserializer { +public class GenericMapSuperclassDeserializer implements ObjectReader { public static final GenericMapSuperclassDeserializer INSTANCE = new GenericMapSuperclassDeserializer(); - @SuppressWarnings({"unchecked", "rawtypes"}) @Override - public T deserialze(DefaultJSONParser parser, Type type, Object fieldName) { + public Object readObject(JSONReader reader, Type type, Object fieldName, long features) { Class clz = (Class) type; Type genericSuperclass = clz.getGenericSuperclass(); Map map; try { - map = (Map) clz.newInstance(); + map = (Map) clz.getDeclaredConstructor().newInstance(); } catch (Exception e) { throw new JSONException("unsupport type " + type, e); } ParameterizedType parameterizedType = (ParameterizedType) genericSuperclass; Type keyType = parameterizedType.getActualTypeArguments()[0]; Type valueType = parameterizedType.getActualTypeArguments()[1]; - if (String.class == keyType) { - return (T) MapDeserializer.parseMap(parser, (Map) map, valueType, fieldName); - } else { - return (T) MapDeserializer.parseMap(parser, map, keyType, valueType, fieldName); + + if (!reader.nextIfObjectStart()) { + throw new JSONException(reader.info("expect '{', but " + reader.current())); } - } - @Override - public int getFastMatchToken() { - return JSONToken.LBRACE; + while (!reader.nextIfObjectEnd()) { + Object key; + if (keyType == String.class) { + key = reader.readFieldName(); + } else { + key = reader.getContext().getProvider().getObjectReader(keyType).readObject(reader, keyType, fieldName, features); + reader.nextIfMatch(':'); + } + + Object value = reader.getContext().getProvider().getObjectReader(valueType).readObject(reader, valueType, fieldName, features); + map.put(key, value); + reader.nextIfComma(); + } + return map; } } 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..384db82fa1c --- /dev/null +++ b/common/src/main/java/org/apache/rocketmq/common/lite/LiteSubscriptionDTO.java @@ -0,0 +1,95 @@ +/* + * 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 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 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 + + ", 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/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 24f7bdb99a5..7e0bbb2de2c 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 @@ -24,6 +24,7 @@ public class MessageConst { public static final String PROPERTY_WAIT_STORE_MSG_OK = "WAIT"; public static final String PROPERTY_DELAY_TIME_LEVEL = "DELAY"; public static final String PROPERTY_RETRY_TOPIC = "RETRY_TOPIC"; + public static final String PROPERTY_ORIGIN_GROUP = "ORIGIN_GROUP"; public static final String PROPERTY_REAL_TOPIC = "REAL_TOPIC"; public static final String PROPERTY_REAL_QUEUE_ID = "REAL_QID"; public static final String PROPERTY_TRANSACTION_PREPARED = "TRAN_MSG"; @@ -39,6 +40,7 @@ public class MessageConst { public static final String PROPERTY_MSG_REGION = "MSG_REGION"; public static final String PROPERTY_TRACE_SWITCH = "TRACE_ON"; public static final String PROPERTY_UNIQ_CLIENT_MESSAGE_ID_KEYIDX = "UNIQ_KEY"; + public static final String PROPERTY_TRANS_OFFSET = "TRANS_OFFSET"; public static final String PROPERTY_EXTEND_UNIQ_INFO = "EXTEND_UNIQ_INFO"; public static final String PROPERTY_MAX_RECONSUME_TIMES = "MAX_RECONSUME_TIMES"; public static final String PROPERTY_CONSUME_START_TIMESTAMP = "CONSUME_START_TIME"; @@ -61,6 +63,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"; @@ -88,6 +91,14 @@ public class MessageConst { public static final String KEY_SEPARATOR = " "; + public final static String INDEX_KEY_TYPE = "K"; + public final static String INDEX_UNIQUE_TYPE = "U"; + public final static String INDEX_TAG_TYPE = "T"; + + public final static String TIMER_ENGINE_TYPE = "E_T"; + public final static String TIMER_ENGINE_ROCKSDB_TIMELINE = "R"; + public final static String TIMER_ENGINE_FILE_TIME_WHEEL = "F"; + public static final HashSet STRING_HASH_SET = new HashSet<>(64); public static final String PROPERTY_TIMER_ENQUEUE_MS = "TIMER_ENQUEUE_MS"; @@ -95,6 +106,7 @@ public class MessageConst { public static final String PROPERTY_TIMER_ROLL_TIMES = "TIMER_ROLL_TIMES"; public static final String PROPERTY_TIMER_OUT_MS = "TIMER_OUT_MS"; public static final String PROPERTY_TIMER_DEL_UNIQKEY = "TIMER_DEL_UNIQKEY"; + public static final String PROPERTY_TIMER_ROLL_LABEL = "TIMER_ROLL_LABEL"; public static final String PROPERTY_TIMER_DELAY_LEVEL = "TIMER_DELAY_LEVEL"; public static final String PROPERTY_TIMER_DELAY_MS = "TIMER_DELAY_MS"; public static final String PROPERTY_CRC32 = "__CRC32#"; @@ -113,6 +125,7 @@ public class MessageConst { STRING_HASH_SET.add(PROPERTY_WAIT_STORE_MSG_OK); STRING_HASH_SET.add(PROPERTY_DELAY_TIME_LEVEL); STRING_HASH_SET.add(PROPERTY_RETRY_TOPIC); + STRING_HASH_SET.add(PROPERTY_ORIGIN_GROUP); STRING_HASH_SET.add(PROPERTY_REAL_TOPIC); STRING_HASH_SET.add(PROPERTY_REAL_QUEUE_ID); STRING_HASH_SET.add(PROPERTY_TRANSACTION_PREPARED); @@ -157,5 +170,6 @@ public class MessageConst { STRING_HASH_SET.add(PROPERTY_DLQ_ORIGIN_TOPIC); STRING_HASH_SET.add(PROPERTY_DLQ_ORIGIN_MESSAGE_ID); STRING_HASH_SET.add(PROPERTY_CRC32); + STRING_HASH_SET.add(PROPERTY_LITE_TOPIC); } } diff --git a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java index 47d45c6dfe7..ade6289c67f 100644 --- a/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java +++ b/common/src/main/java/org/apache/rocketmq/common/topic/TopicValidator.java @@ -27,8 +27,10 @@ public class TopicValidator { public static final String RMQ_SYS_SCHEDULE_TOPIC = "SCHEDULE_TOPIC_XXXX"; public static final String RMQ_SYS_BENCHMARK_TOPIC = "BenchmarkTest"; public static final String RMQ_SYS_TRANS_HALF_TOPIC = "RMQ_SYS_TRANS_HALF_TOPIC"; + public static final String RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC = "RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC"; public static final String RMQ_SYS_TRACE_TOPIC = "RMQ_SYS_TRACE_TOPIC"; public static final String RMQ_SYS_TRANS_OP_HALF_TOPIC = "RMQ_SYS_TRANS_OP_HALF_TOPIC"; + public static final String RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC = "RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC"; public static final String RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC = "TRANS_CHECK_MAX_TIME_TOPIC"; public static final String RMQ_SYS_SELF_TEST_TOPIC = "SELF_TEST_TOPIC"; public static final String RMQ_SYS_OFFSET_MOVED_EVENT = "OFFSET_MOVED_EVENT"; @@ -64,6 +66,8 @@ public class TopicValidator { SYSTEM_TOPIC_SET.add(RMQ_SYS_SELF_TEST_TOPIC); SYSTEM_TOPIC_SET.add(RMQ_SYS_OFFSET_MOVED_EVENT); SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_OFFSET_TOPIC); + SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC); + SYSTEM_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_SCHEDULE_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_TRANS_HALF_TOPIC); @@ -71,6 +75,8 @@ public class TopicValidator { NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_TRANS_CHECK_MAX_TIME_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_SELF_TEST_TOPIC); NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_OFFSET_MOVED_EVENT); + NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC); + NOT_ALLOWED_SEND_TOPIC_SET.add(RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC); // regex: ^[%|a-zA-Z0-9_-]+$ // % diff --git a/common/src/main/java/org/apache/rocketmq/common/utils/FastJsonSerializer.java b/common/src/main/java/org/apache/rocketmq/common/utils/FastJsonSerializer.java index 600054b40b8..943e8235662 100644 --- a/common/src/main/java/org/apache/rocketmq/common/utils/FastJsonSerializer.java +++ b/common/src/main/java/org/apache/rocketmq/common/utils/FastJsonSerializer.java @@ -16,23 +16,13 @@ */ package org.apache.rocketmq.common.utils; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.support.config.FastJsonConfig; +import com.alibaba.fastjson2.JSON; import org.apache.commons.lang3.SerializationException; /** * The object serializer based on fastJson */ public class FastJsonSerializer implements Serializer { - private FastJsonConfig fastJsonConfig = new FastJsonConfig(); - - public FastJsonConfig getFastJsonConfig() { - return this.fastJsonConfig; - } - - public void setFastJsonConfig(FastJsonConfig fastJsonConfig) { - this.fastJsonConfig = fastJsonConfig; - } @Override public byte[] serialize(T t) throws SerializationException { @@ -40,7 +30,7 @@ public byte[] serialize(T t) throws SerializationException { return new byte[0]; } else { try { - return JSON.toJSONBytes(this.fastJsonConfig.getCharset(), t, this.fastJsonConfig.getSerializeConfig(), this.fastJsonConfig.getSerializeFilters(), this.fastJsonConfig.getDateFormat(), JSON.DEFAULT_GENERATE_FEATURE, this.fastJsonConfig.getSerializerFeatures()); + return JSON.toJSONBytes(t); } catch (Exception var3) { throw new SerializationException("Could not serialize: " + var3.getMessage(), var3); } @@ -51,7 +41,7 @@ public byte[] serialize(T t) throws SerializationException { public T deserialize(byte[] bytes, Class type) throws SerializationException { if (bytes != null && bytes.length != 0) { try { - return JSON.parseObject(bytes, this.fastJsonConfig.getCharset(), type, this.fastJsonConfig.getParserConfig(), this.fastJsonConfig.getParseProcess(), JSON.DEFAULT_PARSER_FEATURE, this.fastJsonConfig.getFeatures()); + return JSON.parseObject(bytes, type); } catch (Exception var3) { throw new SerializationException("Could not deserialize: " + var3.getMessage(), var3); } 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 0321679ccc0..4be2898edc2 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 @@ -48,7 +48,7 @@ public void setUp() { @Test public void testTopicMessageTypeSet() { - Set expectedSet = Sets.newHashSet("UNSPECIFIED", "NORMAL", "FIFO", "DELAY", "TRANSACTION", "MIXED"); + Set expectedSet = Sets.newHashSet("UNSPECIFIED", "NORMAL", "FIFO", "DELAY", "TRANSACTION", "MIXED", "LITE"); Set actualSet = TopicMessageType.topicMessageTypeSet(); assertEquals(expectedSet, actualSet); } diff --git a/common/src/test/java/org/apache/rocketmq/common/fastjson/GenericMapSuperclassDeserializerTest.java b/common/src/test/java/org/apache/rocketmq/common/fastjson/GenericMapSuperclassDeserializerTest.java new file mode 100644 index 00000000000..36c633f699c --- /dev/null +++ b/common/src/test/java/org/apache/rocketmq/common/fastjson/GenericMapSuperclassDeserializerTest.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.fastjson; + +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONException; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +public class GenericMapSuperclassDeserializerTest { + + public static class CustomMap extends HashMap { + private static final long serialVersionUID = 1L; + } + + public static class IntKeyMap extends HashMap { + private static final long serialVersionUID = 1L; + } + + @Test + public void testBasicDeserialization() { + JSON.registerIfAbsent(CustomMap.class, GenericMapSuperclassDeserializer.INSTANCE); + String json = "{\"key1\":\"value1\",\"key2\":42,\"key3\":true}"; + CustomMap map = JSON.parseObject(json, CustomMap.class); + + assertNotNull(map); + assertEquals(3, map.size()); + assertEquals("value1", map.get("key1")); + assertEquals(42, map.get("key2")); + assertEquals(true, map.get("key3")); + } + + @Test + public void testNestedObjects() { + JSON.registerIfAbsent(CustomMap.class, GenericMapSuperclassDeserializer.INSTANCE); + String json = "{\"simple\":\"value\",\"nested\":{\"inner\":123},\"array\":[1,2,3]}"; + CustomMap map = JSON.parseObject(json, CustomMap.class); + + assertNotNull(map); + assertEquals(3, map.size()); + assertEquals("value", map.get("simple")); + + assertTrue(map.get("nested") instanceof Map); + Map nestedMap = (Map) map.get("nested"); + assertEquals(123, nestedMap.get("inner")); + + assertTrue(map.get("array") instanceof java.util.List); + java.util.List array = (java.util.List) map.get("array"); + assertEquals(3, array.size()); + assertEquals(1, array.get(0)); + assertEquals(2, array.get(1)); + assertEquals(3, array.get(2)); + } + + @Test + public void testEmptyObject() { + JSON.registerIfAbsent(CustomMap.class, GenericMapSuperclassDeserializer.INSTANCE); + String json = "{}"; + CustomMap map = JSON.parseObject(json, CustomMap.class); + + assertNotNull(map); + assertEquals(0, map.size()); + } + + @Test + public void testNonStringKey() { + JSON.registerIfAbsent(IntKeyMap.class, GenericMapSuperclassDeserializer.INSTANCE); + String json = "{1:\"one\",2:\"two\",3:\"three\"}"; + IntKeyMap map = JSON.parseObject(json, IntKeyMap.class); + + assertNotNull(map); + assertEquals(3, map.size()); + assertEquals("one", map.get(1)); + assertEquals("two", map.get(2)); + assertEquals("three", map.get(3)); + } + + @Test(expected = JSONException.class) + public void testMalformedJson() { + JSON.registerIfAbsent(CustomMap.class, GenericMapSuperclassDeserializer.INSTANCE); + String json = "{\"key\":\"missing closing brace\""; + JSON.parseObject(json, CustomMap.class); + } +} 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/container/BUILD.bazel b/container/BUILD.bazel index 4888de2228a..b828c4c431e 100644 --- a/container/BUILD.bazel +++ b/container/BUILD.bazel @@ -35,7 +35,6 @@ java_library( "@maven//:commons_codec_commons_codec", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", "@maven//:io_netty_netty_all", "@maven//:com_google_guava_guava", "@maven//:org_slf4j_slf4j_api", @@ -65,7 +64,6 @@ java_library( "@maven//:org_apache_commons_commons_lang3", "@maven//:io_netty_netty_all", "@maven//:com_google_guava_guava", - "@maven//:com_alibaba_fastjson", ], resources = glob(["src/test/resources/certs/*.pem"]) + glob(["src/test/resources/certs/*.key"]) ) diff --git a/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java b/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java index aa38fb6224a..e8debfe99b2 100644 --- a/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java +++ b/container/src/main/java/org/apache/rocketmq/container/BrokerContainer.java @@ -22,7 +22,6 @@ import org.apache.rocketmq.broker.BrokerPathConfigHelper; import org.apache.rocketmq.broker.ConfigContext; import org.apache.rocketmq.broker.out.BrokerOuterAPI; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.BrokerIdentity; import org.apache.rocketmq.common.MixAll; @@ -156,9 +155,9 @@ public boolean initialize() { this.updateNamesrvAddr(); LOG.info("Set user specified name server address: {}", this.brokerContainerConfig.getNamesrvAddr()); // also auto update namesrv if specify - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(BrokerIdentity.BROKER_CONTAINER_IDENTITY) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { BrokerContainer.this.updateNamesrvAddr(); } catch (Throwable e) { @@ -167,10 +166,10 @@ public void run0() { } }, 1000 * 10, this.brokerContainerConfig.getUpdateNamesrvAddrInterval(), TimeUnit.MILLISECONDS); } else if (this.brokerContainerConfig.isFetchNamesrvAddrByAddressServer()) { - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(BrokerIdentity.BROKER_CONTAINER_IDENTITY) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { BrokerContainer.this.brokerOuterAPI.fetchNameServerAddr(); } catch (Throwable e) { @@ -180,9 +179,9 @@ public void run0() { }, 1000 * 10, this.brokerContainerConfig.getFetchNamesrvAddrInterval(), TimeUnit.MILLISECONDS); } - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(BrokerIdentity.BROKER_CONTAINER_IDENTITY) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { BrokerContainer.this.brokerOuterAPI.refreshMetadata(); } catch (Exception e) { diff --git a/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java b/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java index 41ce28214bd..102bd4710fb 100644 --- a/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java +++ b/container/src/main/java/org/apache/rocketmq/container/InnerBrokerController.java @@ -21,7 +21,6 @@ import org.apache.rocketmq.auth.config.AuthConfig; import org.apache.rocketmq.broker.BrokerController; import org.apache.rocketmq.broker.out.BrokerOuterAPI; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.remoting.RemotingServer; @@ -82,9 +81,9 @@ public void start() throws Exception { this.registerBrokerAll(true, false, true); } - scheduledFutures.add(this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + scheduledFutures.add(this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { if (System.currentTimeMillis() < shouldStartTime) { BrokerController.LOG.info("Register to namesrv after {}", shouldStartTime); @@ -104,9 +103,9 @@ public void run0() { if (this.brokerConfig.isEnableSlaveActingMaster()) { scheduleSendHeartbeat(); - scheduledFutures.add(this.syncBrokerMemberGroupExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + scheduledFutures.add(this.syncBrokerMemberGroupExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { try { InnerBrokerController.this.syncBrokerMemberGroup(); } catch (Throwable e) { diff --git a/controller/BUILD.bazel b/controller/BUILD.bazel index e1dc61c5003..652dbd4d0c1 100644 --- a/controller/BUILD.bazel +++ b/controller/BUILD.bazel @@ -31,7 +31,7 @@ java_library( "@maven//:commons_codec_commons_codec", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:io_netty_netty_all", "@maven//:com_google_guava_guava", "@maven//:org_slf4j_slf4j_api", @@ -70,7 +70,7 @@ java_library( "@maven//:org_apache_commons_commons_lang3", "@maven//:io_netty_netty_all", "@maven//:com_google_guava_guava", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", ], resources = glob(["src/test/resources/certs/*.pem"]) + glob(["src/test/resources/certs/*.key"]) ) diff --git a/controller/pom.xml b/controller/pom.xml index 4ceb9899cca..56de653e2ab 100644 --- a/controller/pom.xml +++ b/controller/pom.xml @@ -71,4 +71,4 @@ protobuf-java-util - \ No newline at end of file + diff --git a/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManager.java b/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManager.java index c7d5d26fd05..e7a3443001d 100644 --- a/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManager.java +++ b/controller/src/main/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManager.java @@ -16,8 +16,8 @@ */ package org.apache.rocketmq.controller.impl.heartbeat; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.TypeReference; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.TypeReference; import io.netty.channel.Channel; import org.apache.rocketmq.common.ControllerConfig; import org.apache.rocketmq.common.ThreadFactoryImpl; diff --git a/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManager.java b/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManager.java index 492043235b9..046ced90c0a 100644 --- a/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManager.java +++ b/controller/src/main/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManager.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.controller.impl.manager; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import org.apache.rocketmq.common.ControllerConfig; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.controller.helper.BrokerValidPredicate; diff --git a/controller/src/test/java/org/apache/rocketmq/controller/impl/RaftBrokerHeartBeatManagerTest.java b/controller/src/test/java/org/apache/rocketmq/controller/impl/RaftBrokerHeartBeatManagerTest.java index 3c281589822..ee742cc55b2 100644 --- a/controller/src/test/java/org/apache/rocketmq/controller/impl/RaftBrokerHeartBeatManagerTest.java +++ b/controller/src/test/java/org/apache/rocketmq/controller/impl/RaftBrokerHeartBeatManagerTest.java @@ -16,13 +16,9 @@ */ package org.apache.rocketmq.controller.impl; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import io.netty.channel.Channel; import io.netty.channel.DefaultChannelPromise; -import java.util.Collections; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import org.apache.rocketmq.common.ControllerConfig; import org.apache.rocketmq.controller.impl.heartbeat.BrokerIdentityInfo; import org.apache.rocketmq.controller.impl.heartbeat.RaftBrokerHeartBeatManager; @@ -34,6 +30,11 @@ import org.mockito.Mock; import org.mockito.junit.MockitoJUnitRunner; +import java.util.Collections; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; diff --git a/controller/src/test/java/org/apache/rocketmq/controller/impl/event/EventSerializerTest.java b/controller/src/test/java/org/apache/rocketmq/controller/impl/event/EventSerializerTest.java new file mode 100644 index 00000000000..4eb482da64f --- /dev/null +++ b/controller/src/test/java/org/apache/rocketmq/controller/impl/event/EventSerializerTest.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.controller.impl.event; + +import org.apache.commons.lang3.SerializationException; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.common.utils.FastJsonSerializer; +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.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class EventSerializerTest { + + @Mock + private FastJsonSerializer serializer; + + private final EventSerializer eventSerializer = new EventSerializer(); + + @Before + public void init() throws IllegalAccessException { + FieldUtils.writeDeclaredField(eventSerializer, "serializer", serializer, true); + } + + @Test + public void testSerializeValidEventMessageShouldReturnSerializedData() { + EventMessage eventMessage = mock(EventMessage.class); + EventType eventType = EventType.APPLY_BROKER_ID_EVENT; + when(eventMessage.getEventType()).thenReturn(eventType); + when(serializer.serialize(eventMessage)).thenReturn("{\"event\":\"APPLY_BROKER_ID_EVENT\"}".getBytes()); + byte[] result = eventSerializer.serialize(eventMessage); + assertNotNull(result); + } + + @Test + public void testSerializeEventMessageWithNoEventType() { + EventMessage eventMessage = mock(EventMessage.class); + when(eventMessage.getEventType()).thenReturn(null); + assertThrows(NullPointerException.class, () -> eventSerializer.serialize(eventMessage)); + } + + @Test + public void testSerializeSerializerReturnsNullShouldReturnNull() { + EventMessage eventMessage = mock(EventMessage.class); + EventType eventType = EventType.READ_EVENT; + when(eventMessage.getEventType()).thenReturn(eventType); + when(serializer.serialize(eventMessage)).thenReturn(null); + byte[] result = eventSerializer.serialize(eventMessage); + assertNull(result); + } + + @Test + public void testSerializeSerializerThrowsException() { + EventMessage eventMessage = mock(EventMessage.class); + EventType eventType = EventType.ELECT_MASTER_EVENT; + when(eventMessage.getEventType()).thenReturn(eventType); + when(serializer.serialize(eventMessage)).thenThrow(new RuntimeException("Serialization error")); + assertThrows(RuntimeException.class, () -> eventSerializer.serialize(eventMessage)); + } + + @Test + public void testDeserializeBytesLessThanTwoReturnsNull() { + byte[] bytes = new byte[1]; + assertNull(eventSerializer.deserialize(bytes)); + } + + @Test + public void testDeserializeInvalidEventIdReturnsNull() { + assertNull(eventSerializer.deserialize(new byte[]{0, 0xF})); + } + + @Test + public void testDeserializeValidEventTypeReturnsEventMessage() throws SerializationException { + byte[] data = new byte[]{0, 0xF}; + byte[] bytes = new byte[]{0, (byte) EventType.ALTER_SYNC_STATE_SET_EVENT.getId(), data[0], data[1]}; + AlterSyncStateSetEvent alterSyncStateSetEvent = mock(AlterSyncStateSetEvent.class); + when(serializer.deserialize(any(byte[].class), eq(AlterSyncStateSetEvent.class))).thenReturn(alterSyncStateSetEvent); + EventMessage result = eventSerializer.deserialize(bytes); + assertNotNull(result); + assertTrue(result instanceof AlterSyncStateSetEvent); + } + + @Test + public void testDeserializeSerializerThrowsException() throws SerializationException { + byte[] data = new byte[]{0, 0xF}; + byte[] bytes = new byte[]{0, (byte) EventType.ALTER_SYNC_STATE_SET_EVENT.getId(), data[0], data[1]}; + when(serializer.deserialize(any(byte[].class), eq(AlterSyncStateSetEvent.class))).thenThrow(new SerializationException("Deserialization failed")); + assertThrows(SerializationException.class, () -> eventSerializer.deserialize(bytes)); + } + + @Test + public void testDeserializeValidEventTypeUnknownEventReturnsNull() throws SerializationException { + byte[] data = new byte[]{0, 0xF}; + byte[] bytes = new byte[]{0, (short) 99, data[0], data[1]}; + assertNull(eventSerializer.deserialize(bytes)); + } +} \ No newline at end of file diff --git a/controller/src/test/java/org/apache/rocketmq/controller/impl/event/ListEventSerializerTest.java b/controller/src/test/java/org/apache/rocketmq/controller/impl/event/ListEventSerializerTest.java new file mode 100644 index 00000000000..1c5e7f3a04d --- /dev/null +++ b/controller/src/test/java/org/apache/rocketmq/controller/impl/event/ListEventSerializerTest.java @@ -0,0 +1,93 @@ +/* + * 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.controller.impl.event; + +import org.apache.commons.lang3.SerializationException; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class ListEventSerializerTest { + + @Mock + private Logger logger; + + @Test + public void testSerializeEmptyList() { + List events = Collections.emptyList(); + byte[] result = ListEventSerializer.serialize(events, null); + assertNotNull(result); + assertEquals(0, result.length); + } + + @Test + public void testSerializeValidEventMessage() { + EventMessage eventMessage = new ElectMasterEvent("brokerA", 0L); + List events = Collections.singletonList(eventMessage); + byte[] result = ListEventSerializer.serialize(events, null); + assertNotNull(result); + assertTrue(result.length > 0); + } + + @Test + public void testSerializeEventMessageWithNullEventType() { + EventMessage eventMessage = mock(EventMessage.class); + when(eventMessage.getEventType()).thenReturn(null); + List events = Collections.singletonList(eventMessage); + assertThrows(NullPointerException.class, () -> ListEventSerializer.serialize(events, logger)); + } + + @Test + public void testDeserializeBytesIsNull() throws SerializationException { + List result = ListEventSerializer.deserialize(null, logger); + assertNotNull(result); + assertTrue(result.isEmpty()); + } + + @Test + public void testDeserializeBytesLengthLessThanSix() throws SerializationException { + byte[] bytes = new byte[5]; + List result = ListEventSerializer.deserialize(bytes, logger); + assertNotNull(result); + assertTrue(result.isEmpty()); + } + + @Test + public void testDeserializeValidBytesWithKnownEventType() throws SerializationException { + byte[] bytes = new byte[]{0x01, 0x00, 0x06, 0x00, 0x00, 0x00}; + assertNotNull(ListEventSerializer.deserialize(bytes, logger)); + } + + @Test + public void testDeserializeException() throws SerializationException { + byte[] bytes = new byte[]{0x01, 0x00, 0x06, 0x00, 0x00, 0x00, 0x02, 0x00, 0x06, 0x00, 0x00, 0x00}; + assertThrows(ArrayIndexOutOfBoundsException.class, () -> ListEventSerializer.deserialize(bytes, logger)); + } +} \ No newline at end of file diff --git a/controller/src/test/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManagerTest.java b/controller/src/test/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManagerTest.java new file mode 100644 index 00000000000..28c370abbeb --- /dev/null +++ b/controller/src/test/java/org/apache/rocketmq/controller/impl/heartbeat/RaftBrokerHeartBeatManagerTest.java @@ -0,0 +1,352 @@ +/* + * 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.controller.impl.heartbeat; + +import com.alibaba.fastjson2.JSON; +import io.netty.channel.Channel; +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.common.ControllerConfig; +import org.apache.rocketmq.common.JraftConfig; +import org.apache.rocketmq.controller.impl.JRaftController; +import org.apache.rocketmq.controller.impl.task.BrokerCloseChannelRequest; +import org.apache.rocketmq.controller.impl.task.CheckNotActiveBrokerRequest; +import org.apache.rocketmq.controller.impl.task.GetBrokerLiveInfoResponse; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertThrows; +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 RaftBrokerHeartBeatManagerTest { + + @Mock + private JRaftController controller; + + private RaftBrokerHeartBeatManager raftBrokerHeartBeatManager; + + @Before + public void init() throws IllegalAccessException { + ControllerConfig controllerConfig = new ControllerConfig(); + raftBrokerHeartBeatManager = new RaftBrokerHeartBeatManager(controllerConfig); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "controller", controller, true); + } + + @Test + public void testOnBrokerHeartbeatSuccess() { + Channel channel = mock(Channel.class); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + when(controller.onBrokerHeartBeat(any())).thenReturn(future); + raftBrokerHeartBeatManager.onBrokerHeartbeat("cluster1", "broker1", "127.0.0.1:10911", 1L, 3000L, channel, 1, 1000L, 500L, 1); + verify(channel, never()).close(); + } + + @Test + public void testOnBrokerHeartbeatLeaderNotAvailable() { + Channel channel = mock(Channel.class); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.CONTROLLER_NOT_LEADER, "Not Leader")); + when(controller.onBrokerHeartBeat(any())).thenReturn(future); + raftBrokerHeartBeatManager.onBrokerHeartbeat("cluster1", "broker1", "127.0.0.1:10911", 1L, 3000L, channel, 1, 1000L, 500L, 1); + verify(channel, never()).close(); + } + + @Test + public void testOnBrokerHeartbeatException() throws Exception { + Channel channel = mock(Channel.class); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + when(controller.onBrokerHeartBeat(any())).thenReturn(future); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "brokerChannelIdentityInfoMap", null, true); + assertThrows(NullPointerException.class, () -> raftBrokerHeartBeatManager.onBrokerHeartbeat("cluster1", "broker1", "127.0.0.1:10911", 1L, 3000L, channel, 1, 1000L, 500L, 1)); + } + + @Test + public void testOnBrokerChannelCloseBrokerIdentityInfoNotNullSuccess() throws Exception { + Channel channel = mock(Channel.class); + BrokerIdentityInfo brokerIdentityInfo = new BrokerIdentityInfo("cluster1", "broker1", 1L); + Map brokerChannelIdentityInfoMap = new HashMap<>(); + brokerChannelIdentityInfoMap.put(channel, brokerIdentityInfo); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "brokerChannelIdentityInfoMap", brokerChannelIdentityInfoMap, true); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + when(controller.onBrokerCloseChannel(any(BrokerCloseChannelRequest.class))).thenReturn(future); + raftBrokerHeartBeatManager.onBrokerChannelClose(channel); + verify(controller).onBrokerCloseChannel(any(BrokerCloseChannelRequest.class)); + } + + @Test + public void testOnBrokerChannelCloseBrokerIdentityInfoNotNullException() throws Exception { + Channel channel = mock(Channel.class); + BrokerIdentityInfo brokerIdentityInfo = new BrokerIdentityInfo("cluster1", "broker1", 1L); + Map brokerChannelIdentityInfoMap = new HashMap<>(); + brokerChannelIdentityInfoMap.put(channel, brokerIdentityInfo); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "brokerChannelIdentityInfoMap", brokerChannelIdentityInfoMap, true); + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new ExecutionException(new RuntimeException("Test Exception"))); + when(controller.onBrokerCloseChannel(any(BrokerCloseChannelRequest.class))).thenReturn(future); + raftBrokerHeartBeatManager.onBrokerChannelClose(channel); + verify(controller).onBrokerCloseChannel(any(BrokerCloseChannelRequest.class)); + } + + @Test + public void testOnBrokerChannelCloseBrokerIdentityInfoNull() { + Channel channel = mock(Channel.class); + raftBrokerHeartBeatManager.onBrokerChannelClose(channel); + verify(controller, never()).onBrokerCloseChannel(any(BrokerCloseChannelRequest.class)); + } + + @Test + public void testOnBrokerChannelCloseBrokerIdentityInfoNotNullTimeoutException() throws Exception { + Channel channel = mock(Channel.class); + BrokerIdentityInfo brokerIdentityInfo = new BrokerIdentityInfo("cluster1", "broker1", 1L); + Map brokerChannelIdentityInfoMap = new HashMap<>(); + brokerChannelIdentityInfoMap.put(channel, brokerIdentityInfo); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "brokerChannelIdentityInfoMap", brokerChannelIdentityInfoMap, true); + CompletableFuture future = new CompletableFuture<>(); + when(controller.onBrokerCloseChannel(any(BrokerCloseChannelRequest.class))).thenReturn(future); + raftBrokerHeartBeatManager.onBrokerChannelClose(channel); + verify(controller).onBrokerCloseChannel(any(BrokerCloseChannelRequest.class)); + } + + @Test + public void testScanNotActiveBrokerSuccess() throws Exception { + ControllerConfig controllerConfig = new ControllerConfig(); + JraftConfig jraftConfig = new JraftConfig(); + jraftConfig.setjRaftScanWaitTimeoutMs(10000); + controllerConfig.setJraftConfig(jraftConfig); + raftBrokerHeartBeatManager = new RaftBrokerHeartBeatManager(controllerConfig); + controller = mock(JRaftController.class); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "controller", controller, true); + when(controller.isLeaderState()).thenReturn(true); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "firstReceivedHeartbeatTime", 1000, true); + + List inactiveBrokers = new ArrayList<>(); + BrokerIdentityInfo brokerInfo = new BrokerIdentityInfo("testCluster", "testBroker", 1L); + inactiveBrokers.add(brokerInfo); + + RemotingCommand response = RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success"); + response.setBody(JSON.toJSONString(inactiveBrokers).getBytes()); + CompletableFuture future = CompletableFuture.completedFuture(response); + when(controller.checkNotActiveBroker(any())).thenReturn(future); + + Channel channel = mock(Channel.class); + Map brokerChannelMap = new HashMap<>(); + brokerChannelMap.put(channel, brokerInfo); + FieldUtils.writeDeclaredField(raftBrokerHeartBeatManager, "brokerChannelIdentityInfoMap", brokerChannelMap, true); + + Method method = RaftBrokerHeartBeatManager.class.getDeclaredMethod("scanNotActiveBroker"); + method.setAccessible(true); + method.invoke(raftBrokerHeartBeatManager); + + verify(controller).checkNotActiveBroker(any(CheckNotActiveBrokerRequest.class)); + } + + @Test + public void testGetBrokerLiveInfoSuccess() throws Exception { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + BrokerIdentityInfo brokerIdentityInfo = new BrokerIdentityInfo(clusterName, brokerName, brokerId); + BrokerLiveInfo expectedBrokerLiveInfo = new BrokerLiveInfo(brokerName, "127.0.0.1:10911", brokerId, System.currentTimeMillis(), 3000L, null, 1, 1000L, 500); + Map expectedResponse = new HashMap<>(); + expectedResponse.put(brokerIdentityInfo, expectedBrokerLiveInfo); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(expectedResponse).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + BrokerLiveInfo brokerLiveInfo = raftBrokerHeartBeatManager.getBrokerLiveInfo(clusterName, brokerName, brokerId); + assertEquals(expectedBrokerLiveInfo.getBrokerName(), brokerLiveInfo.getBrokerName()); + assertEquals(expectedBrokerLiveInfo.getBrokerAddr(), brokerLiveInfo.getBrokerAddr()); + assertEquals(expectedBrokerLiveInfo.getBrokerId(), brokerLiveInfo.getBrokerId()); + } + + @Test + public void testGetBrokerLiveInfoAllBrokers() throws Exception { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + BrokerIdentityInfo brokerIdentityInfo = new BrokerIdentityInfo(clusterName, brokerName, brokerId); + BrokerLiveInfo expectedBrokerLiveInfo = new BrokerLiveInfo(brokerName, "127.0.0.1:10911", brokerId, System.currentTimeMillis(), 3000L, null, 1, 1000L, 500); + Map expectedResponse = new HashMap<>(); + expectedResponse.put(brokerIdentityInfo, expectedBrokerLiveInfo); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(expectedResponse).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + BrokerLiveInfo brokerLiveInfo = raftBrokerHeartBeatManager.getBrokerLiveInfo(null, null, null); + assertNull(brokerLiveInfo); + } + + @Test + public void testIsBrokerActiveBrokerActive() throws Exception { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + BrokerLiveInfo brokerLiveInfo = new BrokerLiveInfo(brokerName, "127.0.0.1:10911", brokerId, System.currentTimeMillis(), 3000L, null, 1, 1000L, 500); + Map responseMap = new HashMap<>(); + responseMap.put(new BrokerIdentityInfo(clusterName, brokerName, brokerId), brokerLiveInfo); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(responseMap).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + assertTrue(raftBrokerHeartBeatManager.isBrokerActive(clusterName, brokerName, brokerId)); + } + + @Test + public void testIsBrokerActiveBrokerNotActive() throws Exception { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + BrokerLiveInfo brokerLiveInfo = new BrokerLiveInfo(brokerName, "127.0.0.1:10911", brokerId, System.currentTimeMillis() - 4000L, 3000L, null, 1, 1000L, 500); + Map responseMap = new HashMap<>(); + responseMap.put(new BrokerIdentityInfo(clusterName, brokerName, brokerId), brokerLiveInfo); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(responseMap).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + assertFalse(raftBrokerHeartBeatManager.isBrokerActive(clusterName, brokerName, brokerId)); + } + + @Test + public void testIsBrokerActiveException() { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new ExecutionException(new RuntimeException("Test Exception"))); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + assertFalse(raftBrokerHeartBeatManager.isBrokerActive(clusterName, brokerName, brokerId)); + } + + @Test + public void testIsBrokerActiveNoInfo() throws Exception { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(new HashMap()).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + assertFalse(raftBrokerHeartBeatManager.isBrokerActive(clusterName, brokerName, brokerId)); + } + + @Test + public void testIsBrokerActiveInvalidResponseCode() { + String clusterName = "cluster1"; + String brokerName = "broker1"; + Long brokerId = 1L; + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.RPC_TIME_OUT, "Timeout")); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + assertFalse(raftBrokerHeartBeatManager.isBrokerActive(clusterName, brokerName, brokerId)); + } + + @Test + public void testGetActiveBrokersNumAllBrokers() throws Exception { + String clusterName1 = "cluster1"; + String brokerName1 = "broker1"; + Long brokerId1 = 1L; + String clusterName2 = "cluster2"; + String brokerName2 = "broker2"; + Long brokerId2 = 2L; + BrokerIdentityInfo brokerIdentityInfo1 = new BrokerIdentityInfo(clusterName1, brokerName1, brokerId1); + BrokerIdentityInfo brokerIdentityInfo2 = new BrokerIdentityInfo(clusterName2, brokerName2, brokerId2); + BrokerLiveInfo brokerLiveInfo1 = new BrokerLiveInfo(brokerName1, "127.0.0.1:10911", brokerId1, System.currentTimeMillis(), 3000L, null, 1, 1000L, 500); + BrokerLiveInfo brokerLiveInfo2 = new BrokerLiveInfo(brokerName2, "127.0.0.1:10912", brokerId2, System.currentTimeMillis(), 3000L, null, 1, 1000L, 500); + Map responseMap = new HashMap<>(); + responseMap.put(brokerIdentityInfo1, brokerLiveInfo1); + responseMap.put(brokerIdentityInfo2, brokerLiveInfo2); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(responseMap).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + Map> activeBrokersNum = raftBrokerHeartBeatManager.getActiveBrokersNum(); + assertEquals(2, activeBrokersNum.size()); + assertEquals(1, activeBrokersNum.get(clusterName1).size()); + assertEquals(1, activeBrokersNum.get(clusterName2).size()); + assertEquals(1, (int) activeBrokersNum.get(clusterName1).get(brokerName1)); + assertEquals(1, (int) activeBrokersNum.get(clusterName2).get(brokerName2)); + } + + @Test + public void testGetActiveBrokersNum() throws Exception { + String clusterName1 = "cluster1"; + String brokerName1 = "broker1"; + Long brokerId1 = 1L; + String clusterName2 = "cluster2"; + String brokerName2 = "broker2"; + Long brokerId2 = 2L; + BrokerIdentityInfo brokerIdentityInfo1 = new BrokerIdentityInfo(clusterName1, brokerName1, brokerId1); + BrokerIdentityInfo brokerIdentityInfo2 = new BrokerIdentityInfo(clusterName2, brokerName2, brokerId2); + BrokerLiveInfo brokerLiveInfo1 = new BrokerLiveInfo(brokerName1, "127.0.0.1:10911", brokerId1, System.currentTimeMillis(), 3000L, null, 1, 1000L, 500); + BrokerLiveInfo brokerLiveInfo2 = new BrokerLiveInfo(brokerName2, "127.0.0.1:10912", brokerId2, System.currentTimeMillis() - 4000L, 3000L, null, 1, 1000L, 500); + Map responseMap = new HashMap<>(); + responseMap.put(brokerIdentityInfo1, brokerLiveInfo1); + responseMap.put(brokerIdentityInfo2, brokerLiveInfo2); + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(responseMap).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + Map> activeBrokersNum = raftBrokerHeartBeatManager.getActiveBrokersNum(); + assertEquals(1, activeBrokersNum.size()); + } + + @Test + public void testGetActiveBrokersNumException() { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(new ExecutionException(new RuntimeException("Test Exception"))); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + Map> activeBrokersNum = raftBrokerHeartBeatManager.getActiveBrokersNum(); + assertTrue(activeBrokersNum.isEmpty()); + } + + @Test + public void testGetActiveBrokersNumNoBrokers() throws Exception { + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.SUCCESS, "Success")); + future.get().writeCustomHeader(new GetBrokerLiveInfoResponse()); + future.get().setBody(JSON.toJSONString(new HashMap()).getBytes()); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + Map> activeBrokersNum = raftBrokerHeartBeatManager.getActiveBrokersNum(); + assertTrue(activeBrokersNum.isEmpty()); + } + + @Test + public void testGetActiveBrokersNumInvalidResponseCode() { + CompletableFuture future = CompletableFuture.completedFuture(RemotingCommand.createResponseCommand(ResponseCode.RPC_TIME_OUT, "Timeout")); + when(controller.getBrokerLiveInfo(any())).thenReturn(future); + Map> activeBrokersNum = raftBrokerHeartBeatManager.getActiveBrokersNum(); + assertTrue(activeBrokersNum.isEmpty()); + } +} diff --git a/controller/src/test/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManagerTest.java b/controller/src/test/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManagerTest.java new file mode 100644 index 00000000000..b47f072c2c0 --- /dev/null +++ b/controller/src/test/java/org/apache/rocketmq/controller/impl/manager/RaftReplicasInfoManagerTest.java @@ -0,0 +1,243 @@ +/* + * 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.controller.impl.manager; + +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.common.ControllerConfig; +import org.apache.rocketmq.controller.impl.event.ControllerResult; +import org.apache.rocketmq.controller.impl.heartbeat.BrokerIdentityInfo; +import org.apache.rocketmq.controller.impl.heartbeat.BrokerLiveInfo; +import org.apache.rocketmq.controller.impl.task.BrokerCloseChannelRequest; +import org.apache.rocketmq.controller.impl.task.CheckNotActiveBrokerRequest; +import org.apache.rocketmq.controller.impl.task.CheckNotActiveBrokerResponse; +import org.apache.rocketmq.controller.impl.task.GetBrokerLiveInfoRequest; +import org.apache.rocketmq.controller.impl.task.GetBrokerLiveInfoResponse; +import org.apache.rocketmq.controller.impl.task.RaftBrokerHeartBeatEventRequest; +import org.apache.rocketmq.controller.impl.task.RaftBrokerHeartBeatEventResponse; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +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.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +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.junit.Assert.fail; + +@RunWith(MockitoJUnitRunner.class) +public class RaftReplicasInfoManagerTest { + + @Mock + private ControllerConfig controllerConfig; + + private RaftReplicasInfoManager raftReplicasInfoManager; + + @Before + public void init() { + raftReplicasInfoManager = new RaftReplicasInfoManager(controllerConfig); + } + + @Test + public void testGetBrokerLiveInfoBrokerIdentityInfoIsNullReturnsAllBrokersInfo() throws IllegalAccessException { + List brokerIdentityInfos = createBrokerIdentityInfos(2); + List brokerLiveInfos = createBrokerLiveInfos(2); + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(brokerIdentityInfos.get(0), brokerLiveInfos.get(0)); + brokerLiveTable.put(brokerIdentityInfos.get(1), brokerLiveInfos.get(1)); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + GetBrokerLiveInfoRequest request = new GetBrokerLiveInfoRequest(); + ControllerResult result = raftReplicasInfoManager.getBrokerLiveInfo(request); + assertNotNull(result); + assertEquals(ResponseCode.SUCCESS, result.getResponseCode()); + } + + @Test + public void testGetBrokerLiveInfoBrokerIdentityInfoExistsReturnsBrokerInfo() throws IllegalAccessException { + BrokerIdentityInfo brokerIdentityInfo = createBrokerIdentityInfo(); + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(brokerIdentityInfo, createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + GetBrokerLiveInfoRequest request = new GetBrokerLiveInfoRequest(brokerIdentityInfo); + ControllerResult result = raftReplicasInfoManager.getBrokerLiveInfo(request); + assertNotNull(result); + assertEquals(ResponseCode.SUCCESS, result.getResponseCode()); + } + + @Test + public void testGetBrokerLiveInfoBrokerIdentityInfoNotExistsReturnsError() { + GetBrokerLiveInfoRequest request = new GetBrokerLiveInfoRequest(createBrokerIdentityInfo()); + ControllerResult result = raftReplicasInfoManager.getBrokerLiveInfo(request); + assertNotNull(result); + assertEquals(ResponseCode.CONTROLLER_BROKER_LIVE_INFO_NOT_EXISTS, result.getResponseCode()); + } + + @Test + public void testOnBrokerHeartBeatNewBrokerRegistered() { + RaftBrokerHeartBeatEventRequest request = new RaftBrokerHeartBeatEventRequest(createBrokerIdentityInfo(), createBrokerLiveInfo()); + ControllerResult result = raftReplicasInfoManager.onBrokerHeartBeat(request); + assertNotNull(result); + assertEquals(ResponseCode.SUCCESS, result.getResponseCode()); + } + + @Test + public void testOnBrokerHeartBeatExistingBrokerUpdate() throws IllegalAccessException { + BrokerIdentityInfo brokerIdentityInfo = createBrokerIdentityInfo(); + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(brokerIdentityInfo, createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + BrokerLiveInfo updatedInfo = new BrokerLiveInfo("brokerName1", "brokerAddr1", 1L, System.currentTimeMillis(), 2000L, null, 2, 200L, 2); + RaftBrokerHeartBeatEventRequest request = new RaftBrokerHeartBeatEventRequest(brokerIdentityInfo, updatedInfo); + ControllerResult result = raftReplicasInfoManager.onBrokerHeartBeat(request); + assertNotNull(result); + assertEquals(ResponseCode.SUCCESS, result.getResponseCode()); + } + + @Test + public void testOnBrokerCloseChannelBrokerIdentityInfoIsNullLogsWarningAndReturnsResult() { + assertNotNull(raftReplicasInfoManager.onBrokerCloseChannel(new BrokerCloseChannelRequest())); + } + + @Test + public void testCheckNotActiveBrokerNoBrokersInTableReturnsEmptyList() { + CheckNotActiveBrokerRequest request = new CheckNotActiveBrokerRequest(); + ControllerResult result = raftReplicasInfoManager.checkNotActiveBroker(request); + assertNotNull(result); + assertEquals(ResponseCode.SUCCESS, result.getResponseCode()); + } + + @Test + public void testCheckNotActiveBrokerBrokerLiveTableNotEmptyIdentifiesNotActiveBrokers() throws IllegalAccessException { + List brokerIdentityInfos = createBrokerIdentityInfos(2); + List brokerLiveInfos = createBrokerLiveInfos(2); + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(brokerIdentityInfos.get(0), brokerLiveInfos.get(0)); + brokerLiveTable.put(brokerIdentityInfos.get(1), brokerLiveInfos.get(1)); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + CheckNotActiveBrokerRequest request = new CheckNotActiveBrokerRequest(); + ControllerResult result = raftReplicasInfoManager.checkNotActiveBroker(request); + assertNotNull(result); + assertEquals(ResponseCode.SUCCESS, result.getResponseCode()); + assertNotNull(result.getBody()); + } + + @Test + public void testCheckNotActiveBrokerSerializeErrorSetsErrorRemark() throws IllegalAccessException { + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(createBrokerIdentityInfo(), createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + CheckNotActiveBrokerRequest request = new CheckNotActiveBrokerRequest(); + ControllerResult result = raftReplicasInfoManager.checkNotActiveBroker(request); + assertNotNull(result); + } + + @Test + public void testIsBrokerActiveBrokerLiveInfoNotNullAndActiveReturnsTrue() throws IllegalAccessException { + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(createBrokerIdentityInfo(), createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + long invokeTime = System.currentTimeMillis() + 500; + boolean brokerActive = raftReplicasInfoManager.isBrokerActive("cluster0", "broker0", 0L, invokeTime); + assertTrue(brokerActive); + } + + @Test + public void testIsBrokerActiveBrokerLiveInfoNotNullAndNotActiveReturnsFalse() throws IllegalAccessException { + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(createBrokerIdentityInfo(), createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + long invokeTime = System.currentTimeMillis(); + assertFalse(raftReplicasInfoManager.isBrokerActive("cluster1", "broker1", 1L, invokeTime)); + } + + @Test + public void testIsBrokerActiveBrokerLiveInfoNullReturnsFalse() { + assertFalse(raftReplicasInfoManager.isBrokerActive("cluster1", "broker1", 1L, System.currentTimeMillis())); + } + + @Test + public void testSerializeWithPopulatedTablesReturnsByteArray() throws Throwable { + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(createBrokerIdentityInfo(), createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + byte[] result = raftReplicasInfoManager.serialize(); + assertNotNull(result); + assertTrue(result.length > 0); + } + + @Test + public void testDeserializeFromValidDataSuccess() throws Throwable { + BrokerIdentityInfo brokerIdentityInfo = createBrokerIdentityInfo(); + Map brokerLiveTable = new HashMap<>(); + brokerLiveTable.put(brokerIdentityInfo, createBrokerLiveInfo()); + FieldUtils.writeDeclaredField(raftReplicasInfoManager, "brokerLiveTable", brokerLiveTable, true); + raftReplicasInfoManager.deserializeFrom(raftReplicasInfoManager.serialize()); + assertNotNull(brokerLiveTable); + assertEquals(1, brokerLiveTable.size()); + assertTrue(brokerLiveTable.containsKey(brokerIdentityInfo)); + } + + @Test + public void testDeserializeFromInvalidDataExceptionThrown() { + byte[] invalidData = new byte[]{0x00, 0x01, 0x02, 0x03}; + try { + raftReplicasInfoManager.deserializeFrom(invalidData); + fail("Expected an exception to be thrown"); + } catch (Throwable e) { + assertTrue(e instanceof ArrayIndexOutOfBoundsException); + } + } + + private BrokerIdentityInfo createBrokerIdentityInfo() { + return createBrokerIdentityInfos(1).get(0); + } + + private List createBrokerIdentityInfos(final int count) { + List result = new ArrayList<>(); + for (int i = 0; i < count; i++) { + result.add(new BrokerIdentityInfo("cluster" + i, "broker" + i, (long) i)); + } + return result; + } + + private BrokerLiveInfo createBrokerLiveInfo() { + return createBrokerLiveInfos(1).get(0); + } + + private List createBrokerLiveInfos(final int count) { + List result = new ArrayList<>(); + for (int i = 0; i < count; i++) { + result.add(new BrokerLiveInfo("brokerName" + i, + "brokerAddr" + i, + i, + System.currentTimeMillis(), + 1000L, + null, + 1, + 100L, + 1)); + } + return result; + } +} diff --git a/distribution/pom.xml b/distribution/pom.xml index c099c0d2907..71ca98836a3 100644 --- a/distribution/pom.xml +++ b/distribution/pom.xml @@ -124,4 +124,4 @@ - \ No newline at end of file + diff --git a/filter/BUILD.bazel b/filter/BUILD.bazel index c0d59ba6f43..76e3ef43de8 100644 --- a/filter/BUILD.bazel +++ b/filter/BUILD.bazel @@ -26,7 +26,6 @@ java_library( "@maven//:commons_validator_commons_validator", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", "@maven//:io_netty_netty_all", "@maven//:com_google_guava_guava", ], diff --git a/filter/pom.xml b/filter/pom.xml index 25c5f52bbf2..2ec7cc807dd 100644 --- a/filter/pom.xml +++ b/filter/pom.xml @@ -46,4 +46,4 @@ guava - \ No newline at end of file + diff --git a/namesrv/BUILD.bazel b/namesrv/BUILD.bazel index fec42eaa3e4..435fc29a7f8 100644 --- a/namesrv/BUILD.bazel +++ b/namesrv/BUILD.bazel @@ -31,7 +31,7 @@ java_library( "@maven//:commons_validator_commons_validator", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:io_netty_netty_all", "@maven//:ch_qos_logback_logback_classic", "@maven//:ch_qos_logback_logback_core", @@ -60,7 +60,7 @@ java_library( "@maven//:commons_cli_commons_cli", "@maven//:io_netty_netty_all", "@maven//:com_google_guava_guava", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:io_github_aliyunmq_rocketmq_slf4j_api", ], resources = glob(["src/test/resources/certs/*.pem"]) + glob(["src/test/resources/certs/*.key"]) diff --git a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java index 1ef6beadd3d..ef653129a7f 100644 --- a/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java +++ b/namesrv/src/main/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessor.java @@ -17,10 +17,8 @@ package org.apache.rocketmq.namesrv.processor; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSONWriter; import io.netty.channel.ChannelHandlerContext; -import java.util.Optional; -import java.util.concurrent.TimeUnit; import org.apache.rocketmq.common.MQVersion; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.help.FAQUrl; @@ -35,6 +33,9 @@ import org.apache.rocketmq.remoting.protocol.header.namesrv.GetRouteInfoRequestHeader; import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + public class ClientRequestProcessor implements NettyRequestProcessor { private static Logger log = LoggerFactory.getLogger(LoggerName.NAMESRV_LOGGER_NAME); @@ -81,9 +82,8 @@ public RemotingCommand getRouteInfoByTopic(ChannelHandlerContext ctx, byte[] content; Boolean standardJsonOnly = Optional.ofNullable(requestHeader.getAcceptStandardJsonOnly()).orElse(false); if (request.getVersion() >= MQVersion.Version.V4_9_4.ordinal() || standardJsonOnly) { - content = topicRouteData.encode(SerializerFeature.BrowserCompatible, - SerializerFeature.QuoteFieldNames, SerializerFeature.SkipTransientField, - SerializerFeature.MapSortField); + content = topicRouteData.encode(JSONWriter.Feature.BrowserCompatible, + JSONWriter.Feature.MapSortField); } else { content = topicRouteData.encode(); } diff --git a/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessorTest.java b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessorTest.java new file mode 100644 index 00000000000..6fa88ad6ea4 --- /dev/null +++ b/namesrv/src/test/java/org/apache/rocketmq/namesrv/processor/ClientRequestProcessorTest.java @@ -0,0 +1,195 @@ +/* + * 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.namesrv.processor; + +import io.netty.channel.ChannelHandlerContext; +import org.apache.rocketmq.common.MQVersion; +import org.apache.rocketmq.common.namesrv.NamesrvConfig; +import org.apache.rocketmq.namesrv.NamesrvController; +import org.apache.rocketmq.namesrv.routeinfo.RouteInfoManager; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; +import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.RequestCode; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.header.namesrv.GetRouteInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.QueueData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.lang.reflect.Field; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class ClientRequestProcessorTest { + + @Mock + private NamesrvController namesrvController; + + @Mock + private RouteInfoManager routeInfoManager; + + @Mock + private NamesrvConfig namesrvConfig; + + @Mock + private ChannelHandlerContext ctx; + + private ClientRequestProcessor clientRequestProcessor; + + @Before + public void setup() throws NoSuchFieldException, IllegalAccessException { + when(namesrvController.getRouteInfoManager()).thenReturn(routeInfoManager); + when(namesrvController.getNamesrvConfig()).thenReturn(namesrvConfig); + + when(namesrvConfig.getWaitSecondsForService()).thenReturn(0); + when(namesrvConfig.isNeedWaitForService()).thenReturn(true); + + clientRequestProcessor = new ClientRequestProcessor(namesrvController); + + Field startupTimeMillisField = ClientRequestProcessor.class.getDeclaredField("startupTimeMillis"); + startupTimeMillisField.setAccessible(true); + startupTimeMillisField.set(clientRequestProcessor, System.currentTimeMillis() - 60000); + } + + @Test + public void testGetRouteInfoByTopicWithHighVersionClient() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, null); + request.setVersion(MQVersion.Version.V4_9_4.ordinal()); + + GetRouteInfoRequestHeader requestHeader = new GetRouteInfoRequestHeader(); + requestHeader.setTopic("TestTopic"); + + RemotingCommand spyRequest = spy(request); + doReturn(requestHeader).when(spyRequest).decodeCommandCustomHeader(GetRouteInfoRequestHeader.class); + + TopicRouteData topicRouteData = createMockTopicRouteData(); + + when(routeInfoManager.pickupTopicRouteData("TestTopic")).thenReturn(topicRouteData); + + RemotingCommand response = clientRequestProcessor.getRouteInfoByTopic(ctx, spyRequest); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + } + + @Test + public void testGetRouteInfoByTopicWithLowVersionClientAndNoStandardJsonFlag() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, null); + request.setVersion(MQVersion.Version.V4_9_3.ordinal()); + + GetRouteInfoRequestHeader requestHeader = new GetRouteInfoRequestHeader(); + requestHeader.setTopic("TestTopic"); + requestHeader.setAcceptStandardJsonOnly(false); + + RemotingCommand spyRequest = spy(request); + doReturn(requestHeader).when(spyRequest).decodeCommandCustomHeader(GetRouteInfoRequestHeader.class); + + TopicRouteData topicRouteData = createMockTopicRouteData(); + + when(routeInfoManager.pickupTopicRouteData("TestTopic")).thenReturn(topicRouteData); + + RemotingCommand response = clientRequestProcessor.getRouteInfoByTopic(ctx, spyRequest); + + assertEquals(ResponseCode.SUCCESS, response.getCode()); + assertNotNull(response.getBody()); + } + + @Test + public void testGetRouteInfoByTopicWithNameServerNotReady() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, null); + + GetRouteInfoRequestHeader requestHeader = new GetRouteInfoRequestHeader(); + requestHeader.setTopic("TestTopic"); + + RemotingCommand spyRequest = spy(request); + doReturn(requestHeader).when(spyRequest).decodeCommandCustomHeader(GetRouteInfoRequestHeader.class); + + when(namesrvConfig.getWaitSecondsForService()).thenReturn(60); + when(namesrvConfig.isNeedWaitForService()).thenReturn(true); + + try { + Field startupTimeMillisField = ClientRequestProcessor.class.getDeclaredField("startupTimeMillis"); + startupTimeMillisField.setAccessible(true); + startupTimeMillisField.set(clientRequestProcessor, System.currentTimeMillis()); + } catch (Exception e) { + e.printStackTrace(); + } + + RemotingCommand response = clientRequestProcessor.getRouteInfoByTopic(ctx, spyRequest); + + assertEquals(ResponseCode.SYSTEM_ERROR, response.getCode()); + assertEquals("name server not ready", response.getRemark()); + } + + @Test + public void testGetRouteInfoByTopicWithTopicNotExist() throws RemotingCommandException { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, null); + + GetRouteInfoRequestHeader requestHeader = new GetRouteInfoRequestHeader(); + requestHeader.setTopic("NonExistentTopic"); + + RemotingCommand spyRequest = spy(request); + doReturn(requestHeader).when(spyRequest).decodeCommandCustomHeader(GetRouteInfoRequestHeader.class); + + when(routeInfoManager.pickupTopicRouteData("NonExistentTopic")).thenReturn(null); + + RemotingCommand response = clientRequestProcessor.getRouteInfoByTopic(ctx, spyRequest); + + assertEquals(ResponseCode.TOPIC_NOT_EXIST, response.getCode()); + assertNotNull(response.getRemark()); + } + + private TopicRouteData createMockTopicRouteData() { + TopicRouteData result = new TopicRouteData(); + + List queueDataList = new ArrayList<>(); + QueueData queueData = new QueueData(); + queueData.setBrokerName("broker-a"); + queueData.setReadQueueNums(4); + queueData.setWriteQueueNums(4); + queueData.setPerm(6); + queueData.setTopicSysFlag(0); + queueDataList.add(queueData); + result.setQueueDatas(queueDataList); + + List brokerDataList = new ArrayList<>(); + BrokerData brokerData = new BrokerData(); + brokerData.setBrokerName("broker-a"); + brokerData.setCluster("default-cluster"); + HashMap brokerAddrs = new HashMap<>(); + brokerAddrs.put(0L, "127.0.0.1:10911"); + brokerData.setBrokerAddrs(brokerAddrs); + brokerDataList.add(brokerData); + result.setBrokerDatas(brokerDataList); + + return result; + } +} \ No newline at end of file diff --git a/openmessaging/pom.xml b/openmessaging/pom.xml index 8762ac1c8e8..961698f42bb 100644 --- a/openmessaging/pom.xml +++ b/openmessaging/pom.xml @@ -42,4 +42,4 @@ ${project.version} - \ No newline at end of file + diff --git a/pom.xml b/pom.xml index 183218739ce..2aef6301705 100644 --- a/pom.xml +++ b/pom.xml @@ -88,7 +88,7 @@ - 5.3.4-SNAPSHOT + 5.3.4 UTF-8 UTF-8 ${basedir} @@ -127,7 +127,7 @@ 6.0.53 1.0-beta-4 1.4.2 - 2.0.4 + 2.1.0 1.53.0 3.20.1 1.2.10 @@ -155,6 +155,7 @@ 5.0.5 + 1.7.2 2.2 1.0.2 2.7 @@ -204,6 +205,36 @@ + + org.codehaus.mojo + flatten-maven-plugin + ${flatten-maven-plugin.version} + true + + + flatten + process-resources + + flatten + + + true + oss + + remove + remove + + + + + flatten-clean + clean + + clean + + + + org.codehaus.mojo versions-maven-plugin diff --git a/proxy/BUILD.bazel b/proxy/BUILD.bazel index 8b7915ba741..5d7fe24435a 100644 --- a/proxy/BUILD.bazel +++ b/proxy/BUILD.bazel @@ -29,7 +29,6 @@ java_library( "//srvutil", "@maven//:ch_qos_logback_logback_classic", "@maven//:ch_qos_logback_logback_core", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_github_ben_manes_caffeine_caffeine", "@maven//:com_github_luben_zstd_jni", @@ -88,7 +87,6 @@ java_library( "//srvutil", "//remoting", "@maven//:ch_qos_logback_logback_core", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_github_ben_manes_caffeine_caffeine", "@maven//:com_google_guava_guava", diff --git a/proxy/pom.xml b/proxy/pom.xml index 7b941484fd1..1150b909b65 100644 --- a/proxy/pom.xml +++ b/proxy/pom.xml @@ -115,4 +115,4 @@ jul-to-slf4j - \ No newline at end of file + diff --git a/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 15da628dc3c..42a68b1b377 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 @@ -200,6 +200,18 @@ public boolean isEmpty() { return this.receiptHandleMap.isEmpty(); } + public int getMsgCount() { + return this.receiptHandleMap.size(); + } + + public long getHandleNum() { + long handleNum = 0L; + for (Map.Entry> entry : receiptHandleMap.entrySet()) { + handleNum += entry.getValue().size(); + } + return handleNum; + } + public MessageReceiptHandle get(String msgID, String handle) { Map handleMap = this.receiptHandleMap.get(msgID); if (handleMap == null) { @@ -268,13 +280,18 @@ public MessageReceiptHandle removeOne(String msgID) { public void computeIfPresent(String msgID, String handle, Function> function) { + long timeout = ConfigurationManager.getProxyConfig().getLockTimeoutMsInHandleGroup(); + computeIfPresent(msgID, handle, function, timeout); + } + + public void computeIfPresent(String msgID, String handle, + Function> function, long lockTimeout) { Map handleMap = this.receiptHandleMap.get(msgID); if (handleMap == null) { return; } - long timeout = ConfigurationManager.getProxyConfig().getLockTimeoutMsInHandleGroup(); handleMap.computeIfPresent(new HandleKey(handle), (handleKey, handleData) -> { - Long lockTimeMs = handleData.lock(timeout); + Long lockTimeMs = handleData.lock(lockTimeout); if (lockTimeMs == null) { throw new ProxyException(ProxyExceptionCode.INTERNAL_SERVER_ERROR, "try to compute failed"); } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/Configuration.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/Configuration.java index 5b7c6c3007d..175ff438f8e 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/Configuration.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/Configuration.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.proxy.config; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import com.google.common.base.Charsets; import com.google.common.io.CharStreams; import java.io.File; diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ConfigurationManager.java b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ConfigurationManager.java index 0d8c60931d5..4660610ecd3 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/config/ConfigurationManager.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/config/ConfigurationManager.java @@ -17,8 +17,8 @@ package org.apache.rocketmq.proxy.config; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter.Feature; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.auth.config.AuthConfig; import org.apache.rocketmq.common.MixAll; @@ -59,6 +59,6 @@ public static AuthConfig getAuthConfig() { public static String formatProxyConfig() { return JSON.toJSONString(ConfigurationManager.getProxyConfig(), - SerializerFeature.PrettyFormat, SerializerFeature.WriteMapNullValue, SerializerFeature.WriteDateUseDateFormat, SerializerFeature.WriteNullListAsEmpty); + Feature.PrettyFormat, Feature.WriteMapNullValue, Feature.WriteNullListAsEmpty); } } 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 a99b0afc352..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 @@ -17,17 +17,6 @@ package org.apache.rocketmq.proxy.config; -import org.apache.commons.lang3.StringUtils; -import org.apache.rocketmq.common.MixAll; -import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.common.metrics.MetricsExporterType; -import org.apache.rocketmq.common.utils.NetworkUtil; -import org.apache.rocketmq.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; -import org.apache.rocketmq.proxy.ProxyMode; -import org.apache.rocketmq.proxy.common.ProxyException; -import org.apache.rocketmq.proxy.common.ProxyExceptionCode; - import java.net.InetAddress; import java.net.UnknownHostException; import java.time.Duration; @@ -38,6 +27,16 @@ import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.metrics.MetricsExporterType; +import org.apache.rocketmq.common.utils.NetworkUtil; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.proxy.ProxyMode; +import org.apache.rocketmq.proxy.common.ProxyException; +import org.apache.rocketmq.proxy.common.ProxyExceptionCode; public class ProxyConfig implements ConfigFile { private final static Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); @@ -119,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 @@ -204,6 +210,7 @@ public class ProxyConfig implements ConfigFile { private long renewAheadTimeMillis = TimeUnit.SECONDS.toMillis(10); private long renewMaxTimeMillis = TimeUnit.HOURS.toMillis(3); private long renewSchedulePeriodMillis = TimeUnit.SECONDS.toMillis(5); + private int returnHandleGroupThreadPoolNums = 2; private boolean enableAclRpcHookForClusterMode = false; @@ -1537,4 +1544,36 @@ public boolean isEnableMessageBodyEmptyCheck() { 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; + } + + public void setReturnHandleGroupThreadPoolNums(int returnHandleGroupThreadPoolNums) { + this.returnHandleGroupThreadPoolNums = returnHandleGroupThreadPoolNums; + } } diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessingActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessingActivity.java index 6598b9e7e65..2240ce9185f 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessingActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/AbstractMessingActivity.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/DefaultGrpcMessingActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessingActivity.java index 3c6f120ee58..63be02f9740 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessingActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/DefaultGrpcMessingActivity.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/GrpcMessagingApplication.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessagingApplication.java index 9ee3f4fddd4..bc9e7bf340a 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, // todo which thread pool + context, + request, + () -> grpcMessingActivity.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/GrpcMessingActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessingActivity.java index db15f25f6f7..bf4e9f0175f 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessingActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/grpc/v2/GrpcMessingActivity.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/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 a46bc99fef8..5844bfaef42 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,8 @@ 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.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.proxy.common.ProxyContext; @@ -107,6 +114,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 +165,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 +173,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 +191,49 @@ 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()); + 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; + } + } + 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 +257,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 +267,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 +393,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 +473,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 33a4e1312f8..3d482bc447b 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); } @@ -218,6 +220,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 76019a1ca94..75cf77118bc 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 50b6d924fda..803cafe8e0f 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,40 +173,52 @@ public void receiveMessage(ProxyContext ctx, ReceiveMessageRequest request, new PopMessageResultFilterImpl(maxAttempts), request.hasAttemptId() ? request.getAttemptId() : null, timeRemaining - ).thenAccept(popResult -> { - 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; - } - 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); - }) - .exceptionally(t -> { - writer.writeAndComplete(ctx, request, t); - return null; - }); + ); + } + + final boolean autoRenew = proxyConfig.isEnableProxyAutoRenew() && request.getAutoRenew(); + popFuture.thenAccept(popResult -> { + if (autoRenew) { + handleAutoRenew(ctx, request, group, topic, popResult, writer); + } + writer.writeAndComplete(ctx, request, popResult); + }).exceptionally(t -> { + writer.writeAndComplete(ctx, request, t); + return null; + }); } catch (Throwable t) { writer.writeAndComplete(ctx, request, t); } } + private void handleAutoRenew(ProxyContext ctx, ReceiveMessageRequest request, + String group, String topic, PopResult popResult, ReceiveMessageResponseStreamWriter writer + ) { + if (!PopStatus.FOUND.equals(popResult.getPopStatus())) { + return; + } + + 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; + } + 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(), messageExt.getProperty(MessageConst.PROPERTY_LITE_TOPIC)); + 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 d0cfc14ce00..bcb6df49726 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 f7b8014bb99..481bed418be 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 @@ -273,6 +273,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()) { @@ -379,6 +386,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 20ae3aa6c82..c4b8cbfb0dd 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; @@ -63,7 +64,7 @@ public CompletableFuture queryRoute(ProxyContext ctx, QueryR CompletableFuture future = new CompletableFuture<>(); try { validateTopic(request.getTopic()); - List addressList = this.convertToAddressList(request.getEndpoints()); + List addressList = this.convertToAddressList(ctx, request.getEndpoints()); String topicName = request.getTopic().getName(); ProxyTopicRouteData proxyTopicRouteData = this.messagingProcessor.getTopicRouteDataForProxy( @@ -101,18 +102,20 @@ public CompletableFuture queryAssignment(ProxyContext c try { validateTopicAndConsumerGroup(request.getTopic(), request.getGroup()); - List addressList = this.convertToAddressList(request.getEndpoints()); + List addressList = this.convertToAddressList(ctx, request.getEndpoints()); ProxyTopicRouteData proxyTopicRouteData = this.messagingProcessor.getTopicRouteDataForProxy( ctx, 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()) @@ -184,8 +187,7 @@ protected Permission convertToPermission(int perm) { return Permission.NONE; } - protected List convertToAddressList(Endpoints endpoints) { - + protected List convertToAddressList(ProxyContext ctx, Endpoints endpoints) { boolean useEndpointPort = ConfigurationManager.getProxyConfig().isUseEndpointPortFromRequest(); List addressList = new ArrayList<>(); @@ -198,9 +200,8 @@ protected List convertToAddressList(En org.apache.rocketmq.proxy.common.Address.AddressScheme.valueOf(endpoints.getScheme().name()), HostAndPort.fromParts(address.getHost(), port))); } - + log.debug("gRPC build address. clientId={}, addressList={}", ctx.getClientID(), addressList); return addressList; - } protected Map> buildBrokerMap( @@ -304,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/processor/channel/RemoteChannelSerializer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/channel/RemoteChannelSerializer.java index a22401a5f32..dba0655c7ed 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/processor/channel/RemoteChannelSerializer.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/processor/channel/RemoteChannelSerializer.java @@ -17,15 +17,16 @@ package org.apache.rocketmq.proxy.processor.channel; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; -import java.util.HashMap; -import java.util.Map; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.Map; + public class RemoteChannelSerializer { private static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); private static final String REMOTE_PROXY_IP_KEY = "remoteProxyIp"; diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/activity/GetTopicRouteActivity.java b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/activity/GetTopicRouteActivity.java index 56ec34fae6a..759b74fe5d7 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/activity/GetTopicRouteActivity.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/remoting/activity/GetTopicRouteActivity.java @@ -17,15 +17,10 @@ package org.apache.rocketmq.proxy.remoting.activity; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSONWriter; import com.google.common.net.HostAndPort; import io.netty.channel.ChannelHandlerContext; -import java.util.ArrayList; -import java.util.List; import org.apache.rocketmq.common.MQVersion; -import org.apache.rocketmq.remoting.protocol.ResponseCode; -import org.apache.rocketmq.remoting.protocol.header.namesrv.GetRouteInfoRequestHeader; -import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; import org.apache.rocketmq.proxy.common.Address; import org.apache.rocketmq.proxy.common.ProxyContext; import org.apache.rocketmq.proxy.config.ConfigurationManager; @@ -34,6 +29,12 @@ import org.apache.rocketmq.proxy.remoting.pipeline.RequestPipeline; import org.apache.rocketmq.proxy.service.route.ProxyTopicRouteData; import org.apache.rocketmq.remoting.protocol.RemotingCommand; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.header.namesrv.GetRouteInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; + +import java.util.ArrayList; +import java.util.List; public class GetTopicRouteActivity extends AbstractRemotingActivity { public GetTopicRouteActivity(RequestPipeline requestPipeline, @@ -57,9 +58,7 @@ protected RemotingCommand processRequest0(ChannelHandlerContext ctx, RemotingCom byte[] content; Boolean standardJsonOnly = requestHeader.getAcceptStandardJsonOnly(); if (request.getVersion() >= MQVersion.Version.V4_9_4.ordinal() || null != standardJsonOnly && standardJsonOnly) { - content = topicRouteData.encode(SerializerFeature.BrowserCompatible, - SerializerFeature.QuoteFieldNames, SerializerFeature.SkipTransientField, - SerializerFeature.MapSortField); + content = topicRouteData.encode(JSONWriter.Feature.BrowserCompatible, JSONWriter.Feature.MapSortField); } else { content = topicRouteData.encode(); } 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 5dbdea1b2e3..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 @@ -17,8 +17,8 @@ package org.apache.rocketmq.proxy.remoting.channel; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.TypeReference; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.TypeReference; import com.google.common.base.MoreObjects; import io.netty.channel.Channel; import io.netty.channel.ChannelConfig; @@ -28,14 +28,15 @@ 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; +import org.apache.rocketmq.common.utils.FutureUtils; import org.apache.rocketmq.common.utils.NetworkUtil; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.proxy.common.channel.ChannelHelper; -import org.apache.rocketmq.common.utils.ExceptionUtils; -import org.apache.rocketmq.common.utils.FutureUtils; import org.apache.rocketmq.proxy.config.ConfigurationManager; import org.apache.rocketmq.proxy.processor.channel.ChannelExtendAttributeGetter; import org.apache.rocketmq.proxy.processor.channel.ChannelProtocolType; @@ -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; import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; public class RemotingChannel extends ProxyChannel implements RemoteChannelConverter, ChannelExtendAttributeGetter { @@ -177,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 cb9b7a4ae00..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,10 +197,15 @@ 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) { - requestHeader.setBornTime(System.currentTimeMillis()); RemotingCommand request = LocalRemotingCommand.createRequestCommand(RequestCode.POP_MESSAGE, requestHeader, ctx.getLanguage()); CompletableFuture future = new CompletableFuture<>(); SimpleChannel channel = channelManager.createInvocationChannel(ctx); 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/metadata/ClusterMetadataService.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java index 70ce1d3480e..dffa800d4e5 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/metadata/ClusterMetadataService.java @@ -272,7 +272,7 @@ protected Acl getDirectly(String subject) throws Exception { @Override protected void onErr(String key, Exception e) { - log.error("load user failed. username:{}", key, e); + log.error("load acl failed. subject:{}", key, e); } } 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 0cb519306eb..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 @@ -40,6 +40,7 @@ import org.apache.rocketmq.common.thread.ThreadPoolMonitor; import org.apache.rocketmq.common.utils.AbstractStartAndShutdown; import org.apache.rocketmq.common.utils.ConcurrentHashMapUtils; +import org.apache.rocketmq.common.utils.ExceptionUtils; import org.apache.rocketmq.common.utils.StartAndShutdown; import org.apache.rocketmq.common.utils.ThreadUtils; import org.apache.rocketmq.logging.org.slf4j.Logger; @@ -53,7 +54,6 @@ import org.apache.rocketmq.proxy.common.RenewEvent; import org.apache.rocketmq.proxy.common.RenewStrategyPolicy; import org.apache.rocketmq.proxy.common.channel.ChannelHelper; -import org.apache.rocketmq.common.utils.ExceptionUtils; import org.apache.rocketmq.proxy.config.ConfigurationManager; import org.apache.rocketmq.proxy.config.ProxyConfig; import org.apache.rocketmq.proxy.service.metadata.MetadataService; @@ -70,6 +70,7 @@ public class DefaultReceiptHandleManager extends AbstractStartAndShutdown implem protected final ScheduledExecutorService scheduledExecutorService = ThreadUtils.newSingleThreadScheduledExecutor(new ThreadFactoryImpl("RenewalScheduledThread_")); protected final ThreadPoolExecutor renewalWorkerService; + protected final ThreadPoolExecutor returnHandleGroupWorkerService; public DefaultReceiptHandleManager(MetadataService metadataService, ConsumerManager consumerManager, StateEventListener eventListener) { this.metadataService = metadataService; @@ -83,6 +84,13 @@ public DefaultReceiptHandleManager(MetadataService metadataService, ConsumerMana "RenewalWorkerThread", proxyConfig.getRenewThreadPoolQueueCapacity() ); + this.returnHandleGroupWorkerService = ThreadPoolMonitor.createAndMonitor( + proxyConfig.getReturnHandleGroupThreadPoolNums(), + proxyConfig.getReturnHandleGroupThreadPoolNums() * 2, + 1, TimeUnit.MINUTES, + "ReturnHandleGroupWorkerThread", + proxyConfig.getRenewThreadPoolQueueCapacity() + ); consumerManager.appendConsumerIdsChangeListener(new ConsumerIdsChangeListener() { @Override public void handle(ConsumerGroupEvent event, String group, Object... args) { @@ -137,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; } @@ -172,7 +185,7 @@ protected void scheduleRenewTask() { protected void renewMessage(ProxyContext context, ReceiptHandleGroupKey key, ReceiptHandleGroup group, String msgID, String handleStr) { try { - group.computeIfPresent(msgID, handleStr, messageReceiptHandle -> startRenewMessage(context, key, messageReceiptHandle)); + group.computeIfPresent(msgID, handleStr, messageReceiptHandle -> startRenewMessage(context, key, messageReceiptHandle), 0); } catch (Exception e) { log.error("error when renew message. msgID:{}, handleStr:{}", msgID, handleStr, e); } @@ -237,22 +250,33 @@ protected void clearGroup(ReceiptHandleGroupKey key) { if (key == null) { return; } - ProxyConfig proxyConfig = ConfigurationManager.getProxyConfig(); ReceiptHandleGroup handleGroup = receiptHandleGroupMap.remove(key); - if (handleGroup == null) { + returnHandleGroupWorkerService.submit(() -> returnHandleGroup(key, handleGroup)); + } + + // There is no longer any waiting for lock, and only the locked handles will be processed immediately, + // while the handles that cannot be acquired will be kept waiting for the next scheduling. + private void returnHandleGroup(ReceiptHandleGroupKey key, ReceiptHandleGroup handleGroup) { + if (handleGroup == null || handleGroup.isEmpty()) { return; } + ProxyConfig proxyConfig = ConfigurationManager.getProxyConfig(); handleGroup.scan((msgID, handle, v) -> { try { handleGroup.computeIfPresent(msgID, handle, messageReceiptHandle -> { CompletableFuture future = new CompletableFuture<>(); eventListener.fireEvent(new RenewEvent(key, messageReceiptHandle, proxyConfig.getInvisibleTimeMillisWhenClear(), RenewEvent.EventType.CLEAR_GROUP, future)); return CompletableFuture.completedFuture(null); - }); + }, 0); } catch (Exception e) { log.error("error when clear handle for group. key:{}", key, e); } }); + // scheduleRenewTask will trigger cleanup again + if (!handleGroup.isEmpty()) { + log.warn("The handle cannot be completely cleared, the remaining quantity is {}, key:{}", handleGroup.getHandleNum(), key); + receiptHandleGroupMap.putIfAbsent(key, handleGroup); + } } protected void clearAllHandle() { 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/main/java/org/apache/rocketmq/proxy/service/sysmessage/AbstractSystemMessageSyncer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/AbstractSystemMessageSyncer.java index 6c19edf2f86..05eb6726188 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/AbstractSystemMessageSyncer.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/AbstractSystemMessageSyncer.java @@ -17,35 +17,36 @@ package org.apache.rocketmq.proxy.service.sysmessage; -import com.alibaba.fastjson.JSON; -import java.nio.charset.StandardCharsets; -import java.time.Duration; +import com.alibaba.fastjson2.JSON; import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory; import org.apache.rocketmq.client.producer.SendStatus; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.Message; import org.apache.rocketmq.common.message.MessageDecoder; import org.apache.rocketmq.common.topic.TopicValidator; +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.proxy.common.ProxyContext; import org.apache.rocketmq.proxy.common.ProxyException; import org.apache.rocketmq.proxy.common.ProxyExceptionCode; -import org.apache.rocketmq.common.utils.StartAndShutdown; import org.apache.rocketmq.proxy.config.ConfigurationManager; import org.apache.rocketmq.proxy.config.ProxyConfig; import org.apache.rocketmq.proxy.service.admin.AdminService; -import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory; import org.apache.rocketmq.proxy.service.route.AddressableMessageQueue; import org.apache.rocketmq.proxy.service.route.TopicRouteService; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.protocol.header.SendMessageRequestHeader; import org.apache.rocketmq.remoting.protocol.heartbeat.MessageModel; +import java.nio.charset.StandardCharsets; +import java.time.Duration; + public abstract class AbstractSystemMessageSyncer implements StartAndShutdown, MessageListenerConcurrently { protected static final Logger log = LoggerFactory.getLogger(LoggerName.PROXY_LOGGER_NAME); protected final TopicRouteService topicRouteService; diff --git a/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java b/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java index fee3ea87d27..e063d79707b 100644 --- a/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java +++ b/proxy/src/main/java/org/apache/rocketmq/proxy/service/sysmessage/HeartbeatSyncer.java @@ -17,21 +17,15 @@ package org.apache.rocketmq.proxy.service.sysmessage; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import io.netty.channel.Channel; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.TimeUnit; import org.apache.rocketmq.broker.client.ClientChannelInfo; import org.apache.rocketmq.broker.client.ConsumerGroupEvent; import org.apache.rocketmq.broker.client.ConsumerIdsChangeListener; import org.apache.rocketmq.broker.client.ConsumerManager; import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; +import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory; import org.apache.rocketmq.common.consumer.ConsumeFromWhere; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.common.thread.ThreadPoolMonitor; @@ -40,13 +34,20 @@ import org.apache.rocketmq.proxy.config.ProxyConfig; import org.apache.rocketmq.proxy.processor.channel.RemoteChannel; import org.apache.rocketmq.proxy.service.admin.AdminService; -import org.apache.rocketmq.client.impl.mqclient.MQClientAPIFactory; import org.apache.rocketmq.proxy.service.route.TopicRouteService; import org.apache.rocketmq.remoting.RPCHook; 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 java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + public class HeartbeatSyncer extends AbstractSystemMessageSyncer { protected ThreadPoolExecutor threadPoolExecutor; diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationManagerTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationManagerTest.java index 74803609ba5..2caa4e37e97 100644 --- a/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationManagerTest.java +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationManagerTest.java @@ -21,6 +21,8 @@ import org.junit.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; public class ConfigurationManagerTest extends InitConfigTest { @@ -44,4 +46,12 @@ public void testGetProxyConfig() { assertThat(ConfigurationManager.getProxyConfig()).isNotNull(); } + @Test + public void testFormatProxyConfig() { + String actual = ConfigurationManager.formatProxyConfig(); + assertNotNull(actual); + ProxyConfig expected = ConfigurationManager.getProxyConfig(); + assertTrue(actual.contains(expected.getProxyMode())); + assertTrue(actual.contains(expected.getProxyName())); + } } diff --git a/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationTest.java new file mode 100644 index 00000000000..72b7eae6fa4 --- /dev/null +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/config/ConfigurationTest.java @@ -0,0 +1,46 @@ +/* + * 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.config; + +import org.apache.rocketmq.auth.config.AuthConfig; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertNotNull; +import static org.mockito.Mockito.spy; + +public class ConfigurationTest { + + private Configuration configuration; + + @Before + public void init() { + configuration = spy(new Configuration()); + } + + @Test + public void testInit() throws Exception { + configuration.init(); + + ProxyConfig loadedProxyConfig = configuration.getProxyConfig(); + assertNotNull(loadedProxyConfig); + + AuthConfig loadedAuthConfig = configuration.getAuthConfig(); + assertNotNull(loadedAuthConfig); + } +} 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 4882a5ed8b7..dd6ceb7ceaf 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..842f34471f8 --- /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.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +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; + +@ExtendWith(MockitoExtension.class) +class ClientProcessorTest { + + @Mock + private MessagingProcessor messagingProcessor; + + @Mock + private ServiceManager serviceManager; + + @Mock + private ProxyContext ctx; + + @Mock + private SubscriptionGroupConfig groupConfig; + + private ClientProcessor clientProcessor; + + @BeforeEach + void setUp() throws Exception { + ConfigurationManager.intConfig(); + clientProcessor = new ClientProcessor(messagingProcessor, serviceManager); + } + + @Test + 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 + 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 + 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 + 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 + void testValidateLiteSubTopic_emptySubList_noException() { + String group = "group"; + Set subList = new HashSet<>(); + + assertDoesNotThrow(() -> { + clientProcessor.validateLiteSubTopic(ctx, group, subList); + }); + } + + @Test + 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 + 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 + 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 + 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 + 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 + void testGetGroupOrException_groupExists_returnsConfig() { + String group = "group"; + when(messagingProcessor.getSubscriptionGroupConfig(ctx, group)).thenReturn(groupConfig); + + SubscriptionGroupConfig result = clientProcessor.getGroupOrException(ctx, group); + assertEquals(groupConfig, result); + } + + @Test + 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/remoting/activity/GetTopicRouteActivityTest.java b/proxy/src/test/java/org/apache/rocketmq/proxy/remoting/activity/GetTopicRouteActivityTest.java new file mode 100644 index 00000000000..d956da52a16 --- /dev/null +++ b/proxy/src/test/java/org/apache/rocketmq/proxy/remoting/activity/GetTopicRouteActivityTest.java @@ -0,0 +1,171 @@ +/* + * 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.remoting.activity; + +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import org.apache.rocketmq.common.MQVersion; +import org.apache.rocketmq.proxy.common.ProxyContext; +import org.apache.rocketmq.proxy.config.ConfigurationManager; +import org.apache.rocketmq.proxy.processor.MessagingProcessor; +import org.apache.rocketmq.proxy.remoting.pipeline.RequestPipeline; +import org.apache.rocketmq.proxy.service.channel.SimpleChannel; +import org.apache.rocketmq.proxy.service.channel.SimpleChannelHandlerContext; +import org.apache.rocketmq.proxy.service.route.ProxyTopicRouteData; +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.header.namesrv.GetRouteInfoRequestHeader; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.QueueData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +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.ArrayList; +import java.util.HashMap; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class GetTopicRouteActivityTest { + + @Mock + private RequestPipeline requestPipeline; + + @Mock + private MessagingProcessor messagingProcessor; + + private GetTopicRouteActivity getTopicRouteActivity; + + private ChannelHandlerContext ctx; + + private ProxyContext context; + + @Before + public void setup() throws Exception { + getTopicRouteActivity = new GetTopicRouteActivity(requestPipeline, messagingProcessor); + + ConfigurationManager.initEnv(); + ConfigurationManager.intConfig(); + + Channel channel = new SimpleChannel(null, "0.0.0.0:0", "1.1.1.1:1"); + ctx = new SimpleChannelHandlerContext(channel); + + context = ProxyContext.create(); + } + + @Test + public void testProcessRequest0_HighVersion_SerializeWithFeatures() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, null); + request.setVersion(MQVersion.Version.V4_9_4.ordinal()); + + GetRouteInfoRequestHeader header = new GetRouteInfoRequestHeader(); + header.setTopic("TestTopic"); + header.setAcceptStandardJsonOnly(false); + request.writeCustomHeader(header); + + TopicRouteData topicRouteData = prepareTopicRouteData(); + + TopicRouteData spyTopicRouteData = Mockito.spy(topicRouteData); + + ProxyTopicRouteData proxyTopicRouteData = mock(ProxyTopicRouteData.class); + when(proxyTopicRouteData.buildTopicRouteData()).thenReturn(spyTopicRouteData); + when(messagingProcessor.getTopicRouteDataForProxy(any(ProxyContext.class), anyList(), any())) + .thenReturn(proxyTopicRouteData); + + RemotingCommand response = getTopicRouteActivity.processRequest0(ctx, request, context); + + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + + verify(spyTopicRouteData).encode( + JSONWriter.Feature.BrowserCompatible, + JSONWriter.Feature.MapSortField + ); + + TopicRouteData deserializedData = JSON.parseObject(response.getBody(), TopicRouteData.class); + assertEquals(topicRouteData.getOrderTopicConf(), deserializedData.getOrderTopicConf()); + assertEquals(topicRouteData.getQueueDatas().size(), deserializedData.getQueueDatas().size()); + } + + @Test + public void testProcessRequest0_LowVersion_StandardJsonOnly_SerializeWithFeatures() throws Exception { + RemotingCommand request = RemotingCommand.createRequestCommand(RequestCode.GET_ROUTEINFO_BY_TOPIC, null); + request.setVersion(MQVersion.Version.V4_9_3.ordinal()); + + GetRouteInfoRequestHeader header = new GetRouteInfoRequestHeader(); + header.setTopic("TestTopic"); + header.setAcceptStandardJsonOnly(true); + request.writeCustomHeader(header); + + TopicRouteData topicRouteData = prepareTopicRouteData(); + + TopicRouteData spyTopicRouteData = Mockito.spy(topicRouteData); + + ProxyTopicRouteData proxyTopicRouteData = mock(ProxyTopicRouteData.class); + when(proxyTopicRouteData.buildTopicRouteData()).thenReturn(spyTopicRouteData); + when(messagingProcessor.getTopicRouteDataForProxy(any(ProxyContext.class), anyList(), any())) + .thenReturn(proxyTopicRouteData); + + RemotingCommand response = getTopicRouteActivity.processRequest0(ctx, request, context); + + assertNotNull(response); + assertEquals(ResponseCode.SUCCESS, response.getCode()); + + verify(spyTopicRouteData).encode(); + } + + private TopicRouteData prepareTopicRouteData() { + TopicRouteData result = new TopicRouteData(); + result.setOrderTopicConf("orderTopicConf"); + + List queueDatas = new ArrayList<>(); + QueueData queueData = new QueueData(); + queueData.setBrokerName("broker-a"); + queueData.setPerm(6); + queueData.setReadQueueNums(4); + queueData.setWriteQueueNums(4); + queueData.setTopicSysFlag(0); + queueDatas.add(queueData); + result.setQueueDatas(queueDatas); + + List brokerDatas = new ArrayList<>(); + BrokerData brokerData = new BrokerData(); + brokerData.setBrokerName("broker-a"); + HashMap brokerAddrs = new HashMap<>(); + brokerAddrs.put(0L, "127.0.0.1:10911"); + brokerData.setBrokerAddrs(brokerAddrs); + brokerDatas.add(brokerData); + result.setBrokerDatas(brokerDatas); + return result; + } +} \ No newline at end of file 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..4543fcdf0cf --- /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.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +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; + +@ExtendWith(MockitoExtension.class) +public class LiteSubscriptionServiceTest { + + @Mock + private TopicRouteService topicRouteService; + + @Mock + private MQClientAPIFactory mqClientAPIFactory; + + @Mock + private MQClientAPIExt mqClientAPIExt; + + private LiteSubscriptionService liteSubscriptionService; + + @BeforeEach + 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); + + when(mqClientAPIExt.syncLiteSubscriptionAsync(anyString(), any(LiteSubscriptionDTO.class), anyLong())) + .thenReturn(CompletableFuture.completedFuture(null)) + .thenReturn(CompletableFuture.supplyAsync(() -> { + throw new RuntimeException("Broker sync failed"); + })); + + 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/BUILD.bazel b/remoting/BUILD.bazel index 9f806be7635..62273e5e9d0 100644 --- a/remoting/BUILD.bazel +++ b/remoting/BUILD.bazel @@ -22,7 +22,7 @@ java_library( visibility = ["//visibility:public"], deps = [ "//common", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_google_guava_guava", "@maven//:com_google_code_findbugs_jsr305", "@maven//:com_squareup_okio_okio_jvm", @@ -51,7 +51,9 @@ java_library( ":remoting", "//common", "//:test_deps", + "@maven//:org_objenesis_objenesis", "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_google_code_gson_gson", "@maven//:com_google_guava_guava", "@maven//:com_google_code_findbugs_jsr305", diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java index 973d229befe..e39967e3f8d 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/netty/NettyRemotingClient.java @@ -16,8 +16,8 @@ */ package org.apache.rocketmq.remoting.netty; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.TypeReference; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.TypeReference; import com.google.common.base.Stopwatch; import io.netty.bootstrap.Bootstrap; import io.netty.buffer.PooledByteBufAllocator; @@ -40,11 +40,34 @@ import io.netty.handler.timeout.IdleStateEvent; import io.netty.handler.timeout.IdleStateHandler; import io.netty.resolver.NoopAddressResolverGroup; +import io.netty.util.AttributeKey; import io.netty.util.HashedWheelTimer; import io.netty.util.Timeout; import io.netty.util.TimerTask; import io.netty.util.concurrent.DefaultEventExecutorGroup; import io.netty.util.concurrent.EventExecutorGroup; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.Pair; +import org.apache.rocketmq.common.ThreadFactoryImpl; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.utils.FutureUtils; +import org.apache.rocketmq.common.utils.NetworkUtil; +import org.apache.rocketmq.common.utils.ThreadUtils; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.ChannelEventListener; +import org.apache.rocketmq.remoting.InvokeCallback; +import org.apache.rocketmq.remoting.RemotingClient; +import org.apache.rocketmq.remoting.common.RemotingHelper; +import org.apache.rocketmq.remoting.exception.RemotingConnectException; +import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; +import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; +import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException; +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.proxy.SocksProxyConfig; + import java.io.IOException; import java.net.InetSocketAddress; import java.net.SocketAddress; @@ -69,33 +92,15 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.commons.lang3.StringUtils; -import org.apache.rocketmq.common.Pair; -import org.apache.rocketmq.common.ThreadFactoryImpl; -import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.common.utils.FutureUtils; -import org.apache.rocketmq.common.utils.NetworkUtil; -import org.apache.rocketmq.common.utils.ThreadUtils; -import org.apache.rocketmq.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; -import org.apache.rocketmq.remoting.ChannelEventListener; -import org.apache.rocketmq.remoting.InvokeCallback; -import org.apache.rocketmq.remoting.RemotingClient; -import org.apache.rocketmq.remoting.common.RemotingHelper; -import org.apache.rocketmq.remoting.exception.RemotingConnectException; -import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; -import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; -import org.apache.rocketmq.remoting.exception.RemotingTooMuchRequestException; -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.proxy.SocksProxyConfig; import static org.apache.rocketmq.remoting.common.RemotingHelper.convertChannelFutureToCompletableFuture; public class NettyRemotingClient extends NettyRemotingAbstract implements RemotingClient { private static final Logger LOGGER = LoggerFactory.getLogger(LoggerName.ROCKETMQ_REMOTING_NAME); + private static final AttributeKey CHANNEL_WRAPPER_ATTRIBUTE_KEY = AttributeKey.valueOf( + "channelWrapper"); + private static final long LOCK_TIMEOUT_MILLIS = 3000; private static final long MIN_CLOSE_TIMEOUT_MILLIS = 100; @@ -106,7 +111,6 @@ public class NettyRemotingClient extends NettyRemotingAbstract implements Remoti private final Map proxyMap = new HashMap<>(); private final ConcurrentHashMap bootstrapMap = new ConcurrentHashMap<>(); private final ConcurrentMap channelTables = new ConcurrentHashMap<>(); - private final ConcurrentMap channelWrapperTables = new ConcurrentHashMap<>(); private final HashedWheelTimer timer = new HashedWheelTimer(r -> new Thread(r, "ClientHouseKeepingService")); @@ -381,7 +385,6 @@ public void shutdown() { channel.getValue().close(); } - this.channelWrapperTables.clear(); this.channelTables.clear(); this.eventLoopGroupWorker.shutdownGracefully(); @@ -439,7 +442,8 @@ public void closeChannel(final String addr, final Channel channel) { } if (removeItemFromTable) { - ChannelWrapper channelWrapper = this.channelWrapperTables.remove(channel); + ChannelWrapper channelWrapper = + RemotingHelper.getAttributeValue(CHANNEL_WRAPPER_ATTRIBUTE_KEY, channel); if (channelWrapper != null && channelWrapper.tryClose(channel)) { this.channelTables.remove(addrRemote); } @@ -487,7 +491,8 @@ public void closeChannel(final Channel channel) { } if (removeItemFromTable) { - ChannelWrapper channelWrapper = this.channelWrapperTables.remove(channel); + ChannelWrapper channelWrapper = + RemotingHelper.getAttributeValue(CHANNEL_WRAPPER_ATTRIBUTE_KEY, channel); if (channelWrapper != null && channelWrapper.tryClose(channel)) { this.channelTables.remove(addrRemote); } @@ -724,7 +729,6 @@ private ChannelWrapper createChannel(String addr) { LOGGER.info("createChannel: begin to connect remote host[{}] asynchronously", addr); ChannelWrapper cw = new ChannelWrapper(addr, channelFuture); this.channelTables.put(addr, cw); - this.channelWrapperTables.put(channelFuture.channel(), cw); return cw; } @@ -831,17 +835,12 @@ public CompletableFuture invokeImpl(final Channel channel, final if (response.getCode() == ResponseCode.GO_AWAY) { if (nettyClientConfig.isEnableReconnectForGoAway()) { LOGGER.info("Receive go away from channelId={}, channel={}", channel.id(), channel); - ChannelWrapper channelWrapper = channelWrapperTables.computeIfPresent(channel, (channel0, channelWrapper0) -> { - try { - if (channelWrapper0.reconnect(channel0)) { - LOGGER.info("Receive go away from channelId={}, channel={}, recreate the channelId={}", channel0.id(), channel0, channelWrapper0.getChannel().id()); - channelWrapperTables.put(channelWrapper0.getChannel(), channelWrapper0); - } - } catch (Throwable t) { - LOGGER.error("Channel {} reconnect error", channelWrapper0, t); - } - return channelWrapper0; - }); + ChannelWrapper channelWrapper = RemotingHelper.getAttributeValue(CHANNEL_WRAPPER_ATTRIBUTE_KEY, + channel); + if (channelWrapper != null && channelWrapper.reconnect(channel)) { + LOGGER.info("Receive go away from channelId={}, channel={}, recreate the channelId={}", + channel.id(), channel, channelWrapper.getChannel().id()); + } if (channelWrapper != null && !channelWrapper.isWrapperOf(channel)) { RemotingCommand retryRequest = RemotingCommand.createRequestCommand(request.getCode(), request.readCustomHeader()); retryRequest.setBody(request.getBody()); @@ -1006,6 +1005,7 @@ public ChannelWrapper(String address, ChannelFuture channelFuture) { this.channelFuture = channelFuture; this.lastResponseTime = System.currentTimeMillis(); this.channelAddress = address; + RemotingHelper.setPropertyToAttr(channelFuture.channel(), CHANNEL_WRAPPER_ATTRIBUTE_KEY, this); } public boolean isOK() { @@ -1055,10 +1055,13 @@ public boolean reconnect(Channel channel) { if (isWrapperOf(channel)) { channelToClose = channelFuture; channelFuture = doConnect(channelAddress); + RemotingHelper.setPropertyToAttr(channelFuture.channel(), CHANNEL_WRAPPER_ATTRIBUTE_KEY, this); return true; } else { LOGGER.warn("channelWrapper has reconnect, so do nothing, now channelId={}, input channelId={}",getChannel().id(), channel.id()); } + } catch (Throwable t) { + LOGGER.error("ChannelWrapper {} reconnect error", this, t); } finally { lock.writeLock().unlock(); } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/BitSetSerializerDeserializer.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/BitSetSerializerDeserializer.java index 8f53c0250bf..3fe8eb2a0b3 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/BitSetSerializerDeserializer.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/BitSetSerializerDeserializer.java @@ -16,37 +16,46 @@ */ package org.apache.rocketmq.remoting.protocol; -import com.alibaba.fastjson.parser.DefaultJSONParser; -import com.alibaba.fastjson.parser.JSONToken; -import com.alibaba.fastjson.parser.deserializer.ObjectDeserializer; -import com.alibaba.fastjson.serializer.JSONSerializer; -import com.alibaba.fastjson.serializer.ObjectSerializer; -import com.alibaba.fastjson.serializer.SerializeWriter; +import com.alibaba.fastjson2.JSONReader; +import com.alibaba.fastjson2.JSONWriter; +import com.alibaba.fastjson2.reader.ObjectReader; +import com.alibaba.fastjson2.writer.ObjectWriter; -import java.io.IOException; import java.lang.reflect.Type; +import java.util.Base64; import java.util.BitSet; -public class BitSetSerializerDeserializer implements ObjectSerializer, ObjectDeserializer { +public class BitSetSerializerDeserializer implements ObjectReader, ObjectWriter { @Override - public void write(JSONSerializer serializer, Object object, Object fieldName, Type fieldType, int features) throws IOException { - SerializeWriter out = serializer.out; - out.writeByteArray(((BitSet) object).toByteArray()); + public void write(JSONWriter writer, Object object, Object fieldName, Type fieldType, long features) { + if (object == null) { + writer.writeBase64(null); + } else { + writer.writeBase64(((BitSet) object).toByteArray()); + } } - @SuppressWarnings("unchecked") @Override - public T deserialze(DefaultJSONParser parser, Type type, Object fieldName) { - byte[] bytes = parser.parseObject(byte[].class); - if (bytes != null) { - return (T) BitSet.valueOf(bytes); + public BitSet readObject(JSONReader reader, Type fieldType, Object fieldName, long features) { + if (reader.nextIfNull()) { + return null; + } + String base64 = reader.readString(); + if (base64 == null || base64.isEmpty()) { + return null; } - return null; + byte[] bytes = Base64.getDecoder().decode(base64); + return BitSet.valueOf(bytes); + } + + @Override + public long getFeatures() { + return 0L; } @Override - public int getFastMatchToken() { - return JSONToken.LITERAL_STRING; + public Class getObjectClass() { + return ObjectReader.super.getObjectClass(); } } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java index 9b2b0f07b4f..e08a1627d15 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingCommand.java @@ -16,10 +16,20 @@ */ package org.apache.rocketmq.remoting.protocol; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; import com.google.common.base.Stopwatch; import io.netty.buffer.ByteBuf; import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.BoundaryType; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.CommandCallback; +import org.apache.rocketmq.remoting.CommandCustomHeader; +import org.apache.rocketmq.remoting.annotation.CFNotNull; +import org.apache.rocketmq.remoting.exception.RemotingCommandException; + import java.lang.annotation.Annotation; import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; @@ -34,16 +44,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.commons.lang3.StringUtils; -import org.apache.rocketmq.common.BoundaryType; -import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; -import org.apache.rocketmq.remoting.CommandCallback; -import org.apache.rocketmq.remoting.CommandCustomHeader; -import org.apache.rocketmq.remoting.annotation.CFNotNull; -import org.apache.rocketmq.remoting.exception.RemotingCommandException; - public class RemotingCommand { public static final String SERIALIZE_TYPE_PROPERTY = "rocketmq.serialize.type"; public static final String SERIALIZE_TYPE_ENV = "ROCKETMQ_SERIALIZE_TYPE"; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingSerializable.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingSerializable.java index 139a7043d84..c4e4da14684 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingSerializable.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RemotingSerializable.java @@ -16,8 +16,9 @@ */ package org.apache.rocketmq.remoting.protocol; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; + import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; import java.util.List; @@ -34,7 +35,10 @@ public static byte[] encode(final Object obj) { } public static String toJson(final Object obj, boolean prettyFormat) { - return JSON.toJSONString(obj, prettyFormat); + if (prettyFormat) { + return JSON.toJSONString(obj, JSONWriter.Feature.PrettyFormat); + } + return JSON.toJSONString(obj); } public static T decode(final byte[] data, Class classOfT) { @@ -74,7 +78,7 @@ public byte[] encode() { * @param features Features to apply * @return serialized data. */ - public byte[] encode(SerializerFeature...features) { + public byte[] encode(JSONWriter.Feature... features) { final String json = JSON.toJSONString(this, features); return json.getBytes(CHARSET_UTF8); } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/RequestCode.java index 8b2749eaae2..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; @@ -202,9 +214,9 @@ public class RequestCode { public static final int PUSH_REPLY_MESSAGE_TO_CLIENT = 326; public static final int ADD_WRITE_PERM_OF_BROKER = 327; - + public static final int GET_ALL_PRODUCER_INFO = 328; - + public static final int DELETE_EXPIRED_COMMITLOG = 329; public static final int GET_TOPIC_CONFIG = 351; @@ -297,4 +309,6 @@ public class RequestCode { public static final int AUTH_DELETE_ACL = 3008; public static final int AUTH_GET_ACL = 3009; public static final int AUTH_LIST_ACL = 3010; + + public static final int SWITCH_TIMER_ENGINE = 5001; } diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/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/BatchAck.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/BatchAck.java index 82dcd8567ea..6107bdce024 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/BatchAck.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/BatchAck.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.remoting.protocol.body; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; import org.apache.rocketmq.remoting.protocol.BitSetSerializerDeserializer; import java.io.Serializable; 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/body/RegisterBrokerBody.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/RegisterBrokerBody.java index 7312b702331..03920855bbe 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/RegisterBrokerBody.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/body/RegisterBrokerBody.java @@ -17,7 +17,17 @@ package org.apache.rocketmq.remoting.protocol.body; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; +import org.apache.rocketmq.common.MQVersion; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.protocol.DataVersion; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; +import org.apache.rocketmq.remoting.protocol.statictopic.TopicQueueMappingInfo; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -30,15 +40,6 @@ import java.util.zip.Deflater; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; -import org.apache.rocketmq.common.MQVersion; -import org.apache.rocketmq.common.MixAll; -import org.apache.rocketmq.common.TopicConfig; -import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; -import org.apache.rocketmq.remoting.protocol.DataVersion; -import org.apache.rocketmq.remoting.protocol.RemotingSerializable; -import org.apache.rocketmq.remoting.protocol.statictopic.TopicQueueMappingInfo; public class RegisterBrokerBody extends RemotingSerializable { 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/QueryMessageRequestHeader.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java index 1d2a53a6ce1..1d4e4ab6fbf 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/header/QueryMessageRequestHeader.java @@ -42,6 +42,8 @@ public class QueryMessageRequestHeader extends TopicRequestHeader { private Long beginTimestamp; @CFNotNull private Long endTimestamp; + private String indexType; + private String lastKey; @Override public void checkFields() throws RemotingCommandException { @@ -87,4 +89,20 @@ public Long getEndTimestamp() { public void setEndTimestamp(Long endTimestamp) { this.endTimestamp = endTimestamp; } + + public String getIndexType() { + return indexType; + } + + public void setIndexType(String indexType) { + this.indexType = indexType; + } + + public String getLastKey() { + return lastKey; + } + + public void setLastKey(String lastKey) { + this.lastKey = lastKey; + } } diff --git a/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/HeartbeatData.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/HeartbeatData.java index f7b4b9faeff..d915bf0d16a 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/HeartbeatData.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/HeartbeatData.java @@ -20,10 +20,12 @@ */ package org.apache.rocketmq.remoting.protocol.heartbeat; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + import java.util.HashSet; import java.util.Set; -import com.alibaba.fastjson.JSON; -import org.apache.rocketmq.remoting.protocol.RemotingSerializable; public class HeartbeatData extends RemotingSerializable { private String clientID; @@ -79,7 +81,7 @@ public String toString() { } public int computeHeartbeatFingerprint() { - HeartbeatData heartbeatDataCopy = JSON.parseObject(JSON.toJSONString(this), HeartbeatData.class); + HeartbeatData heartbeatDataCopy = JSON.parseObject(JSON.toJSONString(this, JSONWriter.Feature.ReferenceDetection), HeartbeatData.class); for (ConsumerData consumerData : heartbeatDataCopy.getConsumerDataSet()) { for (SubscriptionData subscriptionData : consumerData.getSubscriptionDataSet()) { subscriptionData.setSubVersion(0L); @@ -88,6 +90,6 @@ public int computeHeartbeatFingerprint() { heartbeatDataCopy.setWithoutSub(false); heartbeatDataCopy.setHeartbeatFingerprint(0); heartbeatDataCopy.setClientID(""); - return JSON.toJSONString(heartbeatDataCopy).hashCode(); + return JSON.toJSONString(heartbeatDataCopy, JSONWriter.Feature.ReferenceDetection).hashCode(); } } 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/heartbeat/SubscriptionData.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/SubscriptionData.java index 59088fc42e5..f110fac0613 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/SubscriptionData.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/heartbeat/SubscriptionData.java @@ -20,10 +20,11 @@ */ package org.apache.rocketmq.remoting.protocol.heartbeat; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; +import org.apache.rocketmq.common.filter.ExpressionType; + import java.util.HashSet; import java.util.Set; -import org.apache.rocketmq.common.filter.ExpressionType; public class SubscriptionData implements Comparable { public final static String SUB_ALL = "*"; diff --git a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/GroupRetryPolicy.java b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/GroupRetryPolicy.java index 14d5e537697..00bf2e9576f 100644 --- a/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/GroupRetryPolicy.java +++ b/remoting/src/main/java/org/apache/rocketmq/remoting/protocol/subscription/GroupRetryPolicy.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.remoting.protocol.subscription; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; import com.google.common.base.MoreObjects; public class GroupRetryPolicy { 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 c9c2a8090ce..9d38992064a 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 @@ -17,12 +17,23 @@ package org.apache.rocketmq.remoting.protocol.subscription; +import com.alibaba.fastjson.annotation.JSONField; 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; public class SubscriptionGroupConfig { @@ -173,6 +184,58 @@ public void setAttributes(Map attributes) { this.attributes = attributes; } + @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/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableCompatTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableCompatTest.java new file mode 100644 index 00000000000..35c1c7b8912 --- /dev/null +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableCompatTest.java @@ -0,0 +1,425 @@ +/* + * 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; + +import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.JSON; +import org.apache.rocketmq.remoting.protocol.body.BatchAck; +import org.junit.Test; +import org.objenesis.ObjenesisStd; +import org.reflections.Reflections; + +import java.lang.reflect.Array; +import java.lang.reflect.Field; +import java.lang.reflect.Modifier; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class RemotingSerializableCompatTest { + + @Test + public void testCompatibilityCheck() { + Reflections reflections = new Reflections("org.apache.rocketmq.remoting.protocol"); + Set> subTypes = reflections.getSubTypesOf(RemotingSerializable.class); + + for (Class clazz : subTypes) { + if (clazz.isInterface() || Modifier.isAbstract(clazz.getModifiers()) || clazz.getSimpleName().endsWith("Test") + || clazz.isAnonymousClass() || clazz.getName().contains("$")) { + continue; + } + try { + RemotingSerializable instance; + try { + instance = clazz.getDeclaredConstructor().newInstance(); + } catch (NoSuchMethodException e) { + instance = allocateInstance(clazz); + } + fillDefaultFields(instance, clazz); + assertTrue(checkCompatible(instance, clazz)); + } catch (Exception e) { + System.err.printf("Class %s: incompatible, error: %s\n", clazz.getName(), e.getMessage()); + } + } + } + + @Test + public void testCompatibilityCheckWithBitSet() { + BitSet bitSet = new BitSet(); + bitSet.set(1); + bitSet.set(3); + bitSet.set(5); + String fastjson1Str = "{\"b\":\"Kg==\",\"c\":\"DEFAULT_CONSUMER\",\"it\":5000,\"pt\":1760694281326,\"q\":1,\"r\":\"0\",\"rq\":2,\"so\":100,\"t\":\"myTopic\"}"; + BatchAck batchAck = JSON.parseObject(fastjson1Str, BatchAck.class); + assertEquals(bitSet, batchAck.getBitSet()); + assertEquals("DEFAULT_CONSUMER", batchAck.getConsumerGroup()); + assertEquals(5000, batchAck.getInvisibleTime()); + assertEquals(1760694281326L, batchAck.getPopTime()); + assertEquals(1, batchAck.getQueueId()); + assertEquals("0", batchAck.getRetry()); + assertEquals(2, batchAck.getReviveQueueId()); + assertEquals(100, batchAck.getStartOffset()); + assertEquals("myTopic", batchAck.getTopic()); + } + + private void fillDefaultFields(final Object obj, final Class clazz) throws Exception { + if (null == clazz || clazz == Object.class) { + return; + } + for (Field field : clazz.getDeclaredFields()) { + if (Modifier.isStatic(field.getModifiers())) { + continue; + } + field.setAccessible(true); + Class type = field.getType(); + + if (type.isArray()) { + Class componentType = type.getComponentType(); + Object arr = Array.newInstance(componentType, 1); + Object element = createElementOrDefault(componentType); + if (element != null) { + Array.set(arr, 0, element); + } + field.set(obj, arr); + } else if (Properties.class.isAssignableFrom(type)) { + field.set(obj, new Properties()); + } else if (type.isEnum()) { + Object[] enumConstants = type.getEnumConstants(); + if (enumConstants != null && enumConstants.length > 0) { + field.set(obj, enumConstants[0]); + } + } else if (ConcurrentHashMap.KeySetView.class.isAssignableFrom(type)) { + field.set(obj, ConcurrentHashMap.newKeySet()); + } else if (ConcurrentHashMap.class.isAssignableFrom(type) || ConcurrentMap.class.isAssignableFrom(type)) { + field.set(obj, new ConcurrentHashMap<>()); + } else if (Set.class.isAssignableFrom(type)) { + Set set = type.isInterface() ? new HashSet<>() : (Set) type.getDeclaredConstructor().newInstance(); + Class genericType = getFirstGenericType(field); + Object element = createElementOrDefault(genericType); + if (element != null) + set.add(element); + field.set(obj, set); + } else if (List.class.isAssignableFrom(type)) { + List list = new ArrayList<>(); + Class genericType = getFirstGenericType(field); + Object element = createElementOrDefault(genericType); + if (null != element) { + list.add(element); + } + field.set(obj, list); + } else if (Map.class.isAssignableFrom(type)) { + Map map = type.isInterface() ? new HashMap<>() : (Map) type.getDeclaredConstructor().newInstance(); + Class keyType = getGenericType(field, 0); + Class valueType = getGenericType(field, 1); + Object key = createElementOrDefault(keyType); + Object value = createElementOrDefault(valueType); + if (null != key && null != value) { + map.put(key, value); + } + field.set(obj, map); + } else if (type == AtomicLong.class) { + field.set(obj, new AtomicLong(1)); + } else { + Object value = getDefaultValue(type); + if (null != value) { + field.set(obj, value); + } else if (!type.isPrimitive() && !type.getName().startsWith("java.")) { + Object subObj; + try { + subObj = type.getDeclaredConstructor().newInstance(); + } catch (NoSuchMethodException e) { + subObj = allocateInstance(type); + } + fillDefaultFields(subObj, type); + field.set(obj, subObj); + } + } + } + fillDefaultFields(obj, clazz.getSuperclass()); + } + + private Object createElementOrDefault(final Class type) throws Exception { + if (null == type) { + return null; + } + Object value = getDefaultValue(type); + if (null != value) { + return value; + } + if (type.isEnum()) { + Object[] enumConstants = type.getEnumConstants(); + if (null != enumConstants && enumConstants.length > 0) { + return enumConstants[0]; + } + return null; + } + if (type.isArray()) { + Class componentType = type.getComponentType(); + Object arr = Array.newInstance(componentType, 1); + Object element = createElementOrDefault(componentType); + if (null != element) { + Array.set(arr, 0, element); + } + return arr; + } + if (!type.isPrimitive()) { + Object obj; + try { + obj = type.getDeclaredConstructor().newInstance(); + } catch (NoSuchMethodException e) { + obj = allocateInstance(type); + } + fillDefaultFields(obj, type); + return obj; + } + return null; + } + + private Class getFirstGenericType(final Field field) { + return getGenericType(field, 0); + } + + private Class getGenericType(final Field field, final int index) { + try { + java.lang.reflect.Type genericType = field.getGenericType(); + if (genericType instanceof java.lang.reflect.ParameterizedType) { + java.lang.reflect.Type[] types = ((java.lang.reflect.ParameterizedType) genericType).getActualTypeArguments(); + if (types.length > index && types[index] instanceof Class) { + return (Class) types[index]; + } + } + } catch (Exception ignored) { + } + return null; + } + + private Object getDefaultValue(final Class type) { + if (null == type) { + return null; + } + if (type == boolean.class || type == Boolean.class) { + return false; + } + if (type == byte.class || type == Byte.class) { + return (byte) 1; + } + if (type == short.class || type == Short.class) { + return (short) 1; + } + if (type == int.class || type == Integer.class) { + return 1; + } + if (type == long.class || type == Long.class) { + return 1L; + } + if (type == float.class || type == Float.class) { + return 1f; + } + if (type == double.class || type == Double.class) { + return 1d; + } + if (type == char.class || type == Character.class) { + return '\0'; + } + if (type == String.class) { + return "test"; + } + return null; + } + + private boolean checkCompatible(final Object original, final Object deserialized, final String path, final Map visited) { + if (null == original && null == deserialized) { + return true; + } + if (null == original || null == deserialized) { + System.err.printf("Objects at %s incompatible: one is null\n", path); + return false; + } + + if (!isPrimitiveOrWrapper(original.getClass())) { + if (visited.containsKey(original)) { + return true; + } + visited.put(original, deserialized); + } + + Class clazz = original.getClass(); + boolean result = true; + for (Field field : clazz.getDeclaredFields()) { + if (Modifier.isStatic(field.getModifiers())) { + continue; + } + JSONField jsonField = field.getAnnotation(JSONField.class); + if (null != jsonField && !jsonField.serialize()) { + continue; + } + if ("hash".equals(field.getName()) || "serialVersionUID".equals(field.getName())) { + continue; + } + + field.setAccessible(true); + try { + Object v1 = field.get(original); + Object v2 = field.get(deserialized); + String fieldPath = path + "." + field.getName(); + + if (null == v1 && null == v2) { + continue; + } + if (v1 instanceof Random && v2 instanceof Random) { + continue; + } + if (v1 instanceof AtomicLong && v2 instanceof AtomicLong) { + if (((AtomicLong) v1).get() != ((AtomicLong) v2).get()) { + result = false; + System.err.printf("Field %s incompatible: original=%s, deserialized=%s\n", fieldPath, v1, v2); + } + continue; + } + if (v1 instanceof Set && v2 instanceof Set) { + Set s1 = (Set) v1, s2 = (Set) v2; + if (s1.size() != s2.size()) { + result = false; + System.err.printf("Field %s incompatible: set size original=%d, deserialized=%d\n", fieldPath, s1.size(), s2.size()); + } else if (!s1.isEmpty()) { + List list1 = new ArrayList<>(s1); + List list2 = new ArrayList<>(s2); + if (new HashSet<>(list1).equals(new HashSet<>(list2))) { + continue; + } + boolean elementsCompatible = true; + for (Object e1 : list1) { + boolean foundMatch = false; + for (Object e2 : list2) { + if (checkCompatible(e1, e2, fieldPath + ".element", new HashMap<>(visited))) { + foundMatch = true; + break; + } + } + if (!foundMatch) { + elementsCompatible = false; + break; + } + } + if (!elementsCompatible) { + result = false; + System.err.printf("Field %s incompatible: sets have different elements\n", fieldPath); + } + } + continue; + } + if (v1 instanceof List && v2 instanceof List) { + List l1 = (List) v1, l2 = (List) v2; + if (l1.size() != l2.size()) { + result = false; + System.err.printf("Field %s incompatible: list size original=%d, deserialized=%d\n", fieldPath, l1.size(), l2.size()); + } else { + for (int i = 0; i < l1.size(); i++) { + Object e1 = l1.get(i); + Object e2 = l2.get(i); + if (!checkCompatible(e1, e2, fieldPath + "[" + i + "]", new HashMap<>(visited))) { + result = false; + } + } + } + continue; + } + if (v1 instanceof Map && v2 instanceof Map) { + Map m1 = (Map) v1, m2 = (Map) v2; + if (!m1.keySet().equals(m2.keySet())) { + result = false; + System.err.printf("Field %s incompatible: map keys original=%s, deserialized=%s\n", fieldPath, m1.keySet(), m2.keySet()); + } else { + for (Object key : m1.keySet()) { + Object val1 = m1.get(key), val2 = m2.get(key); + if (val1 != null && val2 != null) { + if (!checkCompatible(val1, val2, fieldPath + "[" + key + "]", new HashMap<>(visited))) { + result = false; + } + } else if (val1 != val2) { + result = false; + System.err.printf("Field %s key %s incompatible: original=%s, deserialized=%s\n", + fieldPath, key, val1, val2); + } + } + } + continue; + } + Class type = field.getType(); + if (null != v1 && null != v2 && !type.isPrimitive() && !type.getName().startsWith("java.")) { + if (!checkCompatible(v1, v2, fieldPath, new HashMap<>(visited))) { + result = false; + } + continue; + } + if (null == v1 || null == v2 || !v1.equals(v2)) { + result = false; + System.err.printf("Field %s incompatible: original=%s, deserialized=%s\n", fieldPath, v1, v2); + } + } catch (Exception e) { + result = false; + System.err.printf("Field %s error: %s\n", path + "." + field.getName(), e.getMessage()); + } + } + if (result) { + System.out.printf("Class %s compatible\n", path); + } + return result; + } + + private boolean isPrimitiveOrWrapper(final Class clazz) { + return clazz.isPrimitive() || + clazz == String.class || + clazz == Boolean.class || + clazz == Character.class || + clazz == Byte.class || + clazz == Short.class || + clazz == Integer.class || + clazz == Long.class || + clazz == Float.class || + clazz == Double.class; + } + + private boolean checkCompatible(final Object original, final Class clazz) { + String json = com.alibaba.fastjson.JSON.toJSONString(original); + Object deserialized; + try { + deserialized = com.alibaba.fastjson2.JSON.parseObject(json, clazz); + } catch (Exception e) { + System.err.printf("Deserialization failed for %s: %s\n", clazz.getName(), e.getMessage()); + return false; + } + return checkCompatible(original, deserialized, clazz.getSimpleName(), new HashMap<>()); + } + + private T allocateInstance(final Class clazz) { + return new ObjenesisStd().newInstance(clazz); + } +} diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableTest.java index 6bd80217da0..e2655fd35da 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/RemotingSerializableTest.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.remoting.protocol; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSONWriter; import com.google.gson.Gson; import com.google.gson.JsonElement; import com.google.gson.TypeAdapter; @@ -105,8 +105,7 @@ public Map getMap() { } Foo foo = new Foo(); String invalid = new String(foo.encode(), Charset.defaultCharset()); - String valid = new String(foo.encode(SerializerFeature.BrowserCompatible, SerializerFeature.QuoteFieldNames, - SerializerFeature.MapSortField), Charset.defaultCharset()); + String valid = new String(foo.encode(JSONWriter.Feature.BrowserCompatible, JSONWriter.Feature.MapSortField), Charset.defaultCharset()); Gson gson = new Gson(); final TypeAdapter strictAdapter = gson.getAdapter(JsonElement.class); diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/body/BatchAckTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/body/BatchAckTest.java index 427a132d646..81e8f133946 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/body/BatchAckTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/body/BatchAckTest.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.remoting.protocol.body; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import org.apache.rocketmq.common.MixAll; import org.junit.Test; diff --git a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicQueueMappingTest.java b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicQueueMappingTest.java index 6b8a1392f5b..7bb7b8ca7e7 100644 --- a/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicQueueMappingTest.java +++ b/remoting/src/test/java/org/apache/rocketmq/remoting/protocol/statictopic/TopicQueueMappingTest.java @@ -17,15 +17,15 @@ package org.apache.rocketmq.remoting.protocol.statictopic; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONArray; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; import com.google.common.collect.ImmutableList; -import java.util.Map; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; import org.junit.Assert; import org.junit.Test; +import java.util.Map; + public class TopicQueueMappingTest { @Test @@ -59,7 +59,6 @@ public void testJsonSerialize() { Assert.assertTrue(mappingDetailMap.containsKey("currIdMap")); Assert.assertEquals(8, mappingDetailMap.size()); Assert.assertEquals(1, ((JSONObject) mappingDetailMap.get("hostedQueues")).size()); - Assert.assertEquals(1, ((JSONArray)((JSONObject) mappingDetailMap.get("hostedQueues")).get("0")).size()); } { TopicQueueMappingDetail mappingDetailFromJson = RemotingSerializable.decode(mappingDetailJson.getBytes(), TopicQueueMappingDetail.class); @@ -69,9 +68,4 @@ public void testJsonSerialize() { Assert.assertEquals(mappingDetailJson, RemotingSerializable.toJson(mappingDetailFromJson, false)); } } - - @Test - public void test() { - - } } diff --git a/srvutil/BUILD.bazel b/srvutil/BUILD.bazel index 699d78bcd91..89094098104 100644 --- a/srvutil/BUILD.bazel +++ b/srvutil/BUILD.bazel @@ -26,7 +26,6 @@ java_library( "@maven//:commons_validator_commons_validator", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", "@maven//:io_netty_netty_all", "@maven//:commons_cli_commons_cli", "@maven//:com_googlecode_concurrentlinkedhashmap_concurrentlinkedhashmap_lru", diff --git a/store/BUILD.bazel b/store/BUILD.bazel index de98657b220..8986c9b236b 100644 --- a/store/BUILD.bazel +++ b/store/BUILD.bazel @@ -23,7 +23,6 @@ java_library( deps = [ "//common", "//remoting", - "@maven//:com_alibaba_fastjson", "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_conversantmedia_disruptor", "@maven//:com_google_guava_guava", @@ -58,7 +57,7 @@ java_library( "//:test_deps", "//common", "//remoting", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:com_conversantmedia_disruptor", "@maven//:io_openmessaging_storage_dledger", "@maven//:org_apache_commons_commons_lang3", diff --git a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java index 29b09bc6497..459f2074b24 100644 --- a/store/src/main/java/org/apache/rocketmq/store/CommitLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/CommitLog.java @@ -36,6 +36,8 @@ import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import java.util.stream.Collectors; + +import io.netty.util.internal.PlatformDependent; import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.ServiceThread; import org.apache.rocketmq.common.SystemClock; @@ -66,10 +68,8 @@ import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.queue.CqUnit; -import org.apache.rocketmq.store.queue.ReferredIterator; import org.apache.rocketmq.store.util.LibC; import org.rocksdb.RocksDBException; -import sun.nio.ch.DirectBuffer; /** * Store all metadata downtime for recovery, data protection reliability @@ -340,12 +340,18 @@ public boolean getLastMappedFile(final long startOffset) { public void recoverNormally(long dispatchFromPhyOffset) throws RocksDBException { boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover(); boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable(); + int maxRecoverNum = this.defaultMessageStore.getMessageStoreConfig().getCommitLogRecoverMaxNum(); + if (maxRecoverNum <= 0) { + maxRecoverNum = 10; + } + log.info("recoverNormally maxRecoverNum: {}", maxRecoverNum); final List mappedFiles = this.mappedFileQueue.getMappedFiles(); if (!mappedFiles.isEmpty()) { int index = mappedFiles.size() - 1; while (index > 0) { MappedFile mappedFile = mappedFiles.get(index); - if (isMappedFileMatchedRecover(mappedFile, true)) { + maxRecoverNum--; + if (isMappedFileMatchedRecover(mappedFile, true) || maxRecoverNum <= 0) { // It's safe to recover from this mapped file break; } @@ -732,6 +738,11 @@ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExc // recover by the minimum time stamp boolean checkCRCOnRecover = this.defaultMessageStore.getMessageStoreConfig().isCheckCRCOnRecover(); boolean checkDupInfo = this.defaultMessageStore.getMessageStoreConfig().isDuplicationEnable(); + int maxRecoverNum = this.defaultMessageStore.getMessageStoreConfig().getCommitLogRecoverMaxNum(); + if (maxRecoverNum <= 0) { + maxRecoverNum = 10; + } + log.info("recoverAbnormally maxRecoverNum: {}", maxRecoverNum); final List mappedFiles = this.mappedFileQueue.getMappedFiles(); if (!mappedFiles.isEmpty()) { // Looking beginning to recover from which file @@ -739,8 +750,9 @@ public void recoverAbnormally(long maxPhyOffsetOfConsumeQueue) throws RocksDBExc MappedFile mappedFile = null; for (; index >= 0; index--) { mappedFile = mappedFiles.get(index); - if (this.isMappedFileMatchedRecover(mappedFile, false)) { - log.info("recover from this mapped file {}", mappedFile.getFileName()); + maxRecoverNum--; + if (this.isMappedFileMatchedRecover(mappedFile, false) || maxRecoverNum <= 0) { + log.info("recover from this mapped file " + mappedFile.getFileName()); break; } } @@ -910,8 +922,18 @@ private boolean isMappedFileMatchedRecover(final MappedFile mappedFile, storeTimestamp, phyOffset, this.defaultMessageStore.getStoreCheckpoint().getIndexMsgTimestamp(), recoverNormally); } - return this.defaultMessageStore.getQueueStore() - .isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); + return isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); + } + + private boolean isMappedFileMatchedRecover(long phyOffset, long storeTimestamp, boolean recoverNormally) throws RocksDBException { + boolean result = this.defaultMessageStore.getQueueStore().isMappedFileMatchedRecover(phyOffset, storeTimestamp, recoverNormally); + if (null != this.defaultMessageStore.getTransRocksDBStore() && defaultMessageStore.getMessageStoreConfig().isTransRocksDBEnable() && !defaultMessageStore.getMessageStoreConfig().isTransWriteOriginTransHalfEnable()) { + result = result && this.defaultMessageStore.getTransRocksDBStore().isMappedFileMatchedRecover(phyOffset); + } + if (null != this.defaultMessageStore.getIndexRocksDBStore() && defaultMessageStore.getMessageStoreConfig().isIndexRocksDBEnable()) { + result = result && this.defaultMessageStore.getIndexRocksDBStore().isMappedFileMatchedRecover(phyOffset); + } + return result; } public boolean resetOffset(long offset) { @@ -2296,6 +2318,10 @@ public MessageStore getMessageStore() { return defaultMessageStore; } + public MappedFile getEarliestMappedFile() { + return mappedFileQueue.getEarliestMappedFile(); + } + @Override public void swapMap(int reserveNum, long forceSwapIntervalMs, long normalSwapIntervalMs) { this.getMappedFileQueue().swapMap(reserveNum, forceSwapIntervalMs, normalSwapIntervalMs); @@ -2433,7 +2459,7 @@ private byte[] sampling(byte[] pageCacheTable, int sampleStep) { private byte[] checkFileInPageCache(MappedFile mappedFile) { long fileSize = mappedFile.getFileSize(); - final long address = ((DirectBuffer) mappedFile.getMappedByteBuffer()).address(); + final long address = PlatformDependent.directBufferAddress(mappedFile.getMappedByteBuffer()); int pageNums = (int) (fileSize + this.pageSize - 1) / this.pageSize; byte[] pageCacheRst = new byte[pageNums]; int mincore = LibC.INSTANCE.mincore(new Pointer(address), new NativeLong(fileSize), pageCacheRst); @@ -2476,14 +2502,17 @@ public boolean isMsgInColdArea(String group, String topic, int queueId, long off if (null == consumeQueue) { return false; } - ReferredIterator bufferConsumeQueue = consumeQueue.iterateFrom(offset, 1); - if (null == bufferConsumeQueue || !bufferConsumeQueue.hasNext()) { + CqUnit cqUnit = consumeQueue.get(offset); + if (null == cqUnit) { + return false; + } + long offsetPy = cqUnit.getPos(); + if (offsetPy < 0L) { return false; } - return defaultMessageStore.checkInColdAreaByCommitOffset(bufferConsumeQueue.next().getPos(), getMaxOffset()); + return defaultMessageStore.checkInColdAreaByCommitOffset(offsetPy, getMaxOffset()); } catch (Exception e) { - log.error("isMsgInColdArea group: {}, topic: {}, queueId: {}, offset: {}", - group, topic, queueId, offset, e); + log.error("isMsgInColdArea group: {}, topic: {}, queueId: {}, offset: {}", group, topic, queueId, offset, e); } return false; } @@ -2509,7 +2538,7 @@ private int setFileReadMode(MappedFile mappedFile, int mode) { log.error("setFileReadMode mappedFile is null"); return -1; } - final long address = ((DirectBuffer) mappedFile.getMappedByteBuffer()).address(); + final long address = PlatformDependent.directBufferAddress(mappedFile.getMappedByteBuffer()); int madvise = LibC.INSTANCE.madvise(new Pointer(address), new NativeLong(mappedFile.getFileSize()), mode); if (madvise != 0) { log.error("setFileReadMode error fileName: {}, madvise: {}, mode:{}", mappedFile.getFileName(), madvise, mode); 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 cb5c41471a9..6a55951d1cc 100644 --- a/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/DefaultMessageStore.java @@ -56,8 +56,9 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; +import com.alibaba.fastjson.JSON; +import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; -import org.apache.rocketmq.common.AbstractBrokerRunnable; import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.BrokerConfig; import org.apache.rocketmq.common.BrokerIdentity; @@ -99,6 +100,7 @@ import org.apache.rocketmq.store.hook.SendMessageBackHook; import org.apache.rocketmq.store.index.IndexService; import org.apache.rocketmq.store.index.QueryOffsetResult; +import org.apache.rocketmq.store.index.rocksdb.IndexRocksDBStore; import org.apache.rocketmq.store.kv.CommitLogDispatcherCompaction; import org.apache.rocketmq.store.kv.CompactionService; import org.apache.rocketmq.store.kv.CompactionStore; @@ -111,8 +113,11 @@ import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; import org.apache.rocketmq.store.queue.CqUnit; import org.apache.rocketmq.store.queue.ReferredIterator; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.apache.rocketmq.store.timer.TimerMessageStore; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; import org.apache.rocketmq.store.util.PerfCounter; import org.apache.rocketmq.store.metrics.StoreMetricsManager; import org.rocksdb.RocksDBException; @@ -132,6 +137,7 @@ public class DefaultMessageStore implements MessageStore { protected final CleanCommitLogService cleanCommitLogService; protected final IndexService indexService; + protected final IndexRocksDBStore indexRocksDBStore; private final AllocateMappedFileService allocateMappedFileService; @@ -161,8 +167,11 @@ public class DefaultMessageStore implements MessageStore { private boolean notifyMessageArriveInBatch = false; protected StoreCheckpoint storeCheckpoint; + private MessageRocksDBStorage messageRocksDBStorage; private TimerMessageStore timerMessageStore; private final DefaultStoreMetricsManager defaultStoreMetricsManager; + private TimerMessageRocksDBStore timerMessageRocksDBStore; + private TransMessageRocksDBStore transMessageRocksDBStore; private final LinkedList dispatcherList = new LinkedList<>(); @@ -230,9 +239,12 @@ public DefaultMessageStore(final MessageStoreConfig messageStoreConfig, final Br this.consumeQueueStore = createConsumeQueueStore(); this.cleanCommitLogService = new CleanCommitLogService(); this.storeStatsService = new StoreStatsService(getBrokerIdentity()); + this.messageRocksDBStorage = new MessageRocksDBStorage(getMessageStoreConfig()); this.indexService = new IndexService(this); + this.indexRocksDBStore = new IndexRocksDBStore(this); this.dispatcherList.addLast(new CommitLogDispatcherBuildConsumeQueue()); this.dispatcherList.addLast(new CommitLogDispatcherBuildIndex()); + this.dispatcherList.addLast(new CommitLogDispatcherBuildTransIndex()); initializeHAService(); @@ -390,7 +402,6 @@ public void start() throws Exception { if (this.isTransientStorePoolEnable()) { this.transientStorePool.init(); } - this.allocateMappedFileService.start(); this.indexService.start(); @@ -524,6 +535,10 @@ public void shutdown() { this.indexService.shutdown(); } + if (this.indexRocksDBStore != null) { + this.indexRocksDBStore.shutdown(); + } + if (this.compactionService != null) { this.compactionService.shutdown(); } @@ -551,6 +566,10 @@ public void shutdown() { this.transientStorePool.destroy(); } + if (this.messageRocksDBStorage != null) { + this.messageRocksDBStorage.shutdown(); + } + if (lock != null) { try { lock.release(); @@ -573,6 +592,7 @@ public void destroy() { this.consumeQueueStore.destroy(false); this.commitLog.destroy(); this.indexService.destroy(); + this.indexRocksDBStore.destroy(); this.deleteFile(StorePathConfigHelper.getAbortFile(this.messageStoreConfig.getStorePathRootDir())); this.deleteFile(StorePathConfigHelper.getStoreCheckpoint(this.messageStoreConfig.getStorePathRootDir())); } @@ -1059,11 +1079,31 @@ public TimerMessageStore getTimerMessageStore() { return this.timerMessageStore; } + @Override + public TimerMessageRocksDBStore getTimerRocksDBStore() { + return this.timerMessageRocksDBStore; + } + + @Override + public TransMessageRocksDBStore getTransRocksDBStore() { + return this.transMessageRocksDBStore; + } + @Override public void setTimerMessageStore(TimerMessageStore timerMessageStore) { this.timerMessageStore = timerMessageStore; } + @Override + public void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore) { + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + } + + @Override + public void setTransRocksDBStore(TransMessageRocksDBStore transMessageRocksDBStore) { + this.transMessageRocksDBStore = transMessageRocksDBStore; + } + @Override public long getCommitLogOffsetInQueue(String topic, int queueId, long consumeQueueOffset) { ConsumeQueueInterface consumeQueue = getConsumeQueue(topic, queueId); @@ -1310,8 +1350,15 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon long lastQueryMsgTime = end; for (int i = 0; i < 3; i++) { - QueryOffsetResult queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime); - if (queryOffsetResult.getPhyOffsets().isEmpty()) { + QueryOffsetResult queryOffsetResult = null; + if (messageStoreConfig.isIndexFileReadEnable()) { + queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, null); + LOGGER.debug("indexService query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } else if (messageStoreConfig.isIndexRocksDBEnable()) { + queryOffsetResult = this.indexRocksDBStore.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, null, null); + LOGGER.debug("indexRocksDBStore query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } + if (null == queryOffsetResult || CollectionUtils.isEmpty(queryOffsetResult.getPhyOffsets())) { break; } @@ -1349,7 +1396,55 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon break; } } + return queryMessageResult; + } + + @Override + public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, long end, String indexType, String lastKey) { + QueryMessageResult queryMessageResult = new QueryMessageResult(); + long lastQueryMsgTime = end; + for (int i = 0; i < 3; i++) { + QueryOffsetResult queryOffsetResult = null; + if (messageStoreConfig.isIndexFileReadEnable()) { + queryOffsetResult = this.indexService.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, indexType); + LOGGER.debug("indexService query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } else if (messageStoreConfig.isIndexRocksDBEnable()) { + queryOffsetResult = this.indexRocksDBStore.queryOffset(topic, key, maxNum, begin, lastQueryMsgTime, indexType, lastKey); + LOGGER.debug("indexRocksDBStore query Message queryOffsetResult : {}", JSON.toJSONString(queryOffsetResult)); + } + if (null == queryOffsetResult || CollectionUtils.isEmpty(queryOffsetResult.getPhyOffsets())) { + break; + } + Collections.sort(queryOffsetResult.getPhyOffsets()); + queryMessageResult.setIndexLastUpdatePhyoffset(queryOffsetResult.getIndexLastUpdatePhyoffset()); + queryMessageResult.setIndexLastUpdateTimestamp(queryOffsetResult.getIndexLastUpdateTimestamp()); + for (int m = 0; m < queryOffsetResult.getPhyOffsets().size(); m++) { + long offset = queryOffsetResult.getPhyOffsets().get(m); + try { + MessageExt msg = this.lookMessageByOffset(offset); + if (0 == m && null != msg) { + lastQueryMsgTime = msg.getStoreTimestamp(); + } + SelectMappedBufferResult result = this.commitLog.getData(offset, false); + if (result != null) { + int size = result.getByteBuffer().getInt(0); + result.getByteBuffer().limit(size); + result.setSize(size); + queryMessageResult.addMessage(result); + } + } catch (Exception e) { + LOGGER.error("queryMessage exception", e); + } + } + if (queryMessageResult.getBufferTotalSize() > 0) { + break; + } + + if (lastQueryMsgTime < begin) { + break; + } + } return queryMessageResult; } @@ -1358,6 +1453,12 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon return CompletableFuture.completedFuture(queryMessage(topic, key, maxNum, begin, end)); } + @Override + public CompletableFuture queryMessageAsync(String topic, String key, + int maxNum, long begin, long end, String indexType, String lastKey) { + return CompletableFuture.completedFuture(queryMessage(topic, key, maxNum, begin, end, indexType, lastKey)); + } + @Override public void updateHaMasterAddress(String newAddr) { if (this.haService != null) { @@ -1427,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 @@ -1775,23 +1878,23 @@ private void createTempFile() throws IOException { private void addScheduleTask() { - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { DefaultMessageStore.this.cleanFilesPeriodically(); } }, 1000 * 60, this.messageStoreConfig.getCleanResourceInterval(), TimeUnit.MILLISECONDS); - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { DefaultMessageStore.this.checkSelf(); } }, 1, 10, TimeUnit.MINUTES); - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { if (DefaultMessageStore.this.getMessageStoreConfig().isDebugLockEnable()) { try { if (DefaultMessageStore.this.commitLog.getBeginTimeInLock() != 0) { @@ -1810,9 +1913,9 @@ public void run0() { } }, 1, 1, TimeUnit.SECONDS); - this.scheduledExecutorService.scheduleAtFixedRate(new AbstractBrokerRunnable(this.getBrokerIdentity()) { + this.scheduledExecutorService.scheduleAtFixedRate(new Runnable() { @Override - public void run0() { + public void run() { DefaultMessageStore.this.storeCheckpoint.flush(); } }, 1, 1, TimeUnit.SECONDS); @@ -2102,7 +2205,34 @@ class CommitLogDispatcherBuildIndex implements CommitLogDispatcher { @Override public void dispatch(DispatchRequest request) { if (DefaultMessageStore.this.messageStoreConfig.isMessageIndexEnable()) { - DefaultMessageStore.this.indexService.buildIndex(request); + if (DefaultMessageStore.this.messageStoreConfig.isIndexFileWriteEnable()) { + DefaultMessageStore.this.indexService.buildIndex(request); + } + if (DefaultMessageStore.this.messageStoreConfig.isIndexRocksDBEnable()) { + DefaultMessageStore.this.indexRocksDBStore.buildIndex(request); + } + } + } + } + + class CommitLogDispatcherBuildTransIndex implements CommitLogDispatcher { + + @Override + public void dispatch(DispatchRequest request) { + if (DefaultMessageStore.this.messageStoreConfig.isTransRocksDBEnable()) { + if (null == request || StringUtils.isEmpty(request.getTopic())) { + return; + } + if (!request.getTopic().equals(TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC) && !request.getTopic().equals(TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC)) { + return; + } + if (null == DefaultMessageStore.this.transMessageRocksDBStore) { + if (System.currentTimeMillis() % 1000 == 0) { + LOGGER.error("CommitLogDispatcherBuildTransIndex dispatch error, transMessageRocksDBStore is null"); + } + return; + } + DefaultMessageStore.this.transMessageRocksDBStore.buildTransIndex(request); } } } @@ -3060,6 +3190,11 @@ public MessageStoreStateMachine getStateMachine() { return stateMachine; } + @Override + public MessageRocksDBStorage getMessageRocksDBStorage() { + return this.messageRocksDBStorage; + } + public boolean isNotifyMessageArriveInBatch() { return notifyMessageArriveInBatch; } @@ -3077,4 +3212,7 @@ public StoreMetricsManager getStoreMetricsManager() { return defaultStoreMetricsManager; } + public IndexRocksDBStore getIndexRocksDBStore() { + return indexRocksDBStore; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java index 2db6ff573af..94235024da9 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java +++ b/store/src/main/java/org/apache/rocketmq/store/MappedFileQueue.java @@ -944,4 +944,20 @@ public List range(final long from, final long to) { return result; } + + public MappedFile getEarliestMappedFile() { + MappedFile mappedFile = null; + while (!this.mappedFiles.isEmpty()) { + try { + mappedFile = this.mappedFiles.get(0); + break; + } catch (IndexOutOfBoundsException e) { + //continue; + } catch (Exception e) { + log.error("getEarliestMappedFile error: {}", e.getMessage()); + break; + } + } + return mappedFile; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java index 0b927513e13..b297ee542f3 100644 --- a/store/src/main/java/org/apache/rocketmq/store/MessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/MessageStore.java @@ -43,8 +43,11 @@ import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.queue.ConsumeQueueInterface; import org.apache.rocketmq.store.queue.ConsumeQueueStoreInterface; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; import org.apache.rocketmq.store.stats.BrokerStatsManager; import org.apache.rocketmq.store.timer.TimerMessageStore; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; import org.apache.rocketmq.store.util.PerfCounter; import org.apache.rocketmq.store.metrics.StoreMetricsManager; import org.rocksdb.RocksDBException; @@ -207,8 +210,16 @@ CompletableFuture getMessageAsync(final String group, final St TimerMessageStore getTimerMessageStore(); + TimerMessageRocksDBStore getTimerRocksDBStore(); + + TransMessageRocksDBStore getTransRocksDBStore(); + void setTimerMessageStore(TimerMessageStore timerMessageStore); + void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore); + + void setTransRocksDBStore(TransMessageRocksDBStore transMessageRocksDBStore); + /** * Get the offset of the message in the commit log, which is also known as physical offset. * @@ -410,6 +421,8 @@ CompletableFuture getMessageStoreTimeStampAsync(final String topic, final QueryMessageResult queryMessage(final String topic, final String key, final int maxNum, final long begin, final long end); + QueryMessageResult queryMessage(final String topic, final String key, final int maxNum, final long begin, final long end, final String indexType, final String lastKey); + /** * Asynchronous query messages by given key. * @see #queryMessage(String, String, int, long, long) queryMessage @@ -423,6 +436,8 @@ QueryMessageResult queryMessage(final String topic, final String key, final int CompletableFuture queryMessageAsync(final String topic, final String key, final int maxNum, final long begin, final long end); + CompletableFuture queryMessageAsync(final String topic, final String key, final int maxNum, final long begin, final long end, final String indexType, final String lastKey); + /** * Update HA master address. * @@ -1000,4 +1015,6 @@ DispatchRequest checkMessageAndReturnSize(final ByteBuffer byteBuffer, final boo void notifyMessageArriveIfNecessary(DispatchRequest dispatchRequest); MessageStoreStateMachine getStateMachine(); + + MessageRocksDBStorage getMessageRocksDBStorage(); } diff --git a/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java b/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java index 88b398a77e6..f415487bd57 100644 --- a/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java +++ b/store/src/main/java/org/apache/rocketmq/store/RunningFlags.java @@ -85,6 +85,15 @@ public boolean isWriteable() { return false; } + public boolean isStoreWriteable() { + if ((this.flagBits & NOT_WRITEABLE_BIT) == 0) { + return true; + } + + return false; + } + + //for consume queue, just ignore the DISK_FULL_BIT public boolean isCQWriteable() { if ((this.flagBits & (NOT_WRITEABLE_BIT | WRITE_LOGICS_QUEUE_ERROR_BIT | WRITE_INDEX_FILE_ERROR_BIT | LOGIC_DISK_FULL_BIT)) == 0) { @@ -94,7 +103,7 @@ public boolean isCQWriteable() { return false; } - public boolean getAndMakeNotWriteable() { + public boolean getAndMakeStoreNotWriteable() { boolean result = this.isWriteable(); if (result) { this.flagBits |= NOT_WRITEABLE_BIT; diff --git a/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java b/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java index 526ca9bf1b0..9ef6f72ab44 100644 --- a/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java +++ b/store/src/main/java/org/apache/rocketmq/store/StoreUtil.java @@ -18,6 +18,8 @@ import com.google.common.base.Preconditions; import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.store.logfile.MappedFile; @@ -76,4 +78,23 @@ public static FileQueueSnapshot getFileQueueSnapshot(MappedFileQueue mappedFileQ } return new FileQueueSnapshot(); } + + public static MessageExt getMessage(long offsetPy, int sizePy, MessageStore messageStore, ByteBuffer byteBuffer) { + try { + if (offsetPy < 0L || sizePy <= 0 || null == messageStore || null == byteBuffer) { + return null; + } + byteBuffer.position(0); + byteBuffer.limit(sizePy); + if (!messageStore.getData(offsetPy, sizePy, byteBuffer)) { + return null; + } + byteBuffer.flip(); + return MessageDecoder.decode(byteBuffer, true, false, false); + } catch (Exception e) { + log.error("getMessage error, offsetPy: {}, sizePy: {}, error: {}", offsetPy, sizePy, e.getMessage()); + } + return null; + } + } diff --git a/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java b/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java index 0d42ee69e68..d9ad4f4ed1e 100644 --- a/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java +++ b/store/src/main/java/org/apache/rocketmq/store/TransientStorePool.java @@ -16,16 +16,17 @@ */ package org.apache.rocketmq.store; -import com.sun.jna.NativeLong; -import com.sun.jna.Pointer; import java.nio.ByteBuffer; import java.util.Deque; import java.util.concurrent.ConcurrentLinkedDeque; + +import com.sun.jna.NativeLong; +import com.sun.jna.Pointer; +import io.netty.util.internal.PlatformDependent; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; import org.apache.rocketmq.store.util.LibC; -import sun.nio.ch.DirectBuffer; public class TransientStorePool { private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); @@ -48,7 +49,7 @@ public void init() { for (int i = 0; i < poolSize; i++) { ByteBuffer byteBuffer = ByteBuffer.allocateDirect(fileSize); - final long address = ((DirectBuffer) byteBuffer).address(); + final long address = PlatformDependent.directBufferAddress(byteBuffer); Pointer pointer = new Pointer(address); LibC.INSTANCE.mlock(pointer, new NativeLong(fileSize)); @@ -58,7 +59,7 @@ public void init() { public void destroy() { for (ByteBuffer byteBuffer : availableBuffers) { - final long address = ((DirectBuffer) byteBuffer).address(); + final long address = PlatformDependent.directBufferAddress(byteBuffer); Pointer pointer = new Pointer(address); LibC.INSTANCE.munlock(pointer, new NativeLong(fileSize)); } 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 85d19f31b4a..5c4294434bf 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 @@ -80,6 +80,7 @@ public class MessageStoreConfig { private boolean timerEnableCheckMetrics = true; private boolean timerInterceptDelayLevel = false; private int timerMaxDelaySec = 3600 * 24 * 3; + private boolean timerWheelSnapshotFlush = false; private boolean timerWheelEnable = true; /** @@ -103,6 +104,26 @@ public class MessageStoreConfig { private int timerMetricSmallThreshold = 1000000; private int timerProgressLogIntervalMs = 10 * 1000; + private int timerWheelSnapshotIntervalMs = 10 * 1000; + + private int commitLogRecoverMaxNum = 10; + private boolean timerRocksDBEnable = false; + private boolean timerRocksDBStopScan = false; + private long timerRocksDBPrecisionMs = 1000L; + private double timerRocksDBRollMaxTps = 8000.0; + private double timerRocksDBTimeExpiredMaxTps = 200000.0; + private int timerRocksDBRollIntervalHours = 1; + private int timerRocksDBRollRangeHours = 2; + private boolean timerRecallToTimeWheelEnable = true; + private boolean timerRecallToTimelineEnable = true; + + private boolean transRocksDBEnable = false; + private boolean transWriteOriginTransHalfEnable = true; + + private boolean indexRocksDBEnable = false; + private int maxRocksDBIndexQueryDays = 7; + private boolean indexFileWriteEnable = true; + private boolean indexFileReadEnable = true; // default, defaultRocksDB @ImportantField @@ -261,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; @@ -1631,6 +1653,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; } @@ -1750,6 +1780,14 @@ public boolean isTimerWarmEnable() { return timerWarmEnable; } + public boolean isTimerWheelSnapshotFlush() { + return timerWheelSnapshotFlush; + } + + public void setTimerWheelSnapshotFlush(boolean timerWheelSnapshotFlush) { + this.timerWheelSnapshotFlush = timerWheelSnapshotFlush; + } + public boolean isTimerWheelEnable() { return timerWheelEnable; } @@ -1795,6 +1833,14 @@ public int getTimerProgressLogIntervalMs() { return timerProgressLogIntervalMs; } + public int getTimerWheelSnapshotIntervalMs() { + return timerWheelSnapshotIntervalMs; + } + + public void setTimerWheelSnapshotIntervalMs(int timerWheelSnapshotIntervalMs) { + this.timerWheelSnapshotIntervalMs = timerWheelSnapshotIntervalMs; + } + public void setTimerProgressLogIntervalMs(final int timerProgressLogIntervalMs) { this.timerProgressLogIntervalMs = timerProgressLogIntervalMs; } @@ -2064,6 +2110,134 @@ public void setEnableRunningFlagsInFlush(boolean enableRunningFlagsInFlush) { this.enableRunningFlagsInFlush = enableRunningFlagsInFlush; } + public boolean isTimerRocksDBEnable() { + return timerRocksDBEnable; + } + + public void setTimerRocksDBEnable(boolean timerRocksDBEnable) { + this.timerRocksDBEnable = timerRocksDBEnable; + } + + public double getTimerRocksDBRollMaxTps() { + return timerRocksDBRollMaxTps; + } + + public void setTimerRocksDBRollMaxTps(double timerRocksDBRollMaxTps) { + this.timerRocksDBRollMaxTps = timerRocksDBRollMaxTps; + } + + public double getTimerRocksDBTimeExpiredMaxTps() { + return timerRocksDBTimeExpiredMaxTps; + } + + public void setTimerRocksDBTimeExpiredMaxTps(double timerRocksDBTimeExpiredMaxTps) { + this.timerRocksDBTimeExpiredMaxTps = timerRocksDBTimeExpiredMaxTps; + } + + public boolean isTransRocksDBEnable() { + return transRocksDBEnable; + } + + public void setTransRocksDBEnable(boolean transRocksDBEnable) { + this.transRocksDBEnable = transRocksDBEnable; + } + + public boolean isIndexRocksDBEnable() { + return indexRocksDBEnable; + } + + public void setIndexRocksDBEnable(boolean indexRocksDBEnable) { + this.indexRocksDBEnable = indexRocksDBEnable; + } + + public int getMaxRocksDBIndexQueryDays() { + return maxRocksDBIndexQueryDays; + } + + public void setMaxRocksDBIndexQueryDays(int maxRocksDBIndexQueryDays) { + this.maxRocksDBIndexQueryDays = maxRocksDBIndexQueryDays; + } + + public boolean isTimerRocksDBStopScan() { + return timerRocksDBStopScan; + } + + public void setTimerRocksDBStopScan(boolean timerRocksDBStopScan) { + this.timerRocksDBStopScan = timerRocksDBStopScan; + } + + public long getTimerRocksDBPrecisionMs() { + return timerRocksDBPrecisionMs; + } + + public void setTimerRocksDBPrecisionMs(long timerRocksDBPrecisionMs) { + this.timerRocksDBPrecisionMs = timerRocksDBPrecisionMs; + } + + public boolean isIndexFileWriteEnable() { + return indexFileWriteEnable; + } + + public void setIndexFileWriteEnable(boolean indexFileWriteEnable) { + this.indexFileWriteEnable = indexFileWriteEnable; + } + + public boolean isIndexFileReadEnable() { + return indexFileReadEnable; + } + + public void setIndexFileReadEnable(boolean indexFileReadEnable) { + this.indexFileReadEnable = indexFileReadEnable; + } + + public boolean isTransWriteOriginTransHalfEnable() { + return transWriteOriginTransHalfEnable; + } + + public void setTransWriteOriginTransHalfEnable(boolean transWriteOriginTransHalfEnable) { + this.transWriteOriginTransHalfEnable = transWriteOriginTransHalfEnable; + } + + public boolean isTimerRecallToTimeWheelEnable() { + return timerRecallToTimeWheelEnable; + } + + public void setTimerRecallToTimeWheelEnable(boolean timerRecallToTimeWheelEnable) { + this.timerRecallToTimeWheelEnable = timerRecallToTimeWheelEnable; + } + + public boolean isTimerRecallToTimelineEnable() { + return timerRecallToTimelineEnable; + } + + public void setTimerRecallToTimelineEnable(boolean timerRecallToTimelineEnable) { + this.timerRecallToTimelineEnable = timerRecallToTimelineEnable; + } + + public int getTimerRocksDBRollIntervalHours() { + return timerRocksDBRollIntervalHours; + } + + public void setTimerRocksDBRollIntervalHours(int timerRocksDBRollIntervalHours) { + this.timerRocksDBRollIntervalHours = timerRocksDBRollIntervalHours; + } + + public int getTimerRocksDBRollRangeHours() { + return timerRocksDBRollRangeHours; + } + + public void setTimerRocksDBRollRangeHours(int timerRocksDBRollRangeHours) { + this.timerRocksDBRollRangeHours = timerRocksDBRollRangeHours; + } + + public int getCommitLogRecoverMaxNum() { + return commitLogRecoverMaxNum; + } + + public void setCommitLogRecoverMaxNum(int commitLogRecoverMaxNum) { + this.commitLogRecoverMaxNum = commitLogRecoverMaxNum; + } + public int getSharedByteBufferNum() { return sharedByteBufferNum; } diff --git a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java index 64dad9aef21..9af47693f17 100644 --- a/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java +++ b/store/src/main/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHAService.java @@ -100,12 +100,19 @@ public void shutdown() { @Override public void removeConnection(HAConnection conn) { if (!defaultMessageStore.isShutdown()) { - final Set syncStateSet = getLocalSyncStateSet(); Long slave = ((AutoSwitchHAConnection) conn).getSlaveId(); - if (syncStateSet.contains(slave)) { - syncStateSet.remove(slave); - markSynchronizingSyncStateSet(syncStateSet); - notifySyncStateSetChanged(syncStateSet); + this.writeLock.lock(); + try { + final Set newSyncStateSet = new HashSet<>(this.syncStateSet); + if (newSyncStateSet.contains(slave)) { + newSyncStateSet.remove(slave); + markSynchronizingSyncStateSet(newSyncStateSet); + notifySyncStateSetChanged(newSyncStateSet); + this.syncStateSet.clear(); + this.syncStateSet.addAll(newSyncStateSet); + } + } finally { + this.writeLock.unlock(); } } super.removeConnection(conn); diff --git a/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java b/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java index 2d325ee13a4..8c16cca2941 100644 --- a/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java +++ b/store/src/main/java/org/apache/rocketmq/store/index/IndexService.java @@ -21,9 +21,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.rocketmq.common.AbstractBrokerRunnable; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; @@ -202,9 +203,56 @@ public QueryOffsetResult queryOffset(String topic, String key, int maxNum, long return new QueryOffsetResult(phyOffsets, indexLastUpdateTimestamp, indexLastUpdatePhyoffset); } + public QueryOffsetResult queryOffset(String topic, String key, int maxNum, long begin, long end, String indexType) { + List phyOffsets = new ArrayList<>(maxNum); + long indexLastUpdateTimestamp = 0; + long indexLastUpdatePhyoffset = 0; + maxNum = Math.min(maxNum, this.defaultMessageStore.getMessageStoreConfig().getMaxMsgsNumBatch()); + try { + this.readWriteLock.readLock().lock(); + if (!this.indexFileList.isEmpty()) { + for (int i = this.indexFileList.size(); i > 0; i--) { + IndexFile f = this.indexFileList.get(i - 1); + boolean lastFile = i == this.indexFileList.size(); + if (lastFile) { + indexLastUpdateTimestamp = f.getEndTimestamp(); + indexLastUpdatePhyoffset = f.getEndPhyOffset(); + } + + if (f.isTimeMatched(begin, end)) { + String queryKey; + if (!StringUtils.isEmpty(indexType) && MessageConst.INDEX_TAG_TYPE.equals(indexType)) { + queryKey = buildKey(topic, key, MessageConst.INDEX_TAG_TYPE); + } else { + queryKey = buildKey(topic, key); + } + f.selectPhyOffset(phyOffsets, queryKey, maxNum, begin, end); + } + + if (f.getBeginTimestamp() < begin) { + break; + } + + if (phyOffsets.size() >= maxNum) { + break; + } + } + } + } catch (Exception e) { + LOGGER.error("queryMsg queryOffset exception", e); + } finally { + this.readWriteLock.readLock().unlock(); + } + + return new QueryOffsetResult(phyOffsets, indexLastUpdateTimestamp, indexLastUpdatePhyoffset); + } + private String buildKey(final String topic, final String key) { return topic + "#" + key; } + private String buildKey(final String topic, final String key, final String indexType) { + return topic + "#" + indexType + "#" + key; + } public void buildIndex(DispatchRequest req) { IndexFile indexFile = retryGetAndCreateIndexFile(); @@ -248,6 +296,19 @@ public void buildIndex(DispatchRequest req) { } } } + + Map propertiesMap = req.getPropertiesMap(); + if (null != propertiesMap && propertiesMap.containsKey(MessageConst.PROPERTY_TAGS)) { + String tags = req.getPropertiesMap().get(MessageConst.PROPERTY_TAGS); + if (!StringUtils.isEmpty(tags)) { + indexFile = putKey(indexFile, msg, buildKey(topic, tags, MessageConst.INDEX_TAG_TYPE)); + if (indexFile == null) { + LOGGER.error("putKey error commitlog {} uniqkey {}", req.getCommitLogOffset(), req.getUniqKey()); + return; + } + } + } + } else { LOGGER.error("build index error, stop building index"); } @@ -339,9 +400,9 @@ public IndexFile getAndCreateLastIndexFile() { if (indexFile != null) { final IndexFile flushThisFile = prevIndexFile; - Thread flushThread = new Thread(new AbstractBrokerRunnable(defaultMessageStore.getBrokerConfig()) { + Thread flushThread = new Thread(new Runnable() { @Override - public void run0() { + public void run() { IndexService.this.flush(flushThisFile); } }, "FlushIndexFileThread"); diff --git a/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBRecord.java b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBRecord.java new file mode 100644 index 00000000000..68f5c3f5fee --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBRecord.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.index.rocksdb; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageConst; + +public class IndexRocksDBRecord { + public static final String KEY_SPLIT = "@"; + public static final byte[] KEY_SPLIT_BYTES = KEY_SPLIT.getBytes(StandardCharsets.UTF_8); + private static final int VALUE_LENGTH = Long.BYTES; + private long storeTime; + private String topic; + private String key; + private String tag; + private String uniqKey; + private long offsetPy; + + public IndexRocksDBRecord(String topic, String key, String tag, long storeTime, String uniqKey, long offsetPy) { + this.topic = topic; + this.key = key; + this.tag = tag; + this.storeTime = storeTime; + this.uniqKey = uniqKey; + this.offsetPy = offsetPy; + } + + public byte[] getKeyBytes() { + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey) || offsetPy < 0L || storeTime <= 0L) { + return null; + } + long storeTimeHour = MixAll.dealTimeToHourStamps(storeTime); + if (storeTimeHour <= 0L) { + return null; + } + String keyMiddleStr; + if (!StringUtils.isEmpty(key)) { + keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + MessageConst.INDEX_KEY_TYPE + KEY_SPLIT + key + KEY_SPLIT + uniqKey + KEY_SPLIT; + } else if (!StringUtils.isEmpty(tag)) { + keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + MessageConst.INDEX_TAG_TYPE + KEY_SPLIT + tag + KEY_SPLIT + uniqKey + KEY_SPLIT; + } else { + keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + MessageConst.INDEX_UNIQUE_TYPE + KEY_SPLIT + uniqKey + KEY_SPLIT; + } + if (StringUtils.isEmpty(keyMiddleStr)) { + return null; + } + byte[] keyMiddleBytes = keyMiddleStr.getBytes(StandardCharsets.UTF_8); + int keyLength = Long.BYTES + keyMiddleBytes.length + Long.BYTES; + return ByteBuffer.allocate(keyLength).putLong(storeTimeHour).put(keyMiddleBytes).putLong(offsetPy).array(); + } + + public byte[] getValueBytes() { + if (storeTime <= 0L) { + return null; + } + return ByteBuffer.allocate(VALUE_LENGTH).putLong(storeTime).array(); + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + public long getStoreTime() { + return storeTime; + } + + public void setStoreTime(long storeTime) { + this.storeTime = storeTime; + } + + public String getUniqKey() { + return uniqKey; + } + + public void setUniqKey(String uniqKey) { + this.uniqKey = uniqKey; + } + + public long getOffsetPy() { + return offsetPy; + } + + public void setOffsetPy(long offsetPy) { + this.offsetPy = offsetPy; + } + + public String getTag() { + return tag; + } + + public void setTag(String tag) { + this.tag = tag; + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBStore.java b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBStore.java new file mode 100644 index 00000000000..8ebf660bd1c --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/index/rocksdb/IndexRocksDBStore.java @@ -0,0 +1,325 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.index.rocksdb; +import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.sysflag.MessageSysFlag; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.index.QueryOffsetResult; +import org.apache.rocketmq.store.logfile.MappedFile; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.rocksdb.RocksDB; +import static org.apache.rocketmq.common.MixAll.dealTimeToHourStamps; + +public class IndexRocksDBStore { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final int DEFAULT_CAPACITY = 100000; + private static final int BATCH_SIZE = 1000; + private static final Set INDEX_TYPE_SET = new HashSet<>(); + static { + INDEX_TYPE_SET.add(MessageConst.INDEX_KEY_TYPE); + INDEX_TYPE_SET.add(MessageConst.INDEX_TAG_TYPE); + INDEX_TYPE_SET.add(MessageConst.INDEX_UNIQUE_TYPE); + } + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + + private final MessageStore messageStore; + private final MessageStoreConfig storeConfig; + private final MessageRocksDBStorage messageRocksDBStorage; + private volatile long lastDeleteIndexTime = 0L; + private IndexBuildService indexBuildService; + private BlockingQueue originIndexMsgQueue; + + public IndexRocksDBStore(MessageStore messageStore) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.messageRocksDBStorage = messageStore.getMessageRocksDBStorage(); + if (this.storeConfig.isIndexRocksDBEnable()) { + this.initAndStart(); + } + } + + private void initAndStart() { + if (this.state == RUNNING) { + return; + } + this.indexBuildService = new IndexBuildService(); + this.originIndexMsgQueue = new LinkedBlockingDeque<>(DEFAULT_CAPACITY); + this.indexBuildService.start(); + this.state = RUNNING; + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + log.info("IndexRocksDBStore start success, lastOffsetPy: {}", lastOffsetPy); + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.indexBuildService) { + this.indexBuildService.shutdown(); + } + this.state = SHUTDOWN; + log.info("IndexRocksDBStore shutdown success"); + } + + public QueryOffsetResult queryOffset(String topic, String key, int maxNum, long beginTime, long endTime, String indexType, String lastKey) { + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(key) || maxNum <= 0 || beginTime < 0L || endTime <= 0L || beginTime > endTime || !StringUtils.isEmpty(indexType) && !INDEX_TYPE_SET.contains(indexType)) { + logError.error("IndexRocksDBStore queryOffset param error, topic: {}, key: {}, maxNum: {}, beginTime: {}, endTime: {}, indexType: {}, lastKey: {}", topic, key, maxNum, beginTime, endTime, indexType, lastKey); + return null; + } + if (beginTime == 0L || Long.MAX_VALUE == endTime) { + endTime = System.currentTimeMillis(); + beginTime = endTime - TimeUnit.DAYS.toMillis(storeConfig.getMaxRocksDBIndexQueryDays()); + } + if ((endTime - beginTime) > (TimeUnit.DAYS.toMillis(storeConfig.getMaxRocksDBIndexQueryDays()))) { + logError.error("IndexRocksDBStore queryOffset index in rocksdb, can not query more than: {} days", storeConfig.getMaxRocksDBIndexQueryDays()); + return null; + } + long lastUpdateTime = 0L; + long lastOffsetPy = 0L; + maxNum = Math.min(maxNum, this.storeConfig.getMaxMsgsNumBatch()); + List phyOffsets = null; + try { + lastUpdateTime = messageRocksDBStorage.getLastStoreTimeStampForIndex(RocksDB.DEFAULT_COLUMN_FAMILY); + lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + //compact old client + if (StringUtils.isEmpty(indexType)) { + phyOffsets = messageRocksDBStorage.queryOffsetForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, topic, MessageConst.INDEX_KEY_TYPE, key, beginTime, endTime, maxNum, null); + if (CollectionUtils.isEmpty(phyOffsets)) { + phyOffsets = messageRocksDBStorage.queryOffsetForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, topic, MessageConst.INDEX_UNIQUE_TYPE, key, beginTime, endTime, maxNum, null); + } + } else { + phyOffsets = messageRocksDBStorage.queryOffsetForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, topic, indexType, key, beginTime, endTime, maxNum, lastKey); + } + } catch (Exception e) { + logError.error("IndexRocksDBStore queryOffset error, topic: {}, key: {}, maxNum: {}, beginTime: {}, endTime: {}, error: {}", topic, key, maxNum, beginTime, endTime, e.getMessage()); + } + return new QueryOffsetResult(phyOffsets, lastUpdateTime, lastOffsetPy); + } + + public void buildIndex(DispatchRequest dispatchRequest) { + if (null == dispatchRequest || dispatchRequest.getCommitLogOffset() < 0L || dispatchRequest.getMsgSize() <= 0 || state != RUNNING || null == this.originIndexMsgQueue) { + logError.error("IndexRocksDBStore buildIndex error, dispatchRequest: {}, state: {}, originIndexMsgQueue: {}", dispatchRequest, state, originIndexMsgQueue); + return; + } + try { + long reqOffsetPy = dispatchRequest.getCommitLogOffset(); + long endOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + if (reqOffsetPy < endOffsetPy) { + if (System.currentTimeMillis() % 1000 == 0) { + log.warn("IndexRocksDBStore recover buildIndex, but ignore, build index offset reqOffsetPy: {}, endOffsetPy: {}", reqOffsetPy, endOffsetPy); + } + return; + } + final int tranType = MessageSysFlag.getTransactionValue(dispatchRequest.getSysFlag()); + switch (tranType) { + case MessageSysFlag.TRANSACTION_NOT_TYPE: + case MessageSysFlag.TRANSACTION_PREPARED_TYPE: + case MessageSysFlag.TRANSACTION_COMMIT_TYPE: + break; + case MessageSysFlag.TRANSACTION_ROLLBACK_TYPE: + return; + } + String topic = dispatchRequest.getTopic(); + String uniqKey = dispatchRequest.getUniqKey(); + long storeTime = dispatchRequest.getStoreTimestamp(); + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey) || storeTime <= 0L || reqOffsetPy < 0L) { + return; + } + String keys = dispatchRequest.getKeys(); + if (!StringUtils.isEmpty(keys)) { + String[] keySplit = keys.split(MessageConst.KEY_SEPARATOR); + if (keySplit.length > 0) { + Set keySet = Arrays.stream(keySplit).filter(i -> !StringUtils.isEmpty(i)).collect(Collectors.toSet()); + for (String key : keySet) { + try { + while (!originIndexMsgQueue.offer(new IndexRocksDBRecord(topic, key, null, storeTime, uniqKey, reqOffsetPy), 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("IndexRocksDBStore buildIndex keys error, topic: {}, key: {}, storeTime: {}, uniqKey: {}, reqOffsetPy: {}", topic, key, storeTime, uniqKey, reqOffsetPy); + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex keys error, key: {}, uniqKey: {}, topic: {}, error: {}", key, uniqKey, topic, e.getMessage()); + } + } + } + } + Map propertiesMap = dispatchRequest.getPropertiesMap(); + if (null != propertiesMap && propertiesMap.containsKey(MessageConst.PROPERTY_TAGS)) { + String tags = propertiesMap.get(MessageConst.PROPERTY_TAGS); + if (!StringUtils.isEmpty(tags)) { + try { + while (!originIndexMsgQueue.offer(new IndexRocksDBRecord(topic, null, tags, storeTime, uniqKey, reqOffsetPy), 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("IndexRocksDBStore buildIndex offer tags error, topic: {}, tags: {}, storeTime: {}, uniqKey: {}, reqOffsetPy: {}", topic, tags, storeTime, uniqKey, reqOffsetPy); + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex tags error, tags: {}, uniqKey: {}, topic: {}, error: {}", tags, uniqKey, topic, e.getMessage()); + } + } + } + try { + while (!originIndexMsgQueue.offer(new IndexRocksDBRecord(topic, null, null, storeTime, uniqKey, reqOffsetPy), 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("IndexRocksDBStore buildIndex uniqKey error, topic: {}, storeTime: {}, uniqKey: {}, reqOffsetPy: {}", topic, storeTime, uniqKey, reqOffsetPy); + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex uniqKey error: {}", e.getMessage()); + } + } catch (Exception e) { + logError.error("IndexRocksDBStore buildIndex error: {}", e.getMessage()); + } + } + + public void deleteExpiredIndex() { + try { + MappedFile mappedFile = messageStore.getCommitLog().getEarliestMappedFile(); + if (null == mappedFile) { + logError.info("IndexRocksDBStore deleteExpiredIndex mappedFile is null"); + return; + } + File file = mappedFile.getFile(); + if (null == file || StringUtils.isEmpty(file.getAbsolutePath())) { + logError.info("IndexRocksDBStore deleteExpiredIndex error, file is null"); + return; + } + Path path = Paths.get(file.getAbsolutePath()); + BasicFileAttributes attrs = Files.readAttributes(path, BasicFileAttributes.class); + long deleteIndexFileTime = attrs.creationTime().toMillis() - TimeUnit.HOURS.toMillis(1); + long desDeleteTimeHour = dealTimeToHourStamps(deleteIndexFileTime); + if (desDeleteTimeHour != lastDeleteIndexTime) { + messageRocksDBStorage.deleteRecordsForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, desDeleteTimeHour, 168); + lastDeleteIndexTime = desDeleteTimeHour; + } else { + log.info("IndexRocksDBStore ignore this delete, lastDeleteIndexTime: {}, desDeleteTimeHour: {}", lastDeleteIndexTime, desDeleteTimeHour); + } + } catch (Exception e) { + logError.error("IndexRocksDBStore deleteExpiredIndex rocksdb error: {}", e.getMessage()); + } + } + + public boolean isMappedFileMatchedRecover(long phyOffset) { + if (!storeConfig.isIndexRocksDBEnable()) { + return true; + } + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(RocksDB.DEFAULT_COLUMN_FAMILY); + log.info("index isMappedFileMatchedRecover lastOffsetPy: {}", lastOffsetPy); + if (null != lastOffsetPy && phyOffset <= lastOffsetPy) { + log.info("isMappedFileMatchedRecover IndexRocksDBStore recover form this offset, phyOffset: {}, lastOffsetPy: {}", phyOffset, lastOffsetPy); + return true; + } + return false; + } + + public void destroy() {} + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) IndexRocksDBStore.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private class IndexBuildService extends ServiceThread { + private final Logger log = IndexRocksDBStore.log; + private List irs; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + irs = new ArrayList<>(BATCH_SIZE); + while (!this.isStopped() || !originIndexMsgQueue.isEmpty()) { + try { + pollAndPutIndexRequest(); + } catch (Exception e) { + irs.clear(); + logError.error("IndexRocksDBStore error occurred in: {}, error: {}", getServiceName(), e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void pollAndPutIndexRequest() { + pollIndexRecord(); + if (CollectionUtils.isEmpty(irs)) { + return; + } + try { + messageRocksDBStorage.writeRecordsForIndex(RocksDB.DEFAULT_COLUMN_FAMILY, irs); + } catch (Exception e) { + logError.error("IndexRocksDBStore IndexBuildService pollAndPutIndexRequest error: {}", e.getMessage()); + } + irs.clear(); + } + + private void pollIndexRecord() { + try { + IndexRocksDBRecord firstReq = originIndexMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null != firstReq) { + irs.add(firstReq); + while (true) { + IndexRocksDBRecord tmpReq = originIndexMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null == tmpReq) { + break; + } + irs.add(tmpReq); + if (irs.size() >= BATCH_SIZE) { + break; + } + } + } + } catch (Exception e) { + logError.error("IndexRocksDBStore IndexBuildService error: {}", e.getMessage()); + } + } + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java b/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java index 0c16d705bd4..fbfffef3bc9 100644 --- a/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java +++ b/store/src/main/java/org/apache/rocketmq/store/logfile/DefaultMappedFile.java @@ -37,6 +37,8 @@ import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; + +import io.netty.util.internal.PlatformDependent; import org.apache.commons.lang3.SystemUtils; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; @@ -57,7 +59,7 @@ import org.apache.rocketmq.store.config.FlushDiskType; import org.apache.rocketmq.store.util.LibC; import sun.misc.Unsafe; -import sun.nio.ch.DirectBuffer; + public class DefaultMappedFile extends AbstractMappedFile { public static final int OS_PAGE_SIZE = 1024 * 4; @@ -592,7 +594,7 @@ public boolean getAndMakeNotWriteable() { if (runningFlags == null) { return false; } - return runningFlags.getAndMakeNotWriteable(); + return runningFlags.getAndMakeStoreNotWriteable(); } public boolean isWriteable() { @@ -914,7 +916,7 @@ public void setFirstCreateInQueue(boolean firstCreateInQueue) { @Override public void mlock() { final long beginTime = System.currentTimeMillis(); - final long address = ((DirectBuffer) (this.mappedByteBuffer)).address(); + final long address = PlatformDependent.directBufferAddress(this.mappedByteBuffer); Pointer pointer = new Pointer(address); { int ret = LibC.INSTANCE.mlock(pointer, new NativeLong(this.fileSize)); @@ -930,7 +932,7 @@ public void mlock() { @Override public void munlock() { final long beginTime = System.currentTimeMillis(); - final long address = ((DirectBuffer) (this.mappedByteBuffer)).address(); + final long address = PlatformDependent.directBufferAddress(this.mappedByteBuffer); Pointer pointer = new Pointer(address); int ret = LibC.INSTANCE.munlock(pointer, new NativeLong(this.fileSize)); log.info("munlock {} {} {} ret = {} time consuming = {}", address, this.fileName, this.fileSize, ret, System.currentTimeMillis() - beginTime); @@ -1049,7 +1051,7 @@ public boolean isLoaded(long position, int size) { return true; } try { - long addr = ((DirectBuffer) mappedByteBuffer).address() + position; + long addr = PlatformDependent.directBufferAddress(mappedByteBuffer) + position; return (boolean) IS_LOADED_METHOD.invoke(mappedByteBuffer, mappingAddr(addr), size, pageCount(size)); } catch (Exception e) { log.info("invoke isLoaded0 of file {} error:", file.getAbsolutePath(), e); diff --git a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java index 673f045bf7f..33908df8628 100644 --- a/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/plugin/AbstractPluginMessageStore.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; +import org.apache.rocketmq.common.BoundaryType; import org.apache.rocketmq.common.Pair; import org.apache.rocketmq.common.SystemClock; import org.apache.rocketmq.common.message.MessageExt; @@ -165,6 +166,11 @@ public long getOffsetInQueueByTime(String topic, int queueId, long timestamp) { return next.getOffsetInQueueByTime(topic, queueId, timestamp); } + @Override + public long getOffsetInQueueByTime(String topic, int queueId, long timestamp, BoundaryType boundaryType) { + return next.getOffsetInQueueByTime(topic, queueId, timestamp, boundaryType); + } + @Override public MessageExt lookMessageByOffset(long commitLogOffset) { return next.lookMessageByOffset(commitLogOffset); diff --git a/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java b/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java index 3e65c104b12..f722c123ea0 100644 --- a/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java +++ b/store/src/main/java/org/apache/rocketmq/store/pop/AckMsg.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.store.pop; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; public class AckMsg { diff --git a/store/src/main/java/org/apache/rocketmq/store/pop/BatchAckMsg.java b/store/src/main/java/org/apache/rocketmq/store/pop/BatchAckMsg.java index 991a1f085de..f2689dfdfb1 100644 --- a/store/src/main/java/org/apache/rocketmq/store/pop/BatchAckMsg.java +++ b/store/src/main/java/org/apache/rocketmq/store/pop/BatchAckMsg.java @@ -16,7 +16,8 @@ */ package org.apache.rocketmq.store.pop; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; + import java.util.ArrayList; import java.util.List; diff --git a/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java b/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java index 38e0a207528..e3587aa28c5 100644 --- a/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java +++ b/store/src/main/java/org/apache/rocketmq/store/pop/PopCheckPoint.java @@ -16,7 +16,8 @@ */ package org.apache.rocketmq.store.pop; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; + import java.util.ArrayList; import java.util.List; @@ -35,7 +36,6 @@ public class PopCheckPoint implements Comparable { private int queueId; @JSONField(name = "t") private String topic; - @JSONField(name = "c") private String cid; @JSONField(name = "ro") private long reviveOffset; @@ -114,10 +114,12 @@ public void setTopic(String topic) { this.topic = topic; } + @JSONField(name = "c") public String getCId() { return cid; } + @JSONField(name = "c") public void setCId(String cid) { this.cid = cid; } 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 c8c3202a785..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 @@ -17,7 +17,7 @@ package org.apache.rocketmq.store.queue; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import com.google.common.annotations.VisibleForTesting; import java.util.Arrays; import java.util.LinkedList; @@ -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 8a5c29e6bc5..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; @@ -828,6 +854,9 @@ protected void deleteExpiredFiles() { } messageStore.getIndexService().deleteExpiredFile(minOffset); + if (messageStoreConfig.isIndexRocksDBEnable() && null != messageStore.getIndexRocksDBStore()) { + messageStore.getIndexRocksDBStore().deleteExpiredIndex(); + } } } diff --git a/store/src/main/java/org/apache/rocketmq/store/queue/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 cf511b1bccc..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(); } @@ -621,6 +631,9 @@ protected void deleteExpiredFiles() { } messageStore.getIndexService().deleteExpiredFile(minOffset); + if (messageStoreConfig.isIndexRocksDBEnable() && null != messageStore.getIndexRocksDBStore()) { + messageStore.getIndexRocksDBStore().deleteExpiredIndex(); + } } } diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/MessageRocksDBStorage.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/MessageRocksDBStorage.java new file mode 100644 index 00000000000..8d32998bdce --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/MessageRocksDBStorage.java @@ -0,0 +1,652 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.rocksdb; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.UtilAll; +import org.apache.rocketmq.common.config.AbstractRocksDBStorage; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.index.rocksdb.IndexRocksDBRecord; +import org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord; +import org.apache.rocketmq.store.transaction.TransRocksDBRecord; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.ReadOptions; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksIterator; +import org.rocksdb.Slice; +import org.rocksdb.WriteBatch; +import static org.apache.rocketmq.common.MixAll.dealTimeToHourStamps; +import static org.apache.rocketmq.common.MixAll.getHours; +import static org.apache.rocketmq.common.MixAll.isHourTime; +import static org.apache.rocketmq.store.index.rocksdb.IndexRocksDBRecord.KEY_SPLIT; +import static org.apache.rocketmq.store.index.rocksdb.IndexRocksDBRecord.KEY_SPLIT_BYTES; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_DELETE; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_PUT; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_UPDATE; + +public class MessageRocksDBStorage extends AbstractRocksDBStorage { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String ROCKSDB_MESSAGE_DIRECTORY = "rocksdbstore"; + + public static final byte[] TIMER_COLUMN_FAMILY = "timer".getBytes(StandardCharsets.UTF_8); + public static final byte[] TRANS_COLUMN_FAMILY = "trans".getBytes(StandardCharsets.UTF_8); + private static final byte[] LAST_OFFSET_PY = "lastOffsetPy".getBytes(StandardCharsets.UTF_8); + private static final byte[] LAST_STORE_TIMESTAMP = "lastStoreTimeStamp".getBytes(StandardCharsets.UTF_8); + private static final byte[] END_SUFFIX_BYTES = new byte[512]; + static { + Arrays.fill(END_SUFFIX_BYTES, (byte) 0xFF); + } + private static final Set COMMON_CHECK_POINT_KEY_SET_FOR_TIMER = new HashSet<>(); + public static final byte[] SYS_TOPIC_SCAN_OFFSET_CHECK_POINT = "sys_topic_scan_offset_checkpoint".getBytes(StandardCharsets.UTF_8); + public static final byte[] TIMELINE_CHECK_POINT = "timeline_checkpoint".getBytes(StandardCharsets.UTF_8); + static { + COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.add(SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.add(TIMELINE_CHECK_POINT); + } + private static final byte[] DELETE_VAL_FLAG = new byte[] {(byte)0xFF}; + private static final int LAST_OFFSET_PY_LENGTH = LAST_OFFSET_PY.length; + + private volatile ColumnFamilyHandle timerCFHandle; + private volatile ColumnFamilyHandle transCFHandle; + + private final ScheduledExecutorService scheduler = Executors.newScheduledThreadPool(1); + private static final Cache DELETE_KEY_CACHE_FOR_TIMER = CacheBuilder.newBuilder() + .maximumSize(10000) + .expireAfterWrite(60, TimeUnit.MINUTES) + .build(); + + public MessageRocksDBStorage(MessageStoreConfig messageStoreConfig) { + super(Paths.get(messageStoreConfig.getStorePathRootDir(), ROCKSDB_MESSAGE_DIRECTORY).toString()); + this.start(); + } + + @Override + protected boolean postLoad() { + try { + UtilAll.ensureDirOK(this.dbPath); + initOptions(); + ColumnFamilyOptions indexCFOptions = RocksDBOptionsFactory.createIndexCFOptions(); + ColumnFamilyOptions timerCFOptions = RocksDBOptionsFactory.createTimerCFOptions(); + ColumnFamilyOptions transCFOptions = RocksDBOptionsFactory.createTransCFOptions(); + this.cfOptions.add(indexCFOptions); + this.cfOptions.add(timerCFOptions); + this.cfOptions.add(transCFOptions); + + List cfDescriptors = new ArrayList<>(); + cfDescriptors.add(new ColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY, indexCFOptions)); + cfDescriptors.add(new ColumnFamilyDescriptor(TIMER_COLUMN_FAMILY, timerCFOptions)); + cfDescriptors.add(new ColumnFamilyDescriptor(TRANS_COLUMN_FAMILY, transCFOptions)); + this.open(cfDescriptors); + this.defaultCFHandle = cfHandles.get(0); + this.timerCFHandle = cfHandles.get(1); + this.transCFHandle = cfHandles.get(2); + scheduler.scheduleAtFixedRate(() -> { + try { + db.flush(flushOptions, timerCFHandle); + log.info("MessageRocksDBStorage flush timer wal success"); + } catch (Exception e) { + logError.error("MessageRocksDBStorage flush timer wal failed, error: {}", e.getMessage()); + } + }, 5, 5, TimeUnit.MINUTES); + + log.info("MessageRocksDBStorage init success, dbPath: {}", this.dbPath); + } catch (final Exception e) { + logError.error("MessageRocksDBStorage init error, dbPath: {}, error: {}", this.dbPath, e.getMessage()); + return false; + } + return true; + } + + protected void initOptions() { + this.options = RocksDBOptionsFactory.createDBOptions(); + super.initOptions(); + } + + public String getFilePath() { + return this.dbPath; + } + + @Override + protected void preShutdown() { + log.info("MessageRocksDBStorage pre shutdown success, dbPath: {}", this.dbPath); + } + + public List queryOffsetForIndex(byte[] columnFamily, String topic, String indexType, String key, long beginTime, long endTime, int maxNum, String lastKey) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || StringUtils.isEmpty(topic) || StringUtils.isEmpty(indexType) || StringUtils.isEmpty(key) || beginTime < 0L || endTime <= 0L || beginTime > endTime || maxNum <= 0) { + logError.error("MessageRocksDBStorage queryOffsetForIndex param error, cfHandle: {}, topic: {}, indexType: {}, key: {}, beginTime: {}, endTime: {}, maxNum: {}", cfHandle, topic, indexType, key, beginTime, endTime, maxNum); + return null; + } + Long lastIndexTime = getLastIndexTimeForIndex(lastKey); + if (!StringUtils.isEmpty(lastKey) && (null == lastIndexTime || lastIndexTime <= 0L || !isHourTime(lastIndexTime))) { + logError.error("MessageRocksDBStorage queryOffsetForIndex parse and check lastIndexTime error, lastIndexTime: {}, lastKey: {}", lastIndexTime, lastKey); + return null; + } + List hours = getHours(beginTime, endTime); + if (CollectionUtils.isEmpty(hours)) { + logError.error("MessageRocksDBStorage queryOffsetForIndex param error, hours is empty, beginTime: {}, endTime: {}", beginTime, endTime); + return null; + } + List offsetPyList = new ArrayList<>(maxNum); + String keyMiddleStr = KEY_SPLIT + topic + KEY_SPLIT + indexType + KEY_SPLIT + key + KEY_SPLIT; + byte[] keyMiddleBytes = keyMiddleStr.getBytes(StandardCharsets.UTF_8); + for (Long hour : hours) { + if (null == hour || null != lastIndexTime && hour < lastIndexTime) { + continue; + } + byte[] seekKeyBytes = null; + byte[] lastKeyBytes = null; + byte[] keyPrefixBytes = ByteBuffer.allocate(Long.BYTES + keyMiddleBytes.length).putLong(hour).put(keyMiddleBytes).array(); + if (!StringUtils.isEmpty(lastKey) && hour.equals(lastIndexTime)) { + seekKeyBytes = lastKeyToBytes(lastKey); + lastKeyBytes = seekKeyBytes; + } else { + seekKeyBytes = keyPrefixBytes; + } + if (null == seekKeyBytes) { + logError.error("MessageRocksDBStorage queryOffsetForIndex error, seekKeyBytes is null"); + return null; + } + try (RocksIterator iterator = db.newIterator(cfHandle, readOptions)) { + for (iterator.seek(seekKeyBytes); iterator.isValid(); iterator.next()) { + try { + byte[] currentKeyBytes = iterator.key(); + if (null == currentKeyBytes || currentKeyBytes.length == 0) { + break; + } + if (null != lastKeyBytes && currentKeyBytes.length == lastKeyBytes.length && MixAll.isByteArrayEqual(currentKeyBytes, 0, currentKeyBytes.length, lastKeyBytes, 0, lastKeyBytes.length)) { + continue; + } + if (currentKeyBytes.length < keyPrefixBytes.length || !MixAll.isByteArrayEqual(currentKeyBytes, 0, keyPrefixBytes.length, keyPrefixBytes, 0, keyPrefixBytes.length)) { + break; + } + ByteBuffer valueBuffer = ByteBuffer.wrap(iterator.value()); + long storeTime = valueBuffer.getLong(); + if (storeTime >= beginTime && storeTime <= endTime) { + byte[] indexKey = iterator.key(); + if (null == indexKey || indexKey.length < Long.BYTES) { + continue; + } + byte[] bytes = Arrays.copyOfRange(indexKey, indexKey.length - Long.BYTES, indexKey.length); + long offset = ByteBuffer.wrap(bytes).getLong(); + offsetPyList.add(offset); + if (offsetPyList.size() >= maxNum) { + return offsetPyList; + } + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage queryOffsetForIndex iterator error: {}", e.getMessage()); + } + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage queryOffsetForIndex error: {}", e.getMessage()); + } + } + return offsetPyList; + } + + private byte[] lastKeyToBytes(String lastKey) { + if (StringUtils.isEmpty(lastKey)) { + return null; + } + String[] split = lastKey.split(KEY_SPLIT); + if (split.length != 6) { + log.error("MessageRocksDBStorage lastKeyToBytes split error, lastKey: {}", lastKey); + return null; + } + try { + long storeTimeHour = Long.parseLong(split[0]); + long offsetPy = Long.parseLong(split[split.length - 1]); + StringBuilder stringBuilder = new StringBuilder(); + for (int i = 1; i < split.length - 1; i++) { + stringBuilder.append(KEY_SPLIT).append(split[i]); + } + byte[] middleKeyBytes = stringBuilder.append(KEY_SPLIT).toString().getBytes(StandardCharsets.UTF_8); + return ByteBuffer.allocate(Long.BYTES + middleKeyBytes.length + Long.BYTES).putLong(storeTimeHour).put(middleKeyBytes).putLong(offsetPy).array(); + } catch (Exception e) { + log.error("MessageRocksDBStorage lastKeyToBytes error, lastKey: {}, error: {}", lastKey, e.getMessage()); + return null; + } + } + + public void deleteRecordsForIndex(byte[] columnFamily, long storeTime, int hours) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || storeTime < 0L || hours <= 0) { + logError.error("MessageRocksDBStorage deleteRecordsForIndex param error, storeTime: {}, hours: {}", storeTime, hours); + return; + } + long endTime = dealTimeToHourStamps(storeTime); + long startTime = endTime - TimeUnit.HOURS.toMillis(hours); + try { + byte[] startKey = ByteBuffer.allocate(Long.BYTES + KEY_SPLIT_BYTES.length).putLong(startTime).put(KEY_SPLIT_BYTES).array(); + byte[] endKey = ByteBuffer.allocate(Long.BYTES + KEY_SPLIT_BYTES.length + END_SUFFIX_BYTES.length).putLong(endTime).put(KEY_SPLIT_BYTES).put(END_SUFFIX_BYTES).array(); + rangeDelete(cfHandle, ableWalWriteOptions, startKey, endKey); + log.info("MessageRocksDBStorage deleteRecordsForIndex delete success, storeTime: {}, hours: {}", storeTime, hours); + } catch (Exception e) { + logError.error("MessageRocksDBStorage deleteRecordsForIndex delete error, storeTime: {}, hours: {}, error: {}", storeTime, hours, e.getMessage()); + } + } + + public void writeRecordsForIndex(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + try (WriteBatch writeBatch = new WriteBatch()) { + for (IndexRocksDBRecord record : recordList) { + try { + if (null == record) { + logError.warn("MessageRocksDBStorage writeRecordsForIndex error, record is null"); + continue; + } + byte[] keyBytes = record.getKeyBytes(); + byte[] valueBytes = record.getValueBytes(); + if (null == keyBytes || keyBytes.length == 0 || null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForIndex param error, keyBytes: {}, valueBytes: {}", keyBytes, valueBytes); + continue; + } + writeBatch.put(cfHandle, keyBytes, valueBytes); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForIndex error: {}", e.getMessage()); + } + } + IndexRocksDBRecord lastRecord = recordList.get(recordList.size() - 1); + if (null != lastRecord && StringUtils.isEmpty(lastRecord.getKey()) && StringUtils.isEmpty(lastRecord.getTag())) { + long offset = lastRecord.getOffsetPy(); + Long lastOffsetPy = getLastOffsetPy(columnFamily); + if (null == lastOffsetPy || offset > lastOffsetPy) { + writeBatch.put(cfHandle, LAST_OFFSET_PY, ByteBuffer.allocate(Long.BYTES).putLong(offset).array()); + } + long storeTime = lastRecord.getStoreTime(); + Long lastStoreTimeStamp = getLastStoreTimeStampForIndex(columnFamily); + if (null == lastStoreTimeStamp || storeTime > lastStoreTimeStamp) { + writeBatch.put(cfHandle, LAST_STORE_TIMESTAMP, ByteBuffer.allocate(Long.BYTES).putLong(storeTime).array()); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForIndex error: {}", e.getMessage()); + } + } + + public Long getLastStoreTimeStampForIndex(byte[] columnFamily) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle) { + return null; + } + try { + byte[] storeTime = get(cfHandle, readOptions, LAST_STORE_TIMESTAMP); + return null == storeTime ? 0L : ByteBuffer.wrap(storeTime).getLong(); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getLastStoreTimeStampForIndex error: {}", e.getMessage()); + return null; + } + } + + private static Long getLastIndexTimeForIndex(String lastKey) { + if (StringUtils.isEmpty(lastKey)) { + return null; + } + try { + String[] split = lastKey.split(KEY_SPLIT); + if (split.length > 0) { + return Long.valueOf(split[0]); + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage getLastIndexTimeForIndex error lastKey: {}, e: {}", lastKey, e.getMessage()); + } + return null; + } + + public void writeRecordsForTimer(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + try (WriteBatch writeBatch = new WriteBatch()) { + for (TimerRocksDBRecord record : recordList) { + if (null == record) { + logError.error("MessageRocksDBStorage writeRecordsForTimer error, record is null"); + continue; + } + try { + byte[] keyBytes = record.getKeyBytes(); + byte[] valueBytes = record.getValueBytes(); + if (null == keyBytes || keyBytes.length == 0 || null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForTimer param error, keyBytes: {}, valueBytes: {}", keyBytes, valueBytes); + continue; + } + if (record.getActionFlag() == TIMER_ROCKSDB_PUT) { + writeBatch.put(cfHandle, keyBytes, valueBytes); + } else if (record.getActionFlag() == TIMER_ROCKSDB_DELETE) { + writeBatch.delete(cfHandle, keyBytes); + DELETE_KEY_CACHE_FOR_TIMER.put(keyBytes, DELETE_VAL_FLAG); + } else if (record.getActionFlag() == TIMER_ROCKSDB_UPDATE) { + byte[] deleteByte = DELETE_KEY_CACHE_FOR_TIMER.getIfPresent(keyBytes); + if (null == deleteByte) { + writeBatch.put(cfHandle, keyBytes, valueBytes); + } + } else { + logError.error("MessageRocksDBStorage writeRecordsForTimer record actionFlag error, actionFlag: {}", record.getActionFlag()); + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForTimer error: {}", e.getMessage()); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForTimer error: {}", e.getMessage()); + } + } + + public List scanRecordsForTimer(byte[] columnFamily, long lowerTime, long upperTime, int size, byte[] startKey) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || lowerTime <= 0L || upperTime <= 0L || lowerTime > upperTime || size <= 0) { + return null; + } + RocksIterator iterator = null; + try (ReadOptions readOptions = new ReadOptions() + .setIterateLowerBound(new Slice(ByteBuffer.allocate(Long.BYTES).putLong(lowerTime).array())) + .setIterateUpperBound(new Slice(ByteBuffer.allocate(Long.BYTES).putLong(upperTime).array()))) { + iterator = db.newIterator(cfHandle, readOptions); + if (null == startKey || startKey.length == 0) { + iterator.seek(ByteBuffer.allocate(Long.BYTES).putLong(lowerTime).array()); + } else { + iterator.seek(startKey); + iterator.next(); + } + List records = new ArrayList<>(); + for (; iterator.isValid(); iterator.next()) { + try { + TimerRocksDBRecord timerRocksDBRecord = TimerRocksDBRecord.decode(iterator.key(), iterator.value()); + if (null == timerRocksDBRecord) { + logError.error("MessageRocksDBStorage scanRecordsForTimer error, decode timerRocksDBRecord is null"); + continue; + } + records.add(timerRocksDBRecord); + if (records.size() >= size) { + break; + } + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTimer iterator error: {}", e.getMessage()); + } + } + return records; + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTimer error: {}", e.getMessage()); + } finally { + if (null != iterator) { + iterator.close(); + } + } + return null; + } + + public void deleteRecordsForTimer(byte[] columnFamily, long lowerTime, long upperTime) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || lowerTime <= 0L || upperTime <= 0L || lowerTime > upperTime) { + logError.error("MessageRocksDBStorage deleteRecordsForTimer param error, cfHandle: {}, lowerTime: {}, upperTime: {}", cfHandle, lowerTime, upperTime); + return; + } + byte[] startKey = ByteBuffer.allocate(Long.BYTES).putLong(lowerTime).array(); + byte[] endKey = ByteBuffer.allocate(Long.BYTES + END_SUFFIX_BYTES.length).putLong(upperTime).put(END_SUFFIX_BYTES).array(); + try { + rangeDelete(cfHandle, ableWalWriteOptions, startKey, endKey); + log.info("MessageRocksDBStorage deleteRecordsForTimer success, lowerTime: {}, upperTime: {}", lowerTime, upperTime); + } catch (Exception e) { + logError.error("MessageRocksDBStorage deleteRecordsForTimer param error, lowerTime: {}, upperTime: {}, error: {}", lowerTime, upperTime, e.getMessage()); + } + } + + public void writeCheckPointForTimer(byte[] columnFamily, byte[] key, long value) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || !COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.contains(key) || value < 0L) { + logError.error("MessageRocksDBStorage writeCheckPointForTimer param error, cfHandle: {}, key: {}, value: {}", cfHandle, key, value); + return; + } + try { + byte[] valueBytes = ByteBuffer.allocate(Long.BYTES).putLong(value).array(); + put(cfHandle, ableWalWriteOptions, key, key.length, valueBytes, valueBytes.length); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeCheckPointForTimer error: {}", e.getMessage()); + } + } + + public long getCheckpointForTimer(byte[] columnFamily, byte[] key) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || !COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.contains(key)) { + logError.error("MessageRocksDBStorage getCheckpointForTimer error, cfHandle: {}, key: {}", cfHandle, key); + return 0L; + } + try { + byte[] checkpoint = get(cfHandle, readOptions, key); + if (null == checkpoint && Arrays.equals(key, TIMELINE_CHECK_POINT)) { + return (System.currentTimeMillis() - TimeUnit.SECONDS.toMillis(10)) / TimeUnit.SECONDS.toMillis(1) * TimeUnit.SECONDS.toMillis(1); + } + return checkpoint == null ? 0L : ByteBuffer.wrap(checkpoint).getLong(); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getCheckpointForTimer error: {}", e.getMessage()); + return 0L; + } + } + + public void deleteCheckPointForTimer(byte[] columnFamily, byte[] key) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || !COMMON_CHECK_POINT_KEY_SET_FOR_TIMER.contains(key)) { + logError.error("MessageRocksDBStorage deleteCheckPointForTimer error, cfHandle: {}, key: {}", cfHandle, key); + return; + } + try { + delete(cfHandle, ableWalWriteOptions, key); + } catch (Exception e) { + logError.error("MessageRocksDBStorage deleteCheckPointForTimer error: {}", e.getMessage()); + throw new RuntimeException("MessageRocksDBStorage deleteCheckPointForTimer error", e); + } + } + + public void writeRecordsForTrans(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + long lastOffsetPy = 0L; + try (WriteBatch writeBatch = new WriteBatch()) { + for (TransRocksDBRecord record : recordList) { + if (null == record) { + logError.error("MessageRocksDBStorage writeRecordsForTrans error, record is null"); + continue; + } + byte[] keyBytes = record.getKeyBytes(); + if (null == keyBytes || keyBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForTrans param error, keyBytes: {}", keyBytes); + continue; + } + if (record.isOp()) { + writeBatch.delete(cfHandle, record.getKeyBytes()); + } else { + byte[] valueBytes = record.getValueBytes(); + if (null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage writeRecordsForTrans param error, valueBytes: {}", valueBytes); + continue; + } + writeBatch.put(cfHandle, keyBytes, valueBytes); + lastOffsetPy = Math.max(lastOffsetPy, record.getOffsetPy()); + } + } + if (lastOffsetPy > 0L) { + Long lastOffsetPyStore = getLastOffsetPy(columnFamily); + if (null == lastOffsetPyStore || lastOffsetPy > lastOffsetPyStore) { + writeBatch.put(cfHandle, LAST_OFFSET_PY, ByteBuffer.allocate(Long.BYTES).putLong(lastOffsetPy).array()); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage writeRecordsForTrans error: {}", e.getMessage()); + } + } + + public void updateRecordsForTrans(byte[] columnFamily, List recordList) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || CollectionUtils.isEmpty(recordList)) { + return; + } + try (WriteBatch writeBatch = new WriteBatch()) { + for (TransRocksDBRecord record : recordList) { + if (null == record) { + logError.error("MessageRocksDBStorage updateRecordsForTrans error, record is null"); + continue; + } + byte[] keyBytes = record.getKeyBytes(); + byte[] valueBytes = record.getValueBytes(); + if (null == keyBytes || keyBytes.length == 0 || null == valueBytes || valueBytes.length == 0) { + logError.error("MessageRocksDBStorage updateRecordsForTrans param error, keyBytes: {}, valueBytes: {}", keyBytes, valueBytes); + continue; + } + if (record.isDelete()) { + writeBatch.delete(cfHandle, keyBytes); + } else { + writeBatch.put(cfHandle, keyBytes, valueBytes); + } + } + batchPut(ableWalWriteOptions, writeBatch); + } catch (Exception e) { + logError.error("MessageRocksDBStorage updateRecordsForTrans error: {}", e.getMessage()); + } + } + + public List scanRecordsForTrans(byte[] columnFamily, int size, byte[] startKey) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || size <= 0) { + return null; + } + RocksIterator iterator = null; + try { + iterator = db.newIterator(cfHandle); + if (null == startKey || startKey.length == 0) { + iterator.seekToFirst(); + } else { + iterator.seek(startKey); + iterator.next(); + } + List records = new ArrayList<>(); + for (; iterator.isValid(); iterator.next()) { + byte[] key = iterator.key(); + if (null == key || key.length == 0 || key.length == LAST_OFFSET_PY_LENGTH && Arrays.equals(key, LAST_OFFSET_PY)) { + continue; + } + TransRocksDBRecord transRocksDBRecord = null; + try { + transRocksDBRecord = TransRocksDBRecord.decode(key, iterator.value()); + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTrans error: {}", e.getMessage()); + } + if (null != transRocksDBRecord) { + records.add(transRocksDBRecord); + } + if (records.size() >= size) { + break; + } + } + return records; + } catch (Exception e) { + logError.error("MessageRocksDBStorage scanRecordsForTrans error: {}", e.getMessage()); + } finally { + if (null != iterator) { + iterator.close(); + } + } + return null; + } + + public TransRocksDBRecord getRecordForTrans(byte[] columnFamily, TransRocksDBRecord transRocksDBRecord) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle || null == transRocksDBRecord) { + return null; + } + try { + byte[] keyBytes = transRocksDBRecord.getKeyBytes(); + if (null == keyBytes) { + return null; + } + byte[] valueBytes = get(cfHandle, readOptions, keyBytes); + if (null == valueBytes || valueBytes.length != TransRocksDBRecord.VALUE_LENGTH) { + return null; + } + return TransRocksDBRecord.decode(keyBytes, valueBytes); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getRecordForTrans error: {}", e.getMessage()); + return null; + } + } + + public Long getLastOffsetPy(byte[] columnFamily) { + ColumnFamilyHandle cfHandle = getColumnFamily(columnFamily); + if (null == cfHandle) { + return null; + } + try { + byte[] offsetBytes = get(cfHandle, readOptions, LAST_OFFSET_PY); + return offsetBytes == null ? 0L : ByteBuffer.wrap(offsetBytes).getLong(); + } catch (Exception e) { + logError.error("MessageRocksDBStorage getLastOffsetPy error: {}", e.getMessage()); + return null; + } + } + + @Override + public synchronized boolean shutdown() { + try { + boolean result = super.shutdown(); + log.info("shutdown MessageRocksDBStorage result: {}", result); + return result; + } catch (Exception e) { + logError.error("shutdown MessageRocksDBStorage error : {}", e.getMessage()); + return false; + } + } + + private ColumnFamilyHandle getColumnFamily(byte[] columnFamily) { + if (Arrays.equals(columnFamily, RocksDB.DEFAULT_COLUMN_FAMILY)) { + return this.defaultCFHandle; + } else if (Arrays.equals(columnFamily, TIMER_COLUMN_FAMILY)) { + return this.timerCFHandle; + } else if (Arrays.equals(columnFamily, TRANS_COLUMN_FAMILY)) { + return this.transCFHandle; + } + throw new RuntimeException("Unknown column family"); + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java b/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java index e365326c76d..07152a953a8 100644 --- a/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java +++ b/store/src/main/java/org/apache/rocketmq/store/rocksdb/RocksDBOptionsFactory.java @@ -217,4 +217,149 @@ public static DBOptions createDBOptions() { setUseDirectIoForFlushAndCompaction(false). setUseDirectReads(false); } + + public static ColumnFamilyOptions createTimerCFOptions() { + BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig() + .setFormatVersion(5) + .setIndexType(IndexType.kBinarySearch) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setDataBlockHashTableUtilRatio(0.75) + .setBlockSize(128 * SizeUnit.KB) + .setMetadataBlockSize(4 * SizeUnit.KB) + .setFilterPolicy(new BloomFilter(16, false)) + .setCacheIndexAndFilterBlocks(false) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(false) + .setPinTopLevelIndexAndFilter(true) + .setBlockCache(new LRUCache(2048 * SizeUnit.MB, 8, false)) + .setWholeKeyFiltering(true); + + //noinspection resource + return new ColumnFamilyOptions() + .setMaxWriteBufferNumber(6) + .setWriteBufferSize(256 * SizeUnit.MB) + .setMinWriteBufferNumberToMerge(1) + .setTableFormatConfig(blockBasedTableConfig) + .setMemTableConfig(new SkipListMemTableConfig()) + .setCompressionType(CompressionType.ZSTD_COMPRESSION) + .setBottommostCompressionType(CompressionType.NO_COMPRESSION) + .setNumLevels(7) + .setCompactionPriority(CompactionPriority.MinOverlappingRatio) + .setCompactionStyle(CompactionStyle.LEVEL) + .setMaxCompactionBytes(256 * SizeUnit.MB) + .setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB) + .setHardPendingCompactionBytesLimit(256 * SizeUnit.GB) + .setLevel0FileNumCompactionTrigger(2) + .setLevel0SlowdownWritesTrigger(8) + .setLevel0StopWritesTrigger(10) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setTargetFileSizeMultiplier(2) + .setMergeOperator(new StringAppendOperator()) + .setReportBgIoStats(true) + .setOptimizeFiltersForHits(true) + .setMaxBytesForLevelBase(512 * SizeUnit.MB); + } + + public static ColumnFamilyOptions createTransCFOptions() { + BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig() + .setFormatVersion(5) + .setIndexType(IndexType.kBinarySearch) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setDataBlockHashTableUtilRatio(0.75) + .setBlockSize(128 * SizeUnit.KB) + .setMetadataBlockSize(4 * SizeUnit.KB) + .setFilterPolicy(new BloomFilter(16, false)) + .setCacheIndexAndFilterBlocks(false) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(false) + .setPinTopLevelIndexAndFilter(true) + .setBlockCache(new LRUCache(1024 * SizeUnit.MB, 8, false)) + .setWholeKeyFiltering(true); + + CompactionOptionsUniversal compactionOption = new CompactionOptionsUniversal() + .setSizeRatio(100) + .setMaxSizeAmplificationPercent(25) + .setAllowTrivialMove(true) + .setMinMergeWidth(2) + .setMaxMergeWidth(Integer.MAX_VALUE) + .setStopStyle(CompactionStopStyle.CompactionStopStyleTotalSize) + .setCompressionSizePercent(-1); + + //noinspection resource + return new ColumnFamilyOptions() + .setMaxWriteBufferNumber(6) + .setWriteBufferSize(128 * SizeUnit.MB) + .setMinWriteBufferNumberToMerge(1) + .setTableFormatConfig(blockBasedTableConfig) + .setMemTableConfig(new SkipListMemTableConfig()) + .setCompressionType(CompressionType.NO_COMPRESSION) + .setBottommostCompressionType(CompressionType.NO_COMPRESSION) + .setNumLevels(7) + .setCompactionPriority(CompactionPriority.MinOverlappingRatio) + .setCompactionStyle(CompactionStyle.UNIVERSAL) + .setCompactionOptionsUniversal(compactionOption) + .setMaxCompactionBytes(100 * SizeUnit.GB) + .setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB) + .setHardPendingCompactionBytesLimit(256 * SizeUnit.GB) + .setLevel0FileNumCompactionTrigger(2) + .setLevel0SlowdownWritesTrigger(8) + .setLevel0StopWritesTrigger(10) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setTargetFileSizeMultiplier(2) + .setMergeOperator(new StringAppendOperator()) + .setReportBgIoStats(true) + .setOptimizeFiltersForHits(true); + } + + public static ColumnFamilyOptions createIndexCFOptions() { + BlockBasedTableConfig blockBasedTableConfig = new BlockBasedTableConfig() + .setFormatVersion(5) + .setIndexType(IndexType.kBinarySearch) + .setDataBlockIndexType(DataBlockIndexType.kDataBlockBinaryAndHash) + .setDataBlockHashTableUtilRatio(0.75) + .setBlockSize(128 * SizeUnit.KB) + .setMetadataBlockSize(4 * SizeUnit.KB) + .setFilterPolicy(new BloomFilter(16, false)) + .setCacheIndexAndFilterBlocks(false) + .setCacheIndexAndFilterBlocksWithHighPriority(true) + .setPinL0FilterAndIndexBlocksInCache(false) + .setPinTopLevelIndexAndFilter(true) + .setBlockCache(new LRUCache(1024 * SizeUnit.MB, 8, false)) + .setWholeKeyFiltering(true); + + CompactionOptionsUniversal compactionOption = new CompactionOptionsUniversal() + .setSizeRatio(100) + .setMaxSizeAmplificationPercent(25) + .setAllowTrivialMove(true) + .setMinMergeWidth(2) + .setMaxMergeWidth(Integer.MAX_VALUE) + .setStopStyle(CompactionStopStyle.CompactionStopStyleTotalSize) + .setCompressionSizePercent(-1); + + //noinspection resource + return new ColumnFamilyOptions() + .setMaxWriteBufferNumber(6) + .setWriteBufferSize(128 * SizeUnit.MB) + .setMinWriteBufferNumberToMerge(1) + .setTableFormatConfig(blockBasedTableConfig) + .setMemTableConfig(new SkipListMemTableConfig()) + .setCompressionType(CompressionType.NO_COMPRESSION) + .setBottommostCompressionType(CompressionType.NO_COMPRESSION) + .setNumLevels(7) + .setCompactionPriority(CompactionPriority.MinOverlappingRatio) + .setCompactionStyle(CompactionStyle.UNIVERSAL) + .setCompactionOptionsUniversal(compactionOption) + .setMaxCompactionBytes(256 * SizeUnit.MB) + .setSoftPendingCompactionBytesLimit(100 * SizeUnit.GB) + .setHardPendingCompactionBytesLimit(256 * SizeUnit.GB) + .setLevel0FileNumCompactionTrigger(8) + .setLevel0SlowdownWritesTrigger(8) + .setLevel0StopWritesTrigger(20) + .setTargetFileSizeBase(256 * SizeUnit.MB) + .setTargetFileSizeMultiplier(2) + .setMergeOperator(new StringAppendOperator()) + .setReportBgIoStats(true) + .setOptimizeFiltersForHits(true); + } + } diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerLog.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerLog.java index 01b56ee449b..689f1d792f0 100644 --- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerLog.java +++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerLog.java @@ -19,6 +19,7 @@ import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.RunningFlags; import org.apache.rocketmq.store.logfile.MappedFile; import org.apache.rocketmq.store.MappedFileQueue; import org.apache.rocketmq.store.SelectMappedBufferResult; @@ -45,8 +46,12 @@ public class TimerLog { private final int fileSize; public TimerLog(final String storePath, final int fileSize) { + this(storePath, fileSize, null, false); + } + + public TimerLog(final String storePath, final int fileSize, RunningFlags runningFlags, boolean writeWithoutMmap) { this.fileSize = fileSize; - this.mappedFileQueue = new MappedFileQueue(storePath, fileSize, null); + this.mappedFileQueue = new MappedFileQueue(storePath, fileSize, null, runningFlags, writeWithoutMmap); } public boolean load() { diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java index 9ff2544db17..53999e72c4c 100644 --- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java +++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMessageStore.java @@ -17,6 +17,7 @@ package org.apache.rocketmq.store.timer; import com.conversantmedia.util.concurrent.DisruptorBlockingQueue; +import io.opentelemetry.api.common.Attributes; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -40,7 +41,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; -import io.opentelemetry.api.common.Attributes; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.math.NumberUtils; import org.apache.rocketmq.common.ServiceThread; @@ -61,7 +61,9 @@ import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.store.MessageStore; import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.RunningFlags; import org.apache.rocketmq.store.SelectMappedBufferResult; +import org.apache.rocketmq.store.StoreUtil; import org.apache.rocketmq.store.config.BrokerRole; import org.apache.rocketmq.store.config.MessageStoreConfig; import org.apache.rocketmq.store.logfile.MappedFile; @@ -160,6 +162,8 @@ public class TimerMessageStore { private final BrokerStatsManager brokerStatsManager; private Function escapeBridgeHook; + private final Object lockWhenFlush = new Object(); + public TimerMessageStore(final MessageStore messageStore, final MessageStoreConfig storeConfig, TimerCheckpoint timerCheckpoint, TimerMetrics timerMetrics, final BrokerStatsManager brokerStatsManager) throws IOException { @@ -172,9 +176,29 @@ public TimerMessageStore(final MessageStore messageStore, final MessageStoreConf // TimerWheel contains the fixed number of slots regardless of precision. this.slotsTotal = TIMER_WHEEL_TTL_DAY * DAY_SECS; + + String timerWheelPath = getTimerWheelPath(storeConfig.getStorePathRootDir()); + long snapOffset = -1; + if (storeConfig.isTimerWheelSnapshotFlush()) { + snapOffset = TimerWheel.getMaxSnapshotFlag(timerWheelPath); + if (snapOffset > 0) { + // correct recover offset + timerCheckpoint.setLastTimerLogFlushPos(snapOffset); + LOGGER.info("found timerWheel snapshot offset {}", snapOffset); + } else { + LOGGER.info("not found timerWheel snapshot", snapOffset); + } + } + + RunningFlags runningFlags = null; + if (storeConfig.isEnableRunningFlagsInFlush() && messageStore != null) { + runningFlags = messageStore.getRunningFlags(); + } + this.timerWheel = new TimerWheel( - getTimerWheelPath(storeConfig.getStorePathRootDir()), this.slotsTotal, precisionMs); - this.timerLog = new TimerLog(getTimerLogPath(storeConfig.getStorePathRootDir()), timerLogFileSize); + timerWheelPath, this.slotsTotal, precisionMs, snapOffset); + this.timerLog = new TimerLog(getTimerLogPath(storeConfig.getStorePathRootDir()), timerLogFileSize, + runningFlags, storeConfig.isWriteWithoutMmap()); this.timerMetrics = timerMetrics; this.timerCheckpoint = timerCheckpoint; this.lastBrokerRole = storeConfig.getBrokerRole(); @@ -293,7 +317,11 @@ public void recover() { currQueueOffset = queueOffset + 1; } currQueueOffset = Math.min(currQueueOffset, timerCheckpoint.getMasterTimerQueueOffset()); - + if (storeConfig.isTimerRocksDBEnable()) { + long commitOffsetInRocksDB = messageStore.getTimerRocksDBStore().getCommitOffsetInRocksDB(); + LOGGER.info("recover time wheel, currQueueOffset: {}, commitOffsetInRocksDB: {}", currQueueOffset, commitOffsetInRocksDB); + currQueueOffset = Math.max(currQueueOffset, commitOffsetInRocksDB); + } ConsumeQueueInterface cq = this.messageStore.getConsumeQueue(TIMER_TOPIC, 0); // Correction based consume queue @@ -617,7 +645,6 @@ public void shutdown() { } } - protected void maybeMoveWriteTime() { if (currWriteTimeMs < formatTimeMs(System.currentTimeMillis())) { currWriteTimeMs = formatTimeMs(System.currentTimeMillis()); @@ -805,7 +832,7 @@ public boolean enqueue(int queueId) { return false; } - public boolean doEnqueue(long offsetPy, int sizePy, long delayedTime, MessageExt messageExt) { + public boolean doEnqueue(long offsetPy, int sizePy, long delayedTime, MessageExt messageExt, boolean isFromTimeline) { LOGGER.debug("Do enqueue [{}] [{}]", new Timestamp(delayedTime), messageExt); //copy the value first, avoid concurrent problem long tmpWriteTimeMs = currWriteTimeMs; @@ -823,6 +850,9 @@ public boolean doEnqueue(long offsetPy, int sizePy, long delayedTime, MessageExt boolean isDelete = messageExt.getProperty(TIMER_DELETE_UNIQUE_KEY) != null; if (isDelete) { magic = magic | MAGIC_DELETE; + if (!isFromTimeline) { + recallToTimeline(delayedTime, offsetPy, sizePy, messageExt); + } } String realTopic = messageExt.getProperty(MessageConst.PROPERTY_REAL_TOPIC); Slot slot = timerWheel.getSlot(delayedTime); @@ -959,7 +989,7 @@ public void checkDequeueLatch(CountDownLatch latch, long delayedTime) throws Exc LOGGER.info("Not Running dequeue, skip checkDequeueLatch for delayedTime:{}", delayedTime); break; } - + if (dequeuePutQueue.size() > 0 || !checkStateForGetMessages(AbstractStateService.WAITING) || !checkStateForPutMessages(AbstractStateService.WAITING)) { @@ -1125,14 +1155,7 @@ private List> splitIntoLists(List origin) { private MessageExt getMessageByCommitOffset(long offsetPy, int sizePy) { for (int i = 0; i < 3; i++) { - MessageExt msgExt = null; - bufferLocal.get().position(0); - bufferLocal.get().limit(sizePy); - boolean res = messageStore.getData(offsetPy, sizePy, bufferLocal.get()); - if (res) { - bufferLocal.get().flip(); - msgExt = MessageDecoder.decode(bufferLocal.get(), true, false, false); - } + MessageExt msgExt = StoreUtil.getMessage(offsetPy, sizePy, messageStore, bufferLocal.get()); if (null == msgExt) { LOGGER.warn("Fail to read msg from commitLog offsetPy:{} sizePy:{}", offsetPy, sizePy); } else { @@ -1387,7 +1410,10 @@ public void run() { TimerMessageStore.LOGGER.info(this.getServiceName() + " service start"); while (!this.isStopped()) { try { - if (!TimerMessageStore.this.enqueue(0)) { + if (storeConfig.isTimerRocksDBEnable() && !storeConfig.isTimerRocksDBStopScan()) { + LOGGER.info("now timer use rocksdb to driver, so will not enqueue in timer wheel"); + waitForRunning(10 * 1000L); + } else if (!TimerMessageStore.this.enqueue(0)) { waitForRunning(100L * precisionMs / 1000); } } catch (Throwable e) { @@ -1451,7 +1477,7 @@ protected void putMessageToTimerWheel(TimerRequest req) { dequeuePutQueue.put(req); } else { boolean doEnqueueRes = doEnqueue( - req.getOffsetPy(), req.getSizePy(), req.getDelayTime(), req.getMsg()); + req.getOffsetPy(), req.getSizePy(), req.getDelayTime(), req.getMsg(), false); req.idempotentRelease(doEnqueueRes || storeConfig.isTimerSkipUnknownError()); } perfCounterTicks.endTick(ENQUEUE_PUT); @@ -1478,7 +1504,13 @@ protected void fetchAndPutTimerRequest() throws Exception { CountDownLatch latch = new CountDownLatch(trs.size()); for (TimerRequest req : trs) { req.setLatch(latch); - this.putMessageToTimerWheel(req); + if (storeConfig.isTimerWheelSnapshotFlush()) { + synchronized (lockWhenFlush) { + this.putMessageToTimerWheel(req); + } + } else { + this.putMessageToTimerWheel(req); + } } checkDequeueLatch(latch, -1); boolean allSuccess = trs.stream().allMatch(TimerRequest::isSucc); @@ -1790,7 +1822,8 @@ public boolean needDelete(int magic) { public class TimerFlushService extends ServiceThread { private final SimpleDateFormat sdf = new SimpleDateFormat("MM-dd HH:mm:ss"); - @Override public String getServiceName() { + @Override + public String getServiceName() { String brokerIdentifier = ""; if (TimerMessageStore.this.messageStore instanceof DefaultMessageStore && ((DefaultMessageStore) TimerMessageStore.this.messageStore).getBrokerConfig().isInBrokerContainer()) { brokerIdentifier = ((DefaultMessageStore) TimerMessageStore.this.messageStore).getBrokerConfig().getIdentifier(); @@ -1805,33 +1838,55 @@ private String format(long time) { @Override public void run() { TimerMessageStore.LOGGER.info(this.getServiceName() + " service start"); - long start = System.currentTimeMillis(); while (!this.isStopped()) { try { - prepareTimerCheckPoint(); - timerLog.getMappedFileQueue().flush(0); - timerWheel.flush(); - timerCheckpoint.flush(); - if (System.currentTimeMillis() - start > storeConfig.getTimerProgressLogIntervalMs()) { - start = System.currentTimeMillis(); - long tmpQueueOffset = currQueueOffset; - ConsumeQueueInterface cq = messageStore.getConsumeQueue(TIMER_TOPIC, 0); - long maxOffsetInQueue = cq == null ? 0 : cq.getMaxOffsetInQueue(); - TimerMessageStore.LOGGER.info("[{}]Timer progress-check commitRead:[{}] currRead:[{}] currWrite:[{}] readBehind:{} currReadOffset:{} offsetBehind:{} behindMaster:{} " + - "enqPutQueue:{} deqGetQueue:{} deqPutQueue:{} allCongestNum:{} enqExpiredStoreTime:{}", - storeConfig.getBrokerRole(), - format(commitReadTimeMs), format(currReadTimeMs), format(currWriteTimeMs), getDequeueBehind(), - tmpQueueOffset, maxOffsetInQueue - tmpQueueOffset, timerCheckpoint.getMasterTimerQueueOffset() - tmpQueueOffset, - enqueuePutQueue.size(), dequeueGetQueue.size(), dequeuePutQueue.size(), getAllCongestNum(), format(lastEnqueueButExpiredStoreTime)); - } - timerMetrics.persist(); - waitForRunning(storeConfig.getTimerFlushIntervalMs()); + this.flush(); } catch (Throwable e) { TimerMessageStore.LOGGER.error("Error occurred in " + getServiceName(), e); } + try { + waitForRunning(storeConfig.getTimerFlushIntervalMs()); + } catch (Throwable e) { + // ignore interrupt + } } TimerMessageStore.LOGGER.info(this.getServiceName() + " service end"); } + + long start = System.currentTimeMillis(); + long lastSnapshotTime = System.currentTimeMillis(); + + public void flush() throws IOException { + if (storeConfig.isTimerWheelSnapshotFlush()) { + synchronized (lockWhenFlush) { + prepareTimerCheckPoint(); + timerLog.getMappedFileQueue().flush(0); + if (System.currentTimeMillis() - lastSnapshotTime > storeConfig.getTimerWheelSnapshotIntervalMs()) { + lastSnapshotTime = System.currentTimeMillis(); + timerWheel.backup(timerLog.getMappedFileQueue().getFlushedWhere()); + } + timerCheckpoint.flush(); + } + } else { + prepareTimerCheckPoint(); + timerLog.getMappedFileQueue().flush(0); + timerWheel.flush(); + timerCheckpoint.flush(); + } + if (System.currentTimeMillis() - start > storeConfig.getTimerProgressLogIntervalMs()) { + start = System.currentTimeMillis(); + long tmpQueueOffset = currQueueOffset; + ConsumeQueueInterface cq = messageStore.getConsumeQueue(TIMER_TOPIC, 0); + long maxOffsetInQueue = cq == null ? 0 : cq.getMaxOffsetInQueue(); + TimerMessageStore.LOGGER.info("[{}]Timer progress-check commitRead:[{}] currRead:[{}] currWrite:[{}] readBehind:{} currReadOffset:{} offsetBehind:{} behindMaster:{} " + + "enqPutQueue:{} deqGetQueue:{} deqPutQueue:{} allCongestNum:{} enqExpiredStoreTime:{}", + storeConfig.getBrokerRole(), + format(commitReadTimeMs), format(currReadTimeMs), format(currWriteTimeMs), getDequeueBehind(), + tmpQueueOffset, maxOffsetInQueue - tmpQueueOffset, timerCheckpoint.getMasterTimerQueueOffset() - tmpQueueOffset, + enqueuePutQueue.size(), dequeueGetQueue.size(), dequeuePutQueue.size(), getAllCongestNum(), format(lastEnqueueButExpiredStoreTime)); + } + timerMetrics.persist(); + } } public long getAllCongestNum() { @@ -2023,4 +2078,51 @@ public TimerCheckpoint getTimerCheckpoint() { public static String buildDeleteKey(String realTopic, String uniqueKey) { return realTopic + "+" + uniqueKey; } + + private void recallToTimeline(long delayTime, long offsetPy, int sizePy, MessageExt messageExt) { + if (!storeConfig.isTimerRecallToTimelineEnable() || !storeConfig.isTimerRocksDBEnable()) { + return; + } + if (delayTime < 0L || offsetPy < 0L || sizePy <= 0 || null == messageExt) { + LOGGER.error("recallToTimeline param error, delayTime: {}, offsetPy: {}, sizePy: {}, messageExt: {}", delayTime, offsetPy, sizePy, messageExt); + return; + } + if (null == messageStore.getTimerRocksDBStore() || null == messageStore.getTimerRocksDBStore().getTimeline()) { + LOGGER.error("recallToTimeline error, timerRocksDBStore is null or timeline is null"); + return; + } + try { + messageStore.getTimerRocksDBStore().getTimeline().putDeleteRecord(delayTime, messageExt.getMsgId(), offsetPy, sizePy, messageExt.getQueueOffset(), messageExt); + } catch (Exception e) { + LOGGER.error("recallToTimeline error: {}", e.getMessage()); + } + } + + public boolean restart() { + try { + if (this.state != RUNNING) { + LOGGER.info("TimerMessageStore restart operation just support for running state"); + return false; + } + this.storeConfig.setTimerRocksDBStopScan(true); + if (this.state == RUNNING && !this.storeConfig.isTimerStopEnqueue()) { + LOGGER.info("restart TimerMessageStore has been running"); + return true; + } + long commitOffsetRocksDB = this.messageStore.getTimerRocksDBStore().getCommitOffsetInRocksDB(); + long commitOffsetFile = this.messageStore.getTimerMessageStore().getCommitQueueOffset(); + long maxCommitOffset = Math.max(commitOffsetFile, commitOffsetRocksDB); + currQueueOffset = maxCommitOffset; + this.storeConfig.setTimerStopEnqueue(false); + LOGGER.info("TimerMessageStore restart commitOffsetRocksDB: {}, commitOffsetFile: {}, currQueueOffset: {}", commitOffsetRocksDB, commitOffsetFile, currQueueOffset); + return true; + } catch (Exception e) { + LOGGER.error("TimerMessageStore restart error: {}", e.getMessage()); + return false; + } + } + + public TimerFlushService getTimerFlushService() { + return timerFlushService; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java index ba7240414a0..338a62252f2 100644 --- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java +++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerMetrics.java @@ -16,9 +16,21 @@ */ package org.apache.rocketmq.store.timer; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; +import org.apache.rocketmq.common.ConfigManager; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.topic.TopicValidator; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.remoting.protocol.DataVersion; +import org.apache.rocketmq.remoting.protocol.RemotingSerializable; + import java.io.File; +import java.io.IOException; import java.io.RandomAccessFile; import java.io.StringWriter; import java.io.Writer; @@ -36,16 +48,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import org.apache.rocketmq.common.ConfigManager; -import org.apache.rocketmq.common.MixAll; -import org.apache.rocketmq.common.constant.LoggerName; -import org.apache.rocketmq.common.message.MessageConst; -import org.apache.rocketmq.common.message.MessageExt; -import org.apache.rocketmq.common.topic.TopicValidator; -import org.apache.rocketmq.logging.org.slf4j.Logger; -import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; -import org.apache.rocketmq.remoting.protocol.DataVersion; -import org.apache.rocketmq.remoting.protocol.RemotingSerializable; public class TimerMetrics extends ConfigManager { private static final Logger log = LoggerFactory.getLogger(LoggerName.BROKER_LOGGER_NAME); @@ -135,11 +137,11 @@ public Map getTimingCount() { return timingCount; } - protected void write0(Writer writer) { + protected void write0(Writer writer) throws IOException { TimerMetricsSerializeWrapper wrapper = new TimerMetricsSerializeWrapper(); wrapper.setTimingCount(timingCount); wrapper.setDataVersion(dataVersion); - JSON.writeJSONString(writer, wrapper, SerializerFeature.BrowserCompatible); + writer.write(JSON.toJSONString(wrapper, JSONWriter.Feature.BrowserCompatible)); } @Override public String encode() { @@ -152,7 +154,7 @@ protected void write0(Writer writer) { @Override public void decode(String jsonString) { if (jsonString != null) { - TimerMetricsSerializeWrapper timerMetricsSerializeWrapper = TimerMetricsSerializeWrapper.fromJson(jsonString, TimerMetricsSerializeWrapper.class); + TimerMetricsSerializeWrapper timerMetricsSerializeWrapper = TimerMetricsSerializeWrapper.fromJson(jsonString, TimerMetricsSerializeWrapper.class); if (timerMetricsSerializeWrapper != null) { this.timingCount.putAll(timerMetricsSerializeWrapper.getTimingCount()); this.dataVersion.assignNewOne(timerMetricsSerializeWrapper.getDataVersion()); @@ -179,7 +181,7 @@ public void cleanMetrics(Set topics) { while (iterator.hasNext()) { Map.Entry entry = iterator.next(); final String topic = entry.getKey(); - if (topic.startsWith(TopicValidator.SYSTEM_TOPIC_PREFIX) || topic.startsWith(MixAll.LMQ_PREFIX)) { + if (topic.startsWith(TopicValidator.SYSTEM_TOPIC_PREFIX) || topic.startsWith(MixAll.LMQ_PREFIX)) { continue; } if (topics.contains(topic)) { diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/TimerWheel.java b/store/src/main/java/org/apache/rocketmq/store/timer/TimerWheel.java index 6c7d1645925..2d5ce382012 100644 --- a/store/src/main/java/org/apache/rocketmq/store/timer/TimerWheel.java +++ b/store/src/main/java/org/apache/rocketmq/store/timer/TimerWheel.java @@ -16,6 +16,12 @@ */ package org.apache.rocketmq.store.timer; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; +import java.util.ArrayList; +import java.util.List; +import org.apache.rocketmq.common.MixAll; import org.apache.rocketmq.common.UtilAll; import org.apache.rocketmq.common.constant.LoggerName; import org.apache.rocketmq.logging.org.slf4j.Logger; @@ -32,13 +38,14 @@ public class TimerWheel { private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + public static final String TIMER_WHEEL_FILE_NAME = "timerwheel"; public static final int BLANK = -1, IGNORE = -2; public final int slotsTotal; public final int precisionMs; - private String fileName; + private final String fileName; + private final MappedByteBuffer mappedByteBuffer; private final RandomAccessFile randomAccessFile; private final FileChannel fileChannel; - private final MappedByteBuffer mappedByteBuffer; private final ByteBuffer byteBuffer; private final ThreadLocal localBuffer = new ThreadLocal() { @Override @@ -48,33 +55,46 @@ protected ByteBuffer initialValue() { }; private final int wheelLength; + private long snapOffset; + public TimerWheel(String fileName, int slotsTotal, int precisionMs) throws IOException { + this(fileName, slotsTotal, precisionMs, -1); + } + public TimerWheel(String fileName, int slotsTotal, int precisionMs, long snapOffset) throws IOException { this.slotsTotal = slotsTotal; this.precisionMs = precisionMs; this.fileName = fileName; this.wheelLength = this.slotsTotal * 2 * Slot.SIZE; + this.snapOffset = snapOffset; - File file = new File(fileName); + String finalFileName = selectSnapshotByFlag(snapOffset); + File file = new File(finalFileName); UtilAll.ensureDirOK(file.getParent()); try { - randomAccessFile = new RandomAccessFile(this.fileName, "rw"); + randomAccessFile = new RandomAccessFile(finalFileName, "rw"); if (file.exists() && randomAccessFile.length() != 0 && randomAccessFile.length() != wheelLength) { throw new RuntimeException(String.format("Timer wheel length:%d != expected:%s", randomAccessFile.length(), wheelLength)); } randomAccessFile.setLength(wheelLength); - fileChannel = randomAccessFile.getChannel(); - mappedByteBuffer = fileChannel.map(FileChannel.MapMode.READ_WRITE, 0, wheelLength); - assert wheelLength == mappedByteBuffer.remaining(); + if (snapOffset < 0) { + fileChannel = randomAccessFile.getChannel(); + mappedByteBuffer = fileChannel.map(FileChannel.MapMode.READ_WRITE, 0, wheelLength); + assert wheelLength == mappedByteBuffer.remaining(); + } else { + fileChannel = null; + mappedByteBuffer = null; + randomAccessFile.close(); + } this.byteBuffer = ByteBuffer.allocateDirect(wheelLength); - this.byteBuffer.put(mappedByteBuffer); + this.byteBuffer.put(Files.readAllBytes(file.toPath())); } catch (FileNotFoundException e) { - log.error("create file channel " + this.fileName + " Failed. ", e); + log.error("create file channel " + finalFileName + " Failed. ", e); throw e; } catch (IOException e) { - log.error("map file " + this.fileName + " Failed. ", e); + log.error("map file " + finalFileName + " Failed. ", e); throw e; } } @@ -105,6 +125,9 @@ public void shutdown(boolean flush) { } public void flush() { + if (mappedByteBuffer == null) { + return; + } ByteBuffer bf = localBuffer.get(); bf.position(0); bf.limit(wheelLength); @@ -118,6 +141,131 @@ public void flush() { this.mappedByteBuffer.force(); } + /** + * Perform backup operation. + *

+ * Select snapshot file based on the provided flag, write current buffer content to a temporary file, + * then rename the temporary file to the formal snapshot file. If rename fails, delete the temporary file. + * Finally clean up expired snapshot files. + * + * @param flushWhere Flag used to select snapshot file. + * @throws IOException If I/O error occurs during backup process. + */ + public void backup(long flushWhere) throws IOException { + // Get current local buffer and position it to the beginning + ByteBuffer bf = localBuffer.get(); + bf.position(0); + bf.limit(wheelLength); + + // Select snapshot file name based on flag + String fileName = selectSnapshotByFlag(flushWhere); + File bakFile = new File(fileName); + // Create or open temporary file for snapshot, ready for writing + File tmpFile = new File(fileName + ".tmp"); + // Delete if exists first + Files.deleteIfExists(tmpFile.toPath()); + try (RandomAccessFile randomAccessFile = new RandomAccessFile(tmpFile, "rw")) { + try (FileChannel fileChannel = randomAccessFile.getChannel()) { + fileChannel.write(bf); + fileChannel.force(true); + } + } + + if (tmpFile.exists()) { + // atomic move + Files.move(tmpFile.toPath(), bakFile.toPath(), StandardCopyOption.ATOMIC_MOVE); + + // sync the directory, ensure that the bak file is visible + MixAll.fsyncDirectory(Paths.get(bakFile.getParent())); + } + cleanExpiredSnapshot(); // Clean up expired snapshot files + } + + /** + * Select snapshot file name based on flag. + * + * @param flag Flag used to select or identify snapshot file. + * @return Name of the snapshot file. + */ + private String selectSnapshotByFlag(long flag) { + if (flag < 0) { + return this.fileName; // If flag is less than 0, return default file name + } + return this.fileName + "." + flag; // Otherwise, return file name with flag suffix + } + + /** + * Clean up expired snapshot files. + *

+ * This method will find and delete all snapshot files with flags smaller than the specified value + * under the current file name, keeping the two snapshot files with the largest flags. + */ + public void cleanExpiredSnapshot() { + File dir = new File(this.fileName).getParentFile(); + File[] files = dir.listFiles(); + if (files == null) { + return; + } + + // Collect all snapshot files and their flags + List snapshotFiles = new ArrayList<>(); + for (File file : files) { + String fileName = file.getName(); + if (fileName.startsWith(TIMER_WHEEL_FILE_NAME + ".")) { + long flag = UtilAll.asLong(fileName.substring(TIMER_WHEEL_FILE_NAME.length() + 1), -1); + if (flag >= 0) { + snapshotFiles.add(new FileWithFlag(file, flag)); + } + } + } + + // Sort by flag in descending order + snapshotFiles.sort((a, b) -> Long.compare(b.flag, a.flag)); + + // Delete all files except the first two + for (int i = 2; i < snapshotFiles.size(); i++) { + UtilAll.deleteFile(snapshotFiles.get(i).file); + } + } + + /** + * Get the maximum flag from existing snapshot files. + * + * @return The maximum flag value, or -1 if no snapshot files exist + */ + public static long getMaxSnapshotFlag(String timerWheelPath) { + File dir = new File(timerWheelPath).getParentFile(); + File[] files = dir.listFiles(); + if (files == null) { + return -1; + } + + long maxFlag = -1; + for (File file : files) { + String fileName = file.getName(); + if (fileName.startsWith(TIMER_WHEEL_FILE_NAME + ".")) { + long flag = UtilAll.asLong(fileName.substring(TIMER_WHEEL_FILE_NAME.length() + 1), -1); + if (flag > maxFlag) { + maxFlag = flag; + } + } + } + return maxFlag; + } + + /** + * Wrapper class for file and flag + */ + private static class FileWithFlag { + final File file; + final long flag; + + FileWithFlag(File file, long flag) { + this.file = file; + this.flag = flag; + } + } + public Slot getSlot(long timeMs) { Slot slot = getRawSlot(timeMs); if (slot.timeMs != timeMs / precisionMs * precisionMs) { @@ -145,6 +293,7 @@ public void putSlot(long timeMs, long firstPos, long lastPos) { localBuffer.get().putLong(firstPos); localBuffer.get().putLong(lastPos); } + public void putSlot(long timeMs, long firstPos, long lastPos, int num, int magic) { localBuffer.get().position(getSlotIndex(timeMs) * Slot.SIZE); localBuffer.get().putLong(timeMs / precisionMs); @@ -212,4 +361,8 @@ public long getAllNum(long timeStartMs) { } return allNum; } + + public String getFileName() { + return fileName; + } } diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/Timeline.java b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/Timeline.java new file mode 100644 index 00000000000..922786bde92 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/Timeline.java @@ -0,0 +1,448 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.timer.rocksdb; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import com.conversantmedia.util.concurrent.DisruptorBlockingQueue; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.math.NumberUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.timer.TimerMessageStore; +import org.apache.rocketmq.store.timer.TimerMetrics; +import static org.apache.rocketmq.common.message.MessageConst.PROPERTY_TIMER_ROLL_LABEL; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TIMER_COLUMN_FAMILY; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_DELETE; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_PUT; +import static org.apache.rocketmq.store.timer.rocksdb.TimerRocksDBRecord.TIMER_ROCKSDB_UPDATE; + +public class Timeline { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String DELETE_KEY_SPLIT = "+"; + private static final int ORIGIN_CAPACITY = 100000; + private static final int BATCH_SIZE = 1000, MAX_BATCH_SIZE_FROM_ROCKSDB = 8000; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + private final AtomicLong commitOffset = new AtomicLong(0); + private final MessageStore messageStore; + private final MessageStoreConfig storeConfig; + private final TimerMessageStore timerMessageStore; + private final MessageRocksDBStorage messageRocksDBStorage; + private final TimerMessageRocksDBStore timerMessageRocksDBStore; + private final long precisionMs; + private final TimerMetrics timerMetrics; + + private TimelineIndexBuildService timelineIndexBuildService; + private TimelineForwardService timelineForwardService; + private TimelineRollService timelineRollService; + private TimelineDeleteService timelineDeleteService; + private BlockingQueue originTimerMsgQueue; + + public Timeline(final MessageStore messageStore, final MessageRocksDBStorage messageRocksDBStorage, final TimerMessageRocksDBStore timerMessageRocksDBStore, final TimerMetrics timerMetrics) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.timerMessageStore = messageStore.getTimerMessageStore(); + this.messageRocksDBStorage = messageRocksDBStorage; + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + this.precisionMs = timerMessageRocksDBStore.precisionMs; + this.timerMetrics = timerMetrics; + initService(); + } + + private void initService() { + this.timelineIndexBuildService = new TimelineIndexBuildService(); + this.timelineForwardService = new TimelineForwardService(); + if (storeConfig.isTimerEnableDisruptor()) { + this.originTimerMsgQueue = new DisruptorBlockingQueue<>(ORIGIN_CAPACITY); + } else { + this.originTimerMsgQueue = new LinkedBlockingDeque<>(ORIGIN_CAPACITY); + } + this.timelineRollService = new TimelineRollService(); + this.timelineDeleteService = new TimelineDeleteService(); + } + + public void start() { + if (this.state == RUNNING) { + return; + } + this.commitOffset.set(this.timerMessageRocksDBStore.getReadOffset().get()); + this.timelineIndexBuildService.start(); + this.timelineForwardService.start(); + this.timelineRollService.start(); + this.timelineDeleteService.start(); + this.state = RUNNING; + log.info("Timeline start success, start commitOffset: {}", this.commitOffset.get()); + } + + public void shutDown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.timelineIndexBuildService) { + this.timelineIndexBuildService.shutdown(); + } + if (null != this.timelineForwardService) { + this.timelineForwardService.shutdown(); + } + if (null != this.timelineRollService) { + this.timelineRollService.shutdown(); + } + if (null != this.timelineDeleteService) { + this.timelineDeleteService.shutdown(); + } + this.state = SHUTDOWN; + log.info("Timeline shutdown success"); + } + + public void putRecord(TimerRocksDBRecord timerRecord) throws InterruptedException { + if (null == timerRecord) { + return; + } + while (!originTimerMsgQueue.offer(timerRecord, 3, TimeUnit.SECONDS)) { + if (null != timerRecord.getMessageExt()) { + logError.error("Timeline originTimerMsgQueue put record failed, topic: {}, uniqKey: {}", timerRecord.getMessageExt().getTopic(), timerRecord.getUniqKey()); + } else { + logError.error("Timeline originTimerMsgQueue put record failed, uniqKey: {}", timerRecord.getUniqKey()); + } + } + } + + public void putDeleteRecord(long delayTime, String uniqKey, long offsetPy, int sizePy, long queueOffset, MessageExt messageExt) throws InterruptedException { + if (delayTime <= 0L || StringUtils.isEmpty(uniqKey) || offsetPy < 0L || sizePy < 0 || queueOffset < 0L || null == messageExt) { + log.info("Timeline putDeleteRecord param error, delayTime: {}, uniqKey: {}, offsetPy: {}, sizePy: {}, queueOffset: {}, messageExt: {}", delayTime, uniqKey, offsetPy, sizePy, queueOffset, messageExt); + return; + } + while (!originTimerMsgQueue.offer(new TimerRocksDBRecord(delayTime, uniqKey, offsetPy, sizePy, queueOffset, messageExt), 3, TimeUnit.SECONDS)) { + log.error("Timeline putDeleteRecord originTimerMsgQueue put delete record failed, uniqKey: {}", uniqKey); + } + } + + public void addMetric(MessageExt msg, int value) { + if (null == msg || null == msg.getProperty(MessageConst.PROPERTY_REAL_TOPIC)) { + return; + } + if (null != msg.getProperty(MessageConst.PROPERTY_TIMER_ENQUEUE_MS) && NumberUtils.toLong(msg.getProperty(MessageConst.PROPERTY_TIMER_ENQUEUE_MS)) == Long.MAX_VALUE) { + return; + } + timerMetrics.addAndGet(msg, value); + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (Timeline.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) Timeline.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private void recallToTimeWheel(TimerRocksDBRecord tr) { + if (!messageStore.getMessageStoreConfig().isTimerRecallToTimeWheelEnable()) { + return; + } + if (null == tr || null == tr.getMessageExt()) { + return; + } + try { + timerMessageStore.doEnqueue(tr.getOffsetPy(), tr.getSizePy(), tr.getDelayTime(), tr.getMessageExt(), true); + } catch (Exception e) { + log.error("Timeline recallToTimeWheel error: {}", e.getMessage()); + } + } + + private boolean scanRecordsToQueue(long checkpoint, long checkRange, BlockingQueue> queue) { + if (checkpoint <= 0L || checkRange <= 0L || null == queue) { + logError.error("Timeline scanRecordsToQueue param error, checkpoint: {}, checkRange: {}, queue: {}", checkpoint, checkRange, queue); + return false; + } + if (storeConfig.isTimerStopDequeue()) { + logError.info("Timeline scanRecordsToQueue storeConfig isTimerStopDequeue is true, stop to scan records to queue"); + return false; + } + long count = 0; + byte[] lastKey = null; + while (true) { + try { + List trs = messageRocksDBStorage.scanRecordsForTimer(TIMER_COLUMN_FAMILY, checkpoint, checkpoint + checkRange, MAX_BATCH_SIZE_FROM_ROCKSDB, lastKey); + if (null == trs || CollectionUtils.isEmpty(trs)) { + break; + } + count += trs.size(); + boolean hasMoreData = trs.size() >= MAX_BATCH_SIZE_FROM_ROCKSDB; + lastKey = hasMoreData ? trs.get(trs.size() - 1).getKeyBytes() : null; + if (null == lastKey) { + trs.get(trs.size() - 1).setCheckPoint(checkpoint + checkRange); + } + while (!queue.offer(trs, 3, TimeUnit.SECONDS)) { + log.debug("Timeline scanRecordsToQueue offer to queue error, queue size: {}, records size: {}", queue.size(), trs.size()); + } + if (!hasMoreData) { + break; + } + } catch (Exception e) { + logError.error("Timeline scanRecordsToQueue error: {}", e.getMessage()); + return false; + } + } + log.info("Timeline scan records from rocksdb, checkpoint: {}, records size: {}", checkpoint, count); + return true; + } + + public class TimelineIndexBuildService extends ServiceThread { + private final Logger log = Timeline.log; + private List trs; + + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + trs = new ArrayList<>(BATCH_SIZE); + while (!this.isStopped() || !originTimerMsgQueue.isEmpty()) { + try { + buildTimelineIndex(); + } catch (Exception e) { + logError.error("Timeline error occurred in: {}, error: {}", getServiceName(), e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void buildTimelineIndex() throws InterruptedException { + pollTimerMessageRecords(); + if (CollectionUtils.isEmpty(trs)) { + return; + } + List cudlist = new ArrayList<>(); + for (TimerRocksDBRecord tr : trs) { + try { + MessageExt messageExt = tr.getMessageExt(); + if (null == messageExt) { + logError.error("Timeline TimelineIndexBuildService buildTimelineIndex error, messageExt is null"); + continue; + } + String timerDelUniqKey = messageExt.getProperty(MessageConst.PROPERTY_TIMER_DEL_UNIQKEY); + if (!StringUtils.isEmpty(timerDelUniqKey)) { + tr.setUniqKey(extractUniqKey(timerDelUniqKey)); + tr.setActionFlag(TIMER_ROCKSDB_DELETE); + cudlist.add(tr); + addMetric(messageExt, -1); + recallToTimeWheel(tr); + } else if (TimerMessageRocksDBStore.isExpired(tr.getDelayTime())) { + timerMessageRocksDBStore.putRealTopicMessage(tr.getMessageExt()); + } else if (!StringUtils.isEmpty(messageExt.getProperty(PROPERTY_TIMER_ROLL_LABEL))) { + tr.setActionFlag(TIMER_ROCKSDB_UPDATE); + cudlist.add(tr); + } else { + tr.setActionFlag(TIMER_ROCKSDB_PUT); + cudlist.add(tr); + addMetric(messageExt, 1); + } + } catch (Exception e) { + logError.error("Timeline TimelineIndexBuildService buildTimelineIndex deal trs error", e); + } + } + if (!CollectionUtils.isEmpty(cudlist)) { + messageRocksDBStorage.writeRecordsForTimer(TIMER_COLUMN_FAMILY, cudlist); + } + synCommitOffset(trs); + trs.clear(); + } + + private String extractUniqKey(String deleteKey) throws IllegalArgumentException { + if (StringUtils.isEmpty(deleteKey)) { + throw new IllegalArgumentException("deleteKey is empty"); + } + int separatorIndex = deleteKey.indexOf(DELETE_KEY_SPLIT); + if (separatorIndex == -1) { + throw new IllegalArgumentException("Invalid deleteKey format"); + } + return deleteKey.substring(separatorIndex + 1); + } + + private void pollTimerMessageRecords() throws InterruptedException { + TimerRocksDBRecord firstReq = originTimerMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null != firstReq) { + trs.add(firstReq); + while (true) { + TimerRocksDBRecord tmpReq = originTimerMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null == tmpReq) { + break; + } + trs.add(tmpReq); + if (trs.size() >= BATCH_SIZE) { + break; + } + } + } + } + + private void synCommitOffset(List trs) { + if (CollectionUtils.isEmpty(trs)) { + return; + } + long lastQueueOffset = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + long queueOffset = trs.get(trs.size() - 1).getQueueOffset() + 1L; + if (queueOffset > lastQueueOffset) { + commitOffset.set(queueOffset); + messageRocksDBStorage.writeCheckPointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT, commitOffset.get()); + } + } + } + + private class TimelineForwardService extends ServiceThread { + private final Logger log = Timeline.log; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + long checkpoint = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.TIMELINE_CHECK_POINT); + log.info(this.getServiceName() + " service start, checkpoint: {}", checkpoint); + while (!this.isStopped()) { + try { + if (!timelineForward(checkpoint, precisionMs)) { + waitForRunning(100L); + } else { + checkpoint += precisionMs; + } + } catch (Exception e) { + logError.error("Timeline error occurred in " + getServiceName(), e); + } + } + log.info(this.getServiceName() + " service end"); + } + + private boolean timelineForward(long checkpoint, long checkRange) { + if (checkpoint > System.currentTimeMillis()) { + return false; + } + try { + long begin = System.currentTimeMillis(); + boolean result = scanRecordsToQueue(checkpoint, checkRange, timerMessageRocksDBStore.getExpiredMessageQueue()); + log.info("Timeline TimelineForwardService timelineForward scanRecordsToQueue end, result: {}, checkpoint: {}, checkRange: {}, checkDelay: {}, cost: {}", result, checkpoint, checkRange, System.currentTimeMillis() - checkpoint, System.currentTimeMillis() - begin); + return result; + } catch (Exception e) { + logError.error("Timeline TimelineForwardService timelineForward error: {}", e.getMessage()); + return false; + } + } + } + + private class TimelineRollService extends ServiceThread { + private final Logger log = Timeline.log; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped()) { + int rollIntervalHour = 1; + int rollRangeHour = 2; + try { + if (storeConfig.getTimerRocksDBRollIntervalHours() > 0) { + rollIntervalHour = storeConfig.getTimerRocksDBRollIntervalHours(); + } + if (storeConfig.getTimerRocksDBRollRangeHours() > 0) { + rollRangeHour = storeConfig.getTimerRocksDBRollRangeHours(); + } + this.waitForRunning(TimeUnit.HOURS.toMillis(rollIntervalHour)); + if (stopped) { + log.info(this.getServiceName() + " service end"); + return; + } + } catch (Exception e) { + logError.error("Timeline TimelineRollService wait error: {}", e.getMessage()); + } + long rollCheckpoint = System.currentTimeMillis(); + try { + log.info("Timeline TimelineRollService start roll rollCheckpoint: {}", rollCheckpoint); + while (!scanRecordsToQueue(rollCheckpoint + TimeUnit.HOURS.toMillis(rollRangeHour), + TimeUnit.SECONDS.toMillis(storeConfig.getTimerMaxDelaySec()), + timerMessageRocksDBStore.getRollMessageQueue())) { + logError.error("Timeline TimelineRollService scanRecordsToQueue error."); + Thread.sleep(200); + } + log.info("Timeline TimelineRollService roll records success, lastRollTime: {}, rollCheckpoint: {}, cost: {}", rollCheckpoint, rollCheckpoint, System.currentTimeMillis() - rollCheckpoint); + } catch (Exception e) { + logError.error("Timeline TimelineRollService failed error: {}", e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + } + + private class TimelineDeleteService extends ServiceThread { + private final Logger log = Timeline.log; + private long lastDeleteCheckPoint = 0L; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped()) { + try { + this.waitForRunning(TimeUnit.MINUTES.toMillis(30)); + if (stopped) { + log.info(this.getServiceName() + " service end"); + return; + } + } catch (Exception e) { + logError.error("Timeline TimelineDeleteService wait error: {}", e.getMessage()); + } + try { + long checkpoint = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.TIMELINE_CHECK_POINT); + if (lastDeleteCheckPoint == checkpoint) { + continue; + } + messageRocksDBStorage.deleteRecordsForTimer(TIMER_COLUMN_FAMILY, checkpoint - TimeUnit.HOURS.toMillis(168), checkpoint - TimeUnit.MINUTES.toMillis(30)); + lastDeleteCheckPoint = checkpoint; + } catch (Exception e) { + logError.error("Timeline TimelineDeleteService delete failed, lastDeleteCheckPoint: {} error: {}", lastDeleteCheckPoint, e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerMessageRocksDBStore.java b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerMessageRocksDBStore.java new file mode 100644 index 00000000000..ec13971d922 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerMessageRocksDBStore.java @@ -0,0 +1,619 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.timer.rocksdb; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import com.conversantmedia.util.concurrent.DisruptorBlockingQueue; +import com.google.common.util.concurrent.RateLimiter; +import io.opentelemetry.api.common.Attributes; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.TopicFilterType; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageClientIDSetter; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.StoreUtil; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.metrics.DefaultStoreMetricsConstant; +import org.apache.rocketmq.store.metrics.DefaultStoreMetricsManager; +import org.apache.rocketmq.store.metrics.StoreMetricsManager; +import org.apache.rocketmq.store.queue.ConsumeQueueInterface; +import org.apache.rocketmq.store.queue.CqUnit; +import org.apache.rocketmq.store.queue.ReferredIterator; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import org.apache.rocketmq.store.timer.TimerMetrics; +import org.apache.rocketmq.store.util.PerfCounter; +import static org.apache.rocketmq.common.message.MessageConst.PROPERTY_TIMER_ROLL_LABEL; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TIMER_COLUMN_FAMILY; +import static org.apache.rocketmq.store.timer.TimerMessageStore.TIMER_TOPIC; + +public class TimerMessageRocksDBStore { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String SCAN_SYS_TOPIC = "scanSysTopic"; + private static final String SCAN_SYS_TOPIC_MISS = "scanSysTopicMiss"; + private static final String OUT_BIZ_MESSAGE = "outBizMsg"; + private static final String ROLL_LABEL = "R"; + private static final int PUT_OK = 0, PUT_NEED_RETRY = 1, PUT_NO_RETRY = 2; + private static final int MAX_GET_MSG_TIMES = 3, MAX_PUT_MSG_TIMES = 3; + private static final int TIME_UP_CAPACITY = 100, ROLL_CAPACITY = 50; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + private static long expirationThresholdMillis = 999L; + private final AtomicLong readOffset = new AtomicLong(0);//timerSysTopic read offset + private final MessageStore messageStore; + private final TimerMetrics timerMetrics; + private final MessageStoreConfig storeConfig; + private final BrokerStatsManager brokerStatsManager; + private final MessageRocksDBStorage messageRocksDBStorage; + private Timeline timeline; + private TimerSysTopicScanService timerSysTopicScanService; + private TimerMessageReputService expiredMessageReputService; + private TimerMessageReputService rollMessageReputService; + protected long precisionMs; + private BlockingQueue> expiredMessageQueue; + private BlockingQueue> rollMessageQueue; + protected final PerfCounter.Ticks perfCounterTicks = new PerfCounter.Ticks(log); + private Function escapeBridgeHook; + private ThreadLocal bufferLocal = null; + + public TimerMessageRocksDBStore(final MessageStore messageStore, final TimerMetrics timerMetrics, + final BrokerStatsManager brokerStatsManager) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.precisionMs = storeConfig.getTimerRocksDBPrecisionMs() < 100L ? 1000L : storeConfig.getTimerRocksDBPrecisionMs(); + expirationThresholdMillis = precisionMs - 1L; + this.messageRocksDBStorage = messageStore.getMessageRocksDBStorage(); + this.timerMetrics = timerMetrics; + this.brokerStatsManager = brokerStatsManager; + bufferLocal = ThreadLocal.withInitial(() -> ByteBuffer.allocate(storeConfig.getMaxMessageSize())); + } + + public synchronized boolean load() { + initService(); + boolean result = this.timerMetrics.load(); + log.info("TimerMessageRocksDBStore load result: {}", result); + return result; + } + + public synchronized void start() { + if (this.state == RUNNING) { + return; + } + long commitOffsetFile = null != this.messageStore.getTimerMessageStore() ? this.messageStore.getTimerMessageStore().getCommitQueueOffset() : 0L; + long commitOffsetRocksDB = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + long maxCommitOffset = Math.max(commitOffsetFile, commitOffsetRocksDB); + this.readOffset.set(maxCommitOffset); + this.expiredMessageReputService.start(); + this.rollMessageReputService.start(); + this.timeline.start(); + this.timerSysTopicScanService.start(); + this.state = RUNNING; + log.info("TimerMessageRocksDBStore start success, start commitOffsetFile: {}, commitOffsetRocksDB: {}, readOffset: {}", commitOffsetFile, commitOffsetRocksDB, this.readOffset.get()); + } + + public synchronized boolean restart() { + try { + this.storeConfig.setTimerStopEnqueue(true); + if (this.state == RUNNING && !this.storeConfig.isTimerRocksDBStopScan()) { + log.info("restart TimerMessageRocksDBStore has been running"); + return true; + } + if (this.state == RUNNING && this.storeConfig.isTimerRocksDBStopScan()) { + long commitOffsetFile = null != this.messageStore.getTimerMessageStore() ? this.messageStore.getTimerMessageStore().getCommitQueueOffset() : 0L; + long commitOffsetRocksDB = messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + long maxCommitOffset = Math.max(commitOffsetFile, commitOffsetRocksDB); + this.readOffset.set(maxCommitOffset); + log.info("restart TimerMessageRocksDBStore has been recover running, commitOffsetFile: {}, commitOffsetRocksDB: {}, readOffset: {}", commitOffsetFile, commitOffsetRocksDB, readOffset.get()); + } else { + this.load(); + this.start(); + } + this.storeConfig.setTimerRocksDBEnable(true); + this.storeConfig.setTimerRocksDBStopScan(false); + return true; + } catch (Exception e) { + logError.error("TimerMessageRocksDBStore restart error: {}", e.getMessage()); + return false; + } + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.timerSysTopicScanService) { + this.timerSysTopicScanService.shutdown(); + } + if (null != this.timeline) { + this.timeline.shutDown(); + } + if (null != this.expiredMessageReputService) { + this.expiredMessageReputService.shutdown(); + } + if (null != this.rollMessageReputService) { + this.rollMessageReputService.shutdown(); + } + this.state = SHUTDOWN; + log.info("TimerMessageRocksDBStore shutdown success"); + } + + public void putRealTopicMessage(MessageExt msg) { + if (null == msg) { + logError.error("putRealTopicMessage msg is null"); + return; + } + MessageExtBrokerInner messageExtBrokerInner = convertMessage(msg); + if (null == messageExtBrokerInner) { + logError.error("putRealTopicMessage error, messageExtBrokerInner is null"); + return; + } + doPut(messageExtBrokerInner); + } + + public MessageStore getMessageStore() { + return messageStore; + } + + public TimerMetrics getTimerMetrics() { + return timerMetrics; + } + + public BrokerStatsManager getBrokerStatsManager() { + return brokerStatsManager; + } + + public AtomicLong getReadOffset() { + return readOffset; + } + + public BlockingQueue> getExpiredMessageQueue() { + return expiredMessageQueue; + } + + public BlockingQueue> getRollMessageQueue() { + return rollMessageQueue; + } + + public long getCommitOffsetInRocksDB() { + if (null == messageRocksDBStorage || !storeConfig.isTimerRocksDBEnable()) { + return 0L; + } + return messageRocksDBStorage.getCheckpointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.SYS_TOPIC_SCAN_OFFSET_CHECK_POINT); + } + + public Timeline getTimeline() { + return timeline; + } + + private void initService() { + if (storeConfig.isTimerEnableDisruptor()) { + this.expiredMessageQueue = new DisruptorBlockingQueue<>(TIME_UP_CAPACITY); + this.rollMessageQueue = new DisruptorBlockingQueue<>(ROLL_CAPACITY); + } else { + this.expiredMessageQueue = new LinkedBlockingDeque<>(TIME_UP_CAPACITY); + this.rollMessageQueue = new LinkedBlockingDeque<>(ROLL_CAPACITY); + } + this.expiredMessageReputService = new TimerMessageReputService(expiredMessageQueue, storeConfig.getTimerRocksDBTimeExpiredMaxTps(), true); + this.rollMessageReputService = new TimerMessageReputService(rollMessageQueue, storeConfig.getTimerRocksDBRollMaxTps(), false); + this.timeline = new Timeline(messageStore, messageRocksDBStorage, this, timerMetrics); + this.timerSysTopicScanService = new TimerSysTopicScanService(); + } + + private MessageExtBrokerInner convertMessage(MessageExt msgExt) { + if (null == msgExt) { + logError.error("convertMessage msgExt is null"); + return null; + } + MessageExtBrokerInner msgInner = null; + try { + msgInner = new MessageExtBrokerInner(); + msgInner.setBody(msgExt.getBody()); + msgInner.setFlag(msgExt.getFlag()); + msgInner.setTags(msgExt.getTags()); + msgInner.setSysFlag(msgExt.getSysFlag()); + TopicFilterType topicFilterType = MessageExt.parseTopicFilterType(msgInner.getSysFlag()); + long tagsCodeValue = MessageExtBrokerInner.tagsString2tagsCode(topicFilterType, msgInner.getTags()); + msgInner.setTagsCode(tagsCodeValue); + msgInner.setBornTimestamp(msgExt.getBornTimestamp()); + msgInner.setBornHost(msgExt.getBornHost()); + msgInner.setStoreHost(msgExt.getStoreHost()); + msgInner.setReconsumeTimes(msgExt.getReconsumeTimes()); + msgInner.setWaitStoreMsgOK(false); + MessageAccessor.setProperties(msgInner, MessageAccessor.deepCopyProperties(msgExt.getProperties())); + if (isNeedRoll(msgInner)) { + msgInner.setTopic(msgExt.getTopic()); + msgInner.setQueueId(msgExt.getQueueId()); + msgInner.getProperties().put(PROPERTY_TIMER_ROLL_LABEL, ROLL_LABEL); + } else { + msgInner.setTopic(msgInner.getProperty(MessageConst.PROPERTY_REAL_TOPIC)); + msgInner.setQueueId(Integer.parseInt(msgInner.getProperty(MessageConst.PROPERTY_REAL_QUEUE_ID))); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_TOPIC); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_REAL_QUEUE_ID); + MessageAccessor.clearProperty(msgInner, MessageConst.PROPERTY_TIMER_ROLL_LABEL); + } + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + } catch (Exception e) { + logError.error("convertMessage error: {}", e.getMessage()); + return null; + } + return msgInner; + } + + private MessageExt getMessageByCommitOffset(long offsetPy, int sizePy) { + if (offsetPy < 0L || sizePy <= 0 || sizePy > storeConfig.getMaxMessageSize()) { + logError.error("getMessageByCommitOffset param error, offsetPy: {}, sizePy: {}, maxMsgSize: {}", offsetPy, sizePy, storeConfig.getMaxMessageSize()); + return null; + } + if (sizePy > bufferLocal.get().limit()) { + bufferLocal.remove(); + bufferLocal.set(ByteBuffer.allocate(sizePy)); + } + for (int i = 0; i < MAX_GET_MSG_TIMES; i++) { + MessageExt msgExt = StoreUtil.getMessage(offsetPy, sizePy, messageStore, bufferLocal.get()); + if (null == msgExt) { + log.warn("Fail to read msg from commitLog offsetPy: {} sizePy: {}", offsetPy, sizePy); + } else { + return msgExt; + } + } + return null; + } + + private boolean isNeedRoll(MessageExt messageExt) { + try { + String property = messageExt.getProperty(MessageConst.PROPERTY_TIMER_OUT_MS); + if (StringUtils.isEmpty(property)) { + return false; + } + if (!isExpired(Long.parseLong(property))) { + return true; + } + } catch (Exception e) { + logError.error("isNeedRoll error : {}", e.getMessage()); + } + return false; + } + + private Long getDelayTime(MessageExt msgExt) { + if (null == msgExt) { + logError.error("getDelayTime msgExt is null"); + return null; + } + String delayTimeStr = msgExt.getProperty(MessageConst.PROPERTY_TIMER_OUT_MS); + if (StringUtils.isEmpty(delayTimeStr)) { + logError.error("getDelayTime is empty, queueOffset: {}, delayTimeStr: {}", msgExt.getQueueOffset(), delayTimeStr); + return null; + } + try { + return Long.parseLong(delayTimeStr); + } catch (Exception e) { + logError.error("getDelayTime parse to long error : {}", e.getMessage()); + } + return null; + } + + private int doPut(MessageExtBrokerInner message) { + if (null == message) { + logError.error("doPut message is null"); + return PUT_NO_RETRY; + } + if (null != message.getProperty(MessageConst.PROPERTY_TIMER_DEL_UNIQKEY)) { + logError.warn("Trying do put delete timer msg:[{}]", message); + return PUT_NO_RETRY; + } + PutMessageResult putMessageResult = null; + if (escapeBridgeHook != null) { + putMessageResult = escapeBridgeHook.apply(message); + } else { + putMessageResult = messageStore.putMessage(message); + } + if (null != putMessageResult && null != putMessageResult.getPutMessageStatus()) { + switch (putMessageResult.getPutMessageStatus()) { + case PUT_OK: + if (null != brokerStatsManager) { + brokerStatsManager.incTopicPutNums(message.getTopic(), 1, 1); + if (null != putMessageResult.getAppendMessageResult()) { + brokerStatsManager.incTopicPutSize(message.getTopic(), putMessageResult.getAppendMessageResult().getWroteBytes()); + } + brokerStatsManager.incBrokerPutNums(message.getTopic(), 1); + } + return PUT_OK; + + case MESSAGE_ILLEGAL: + case PROPERTIES_SIZE_EXCEEDED: + case WHEEL_TIMER_NOT_ENABLE: + case WHEEL_TIMER_MSG_ILLEGAL: + return PUT_NO_RETRY; + + case SERVICE_NOT_AVAILABLE: + case FLUSH_DISK_TIMEOUT: + case FLUSH_SLAVE_TIMEOUT: + case OS_PAGE_CACHE_BUSY: + case CREATE_MAPPED_FILE_FAILED: + case SLAVE_NOT_AVAILABLE: + return PUT_NEED_RETRY; + + case UNKNOWN_ERROR: + default: + if (storeConfig.isTimerSkipUnknownError()) { + logError.warn("Skipping message due to unknown error, msg: {}", message); + return PUT_NO_RETRY; + } else { + return PUT_NEED_RETRY; + } + } + } + return PUT_NEED_RETRY; + } + + public static boolean isExpired(long delayedTime) { + return delayedTime <= System.currentTimeMillis() + expirationThresholdMillis; + } + + public void registerEscapeBridgeHook(Function escapeBridgeHook) { + this.escapeBridgeHook = escapeBridgeHook; + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (TimerMessageRocksDBStore.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore)TimerMessageRocksDBStore.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + private class TimerSysTopicScanService extends ServiceThread { + private final Logger log = TimerMessageRocksDBStore.log; + + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + long waitTime; + while (!this.isStopped()) { + try { + if (!storeConfig.isTimerRocksDBEnable() || storeConfig.isTimerRocksDBStopScan()) { + waitTime = TimeUnit.SECONDS.toMillis(10); + } else { + scanSysTimerTopic(); + waitTime = 100L; + } + waitForRunning(waitTime); + } catch (Exception e) { + logError.error("TimerMessageRocksDBStore error occurred in: {}, error: {}", getServiceName(), + e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void scanSysTimerTopic() { + ConsumeQueueInterface cq = messageStore.getConsumeQueue(TIMER_TOPIC, 0); + if (null == cq) { + return; + } + if (readOffset.get() < cq.getMinOffsetInQueue()) { + logError.warn("scanSysTimerTopic readOffset: {} is smaller than minOffsetInQueue: {}, use minOffsetInQueue to scan timer sysTimerTopic", readOffset.get(), cq.getMinOffsetInQueue()); + readOffset.set(cq.getMinOffsetInQueue()); + } else if (readOffset.get() > cq.getMaxOffsetInQueue()) { + logError.warn("scanSysTimerTopic readOffset: {} is bigger than maxOffsetInQueue: {}, use maxOffsetInQueue to scan timer sysTimerTopic", readOffset.get(), cq.getMaxOffsetInQueue()); + readOffset.set(cq.getMaxOffsetInQueue()); + } + ReferredIterator iterator = null; + try { + iterator = cq.iterateFrom(readOffset.get()); + if (null == iterator) { + return; + } + while (iterator.hasNext()) { + perfCounterTicks.startTick(SCAN_SYS_TOPIC); + try { + CqUnit cqUnit = iterator.next(); + if (null == cqUnit) { + logError.error("scanSysTimerTopic cqUnit is null, readOffset: {}", readOffset.get()); + break; + } + long offsetPy = cqUnit.getPos(); + int sizePy = cqUnit.getSize(); + long queueOffset = cqUnit.getQueueOffset(); + MessageExt msgExt = getMessageByCommitOffset(offsetPy, sizePy); + if (null == msgExt) { + perfCounterTicks.getCounter(SCAN_SYS_TOPIC_MISS); + break; + } + Long delayedTime = getDelayTime(msgExt); + if (null == delayedTime) { + readOffset.incrementAndGet(); + continue; + } + if (isExpired(delayedTime)) { + putRealTopicMessage(msgExt); + readOffset.incrementAndGet(); + continue; + } + TimerRocksDBRecord timerRecord = new TimerRocksDBRecord(delayedTime, MessageClientIDSetter.getUniqID(msgExt), offsetPy, sizePy, queueOffset, msgExt); + timeline.putRecord(timerRecord); + readOffset.incrementAndGet(); + + StoreMetricsManager metricsManager = messageStore.getStoreMetricsManager(); + if (metricsManager instanceof DefaultStoreMetricsManager) { + DefaultStoreMetricsManager defaultMetricsManager = (DefaultStoreMetricsManager)metricsManager; + Attributes attributes = defaultMetricsManager.newAttributesBuilder().put(DefaultStoreMetricsConstant.LABEL_TOPIC, msgExt.getProperty(MessageConst.PROPERTY_REAL_TOPIC)).build(); + defaultMetricsManager.getTimerMessageSetLatency().record((delayedTime - msgExt.getBornTimestamp()) / 1000, attributes); + } + } catch (Exception e) { + logError.error("Unknown error in scan the system topic error: {}", e.getMessage()); + } finally { + perfCounterTicks.endTick(SCAN_SYS_TOPIC); + } + } + } catch (Exception e) { + logError.error("scanSysTimerTopic throw Unknown exception. {}", e.getMessage()); + } finally { + if (iterator != null) { + iterator.release(); + } + } + } + } + + private class TimerMessageReputService extends ServiceThread { + private final Logger log = TimerMessageRocksDBStore.log; + private final BlockingQueue> queue; + private final RateLimiter rateLimiter; + private final boolean writeCheckPoint; + ExecutorService executor = new ThreadPoolExecutor( + 6, + 6, + 60, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(10000), + new ThreadPoolExecutor.CallerRunsPolicy() + ); + + public TimerMessageReputService(BlockingQueue> queue, double maxTps, boolean writeCheckPoint) { + this.queue = queue; + this.rateLimiter = RateLimiter.create(maxTps); + this.writeCheckPoint = writeCheckPoint; + } + + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + " service start"); + while (!this.isStopped() || !queue.isEmpty()) { + try { + List trs = queue.poll(100L, TimeUnit.MILLISECONDS); + if (CollectionUtils.isEmpty(trs)) { + continue; + } + long start = System.currentTimeMillis(); + CountDownLatch countDownLatch = new CountDownLatch(trs.size()); + for (TimerRocksDBRecord record : trs) { + executor.submit(new Task(countDownLatch, record)); + } + countDownLatch.await(); + log.info("TimerMessageReputService reput messages to commitlog, cost: {}, trs size: {}, checkPoint: {}", System.currentTimeMillis() - start, trs.size(), trs.get(trs.size() - 1).getCheckPoint()); + if (this.writeCheckPoint && !CollectionUtils.isEmpty(trs) && trs.get(trs.size() - 1).getCheckPoint() > 0L) { + log.info("TimerMessageReputService reput messages to commitlog, checkPoint: {}", trs.get(trs.size() - 1).getCheckPoint()); + messageRocksDBStorage.writeCheckPointForTimer(TIMER_COLUMN_FAMILY, MessageRocksDBStorage.TIMELINE_CHECK_POINT, trs.get(trs.size() - 1).getCheckPoint()); + } + } catch (Exception e) { + logError.error("TimerMessageReputService error: {}", e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + private void putMsgWithRetry(MessageExtBrokerInner msg) throws InterruptedException { + if (null == msg) { + return; + } + for (int retryCount = 0; !isStopped() && retryCount <= MAX_PUT_MSG_TIMES; retryCount++) { + int result = doPut(msg); + switch (result) { + case PUT_OK: + return; + case PUT_NO_RETRY: + logError.warn("Skipping message due to unrecoverable error. Msg: {}", msg); + return; + default: + if (retryCount == MAX_PUT_MSG_TIMES) { + logError.error("Message processing failed after {} retries. Msg: {}", retryCount, msg); + return; + } else { + Thread.sleep(100L); + logError.warn("Retrying to process message. Retry count: {}, Msg: {}", retryCount, msg); + } + } + } + } + + class Task implements Callable { + private CountDownLatch countDownLatch; + private TimerRocksDBRecord record; + + public Task(CountDownLatch countDownLatch, TimerRocksDBRecord record) { + this.countDownLatch = countDownLatch; + this.record = record; + } + + @Override + public Void call() throws Exception { + try { + perfCounterTicks.startTick(OUT_BIZ_MESSAGE); + MessageExt messageExt = record.getMessageExt(); + if (null == messageExt) { + messageExt = getMessageByCommitOffset(record.getOffsetPy(), record.getSizePy()); + if (null == messageExt) { + return null; + } + } + MessageExtBrokerInner msg = convertMessage(messageExt); + if (null == msg) { + return null; + } + record.setUniqKey(MessageClientIDSetter.getUniqID(msg)); + putMsgWithRetry(msg); + timeline.addMetric(msg, -1); + perfCounterTicks.endTick(OUT_BIZ_MESSAGE); + rateLimiter.acquire(); + } catch (Exception e) { + logError.error("TimerMessageReputService running error: {}", e.getMessage()); + } finally { + countDownLatch.countDown(); + } + return null; + } + } + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerRocksDBRecord.java b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerRocksDBRecord.java new file mode 100644 index 00000000000..8b9123a63b5 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/timer/rocksdb/TimerRocksDBRecord.java @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.timer.rocksdb; + +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; + +public class TimerRocksDBRecord { + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + public static final byte TIMER_ROCKSDB_PUT = (byte)0; + public static final byte TIMER_ROCKSDB_DELETE = (byte)1; + public static final byte TIMER_ROCKSDB_UPDATE = (byte)2; + private static final int VALUE_LENGTH = Integer.BYTES + Long.BYTES; + + private long delayTime; + private String uniqKey; + private int sizePy; + private long offsetPy; + private long queueOffset; + private long checkPoint; + private byte actionFlag; + private MessageExt messageExt; + + public TimerRocksDBRecord() {} + + public TimerRocksDBRecord(long delayTime, String uniqKey, long offsetPy, int sizePy, long queueOffset, MessageExt messageExt) { + this.delayTime = delayTime; + this.uniqKey = uniqKey; + this.offsetPy = offsetPy; + this.sizePy = sizePy; + this.messageExt = messageExt; + this.queueOffset = queueOffset; + } + + public byte[] getKeyBytes() { + if (StringUtils.isEmpty(uniqKey) || delayTime <= 0L) { + return null; + } + try { + byte[] uniqKeyBytes = uniqKey.getBytes(StandardCharsets.UTF_8); + int keyLength = Long.BYTES + uniqKeyBytes.length; + return ByteBuffer.allocate(keyLength).putLong(delayTime).put(uniqKeyBytes).array(); + } catch (Exception e) { + logError.error("TimerRocksDBRecord getKeyBytes error: {}", e.getMessage()); + return null; + } + } + + public byte[] getValueBytes() { + if (sizePy <= 0 || offsetPy < 0L) { + return null; + } + try { + return ByteBuffer.allocate(VALUE_LENGTH).putInt(sizePy).putLong(offsetPy).array(); + } catch (Exception e) { + logError.error("TimerRocksDBRecord getValueBytes error: {}", e.getMessage()); + return null; + } + } + + public static TimerRocksDBRecord decode(byte[] key, byte[] value) { + if (null == key || key.length < Long.BYTES || null == value || value.length != VALUE_LENGTH) { + return null; + } + try { + TimerRocksDBRecord rocksDBRecord = new TimerRocksDBRecord(); + ByteBuffer keyBuffer = ByteBuffer.wrap(key); + rocksDBRecord.setDelayTime(keyBuffer.getLong()); + byte[] uniqKey = new byte[key.length - Long.BYTES]; + keyBuffer.get(uniqKey); + rocksDBRecord.setUniqKey(new String(uniqKey, StandardCharsets.UTF_8)); + ByteBuffer valueByteBuffer = ByteBuffer.wrap(value); + rocksDBRecord.setSizePy(valueByteBuffer.getInt()); + rocksDBRecord.setOffsetPy(valueByteBuffer.getLong()); + return rocksDBRecord; + } catch (Exception e) { + logError.error("TimerRocksDBRecord decode error: {}", e.getMessage()); + return null; + } + } + + public void setDelayTime(long delayTime) { + this.delayTime = delayTime; + } + + public void setOffsetPy(long offsetPy) { + this.offsetPy = offsetPy; + } + + public void setSizePy(int sizePy) { + this.sizePy = sizePy; + } + + public int getSizePy() { + return sizePy; + } + + public long getDelayTime() { + return delayTime; + } + + public long getOffsetPy() { + return offsetPy; + } + + public MessageExt getMessageExt() { + return messageExt; + } + + public void setMessageExt(MessageExt messageExt) { + this.messageExt = messageExt; + } + + public String getUniqKey() { + return uniqKey; + } + + public void setUniqKey(String uniqKey) { + this.uniqKey = uniqKey; + } + + public void setCheckPoint(long checkPoint) { + this.checkPoint = checkPoint; + } + + public long getCheckPoint() { + return checkPoint; + } + + public long getQueueOffset() { + return queueOffset; + } + + public void setQueueOffset(long queueOffset) { + this.queueOffset = queueOffset; + } + + public byte getActionFlag() { + return actionFlag; + } + + public void setActionFlag(byte actionFlag) { + this.actionFlag = actionFlag; + } + + @Override + public String toString() { + return "TimerRocksDBRecord{" + + "delayTime=" + delayTime + + ", uniqKey=" + uniqKey + + ", sizePy=" + sizePy + + ", offsetPy=" + offsetPy + + ", queueOffset=" + queueOffset + + ", checkPoint=" + checkPoint + + '}'; + } + +} diff --git a/store/src/main/java/org/apache/rocketmq/store/transaction/TransMessageRocksDBStore.java b/store/src/main/java/org/apache/rocketmq/store/transaction/TransMessageRocksDBStore.java new file mode 100644 index 00000000000..d71227c4af3 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/transaction/TransMessageRocksDBStore.java @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.transaction; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.ServiceThread; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageAccessor; +import org.apache.rocketmq.common.message.MessageClientIDSetter; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.common.message.MessageDecoder; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageExtBrokerInner; +import org.apache.rocketmq.common.topic.TopicValidator; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; +import org.apache.rocketmq.store.DefaultMessageStore; +import org.apache.rocketmq.store.DispatchRequest; +import org.apache.rocketmq.store.MessageStore; +import org.apache.rocketmq.store.PutMessageResult; +import org.apache.rocketmq.store.PutMessageStatus; +import org.apache.rocketmq.store.StoreUtil; +import org.apache.rocketmq.store.config.MessageStoreConfig; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.stats.BrokerStatsManager; +import static org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage.TRANS_COLUMN_FAMILY; + +public class TransMessageRocksDBStore { + private static final Logger log = LoggerFactory.getLogger(LoggerName.STORE_LOGGER_NAME); + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + private static final String REMOVE_TAG = "d"; + private static final byte[] FILL_BYTE = new byte[] {(byte) 0}; + private static final int DEFAULT_CAPACITY = 100000; + private static final int BATCH_SIZE = 1000; + private static final int MAX_GET_MSG_TIMES = 3; + private static final int INITIAL = 0, RUNNING = 1, SHUTDOWN = 2; + private volatile int state = INITIAL; + + private final MessageStore messageStore; + private final MessageStoreConfig storeConfig; + private final MessageRocksDBStorage messageRocksDBStorage; + private final BrokerStatsManager brokerStatsManager; + private final SocketAddress storeHost; + private ThreadLocal bufferLocal = null; + private TransIndexBuildService transIndexBuildService; + protected BlockingQueue originTransMsgQueue; + + public TransMessageRocksDBStore(final MessageStore messageStore, final BrokerStatsManager brokerStatsManager, final SocketAddress storeHost) { + this.messageStore = messageStore; + this.storeConfig = messageStore.getMessageStoreConfig(); + this.messageRocksDBStorage = messageStore.getMessageRocksDBStorage(); + this.brokerStatsManager = brokerStatsManager; + this.storeHost = storeHost; + bufferLocal = ThreadLocal.withInitial(() -> ByteBuffer.allocate(storeConfig.getMaxMessageSize())); + if (storeConfig.isTransRocksDBEnable()) { + init(); + } + } + + private void init() { + if (this.state == RUNNING) { + return; + } + this.transIndexBuildService = new TransIndexBuildService(); + this.originTransMsgQueue = new LinkedBlockingDeque<>(DEFAULT_CAPACITY); + this.transIndexBuildService.start(); + this.state = RUNNING; + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(TRANS_COLUMN_FAMILY); + log.info("TransMessageRocksDBStore start success, lastOffsetPy: {}", lastOffsetPy); + } + + public void shutdown() { + if (this.state != RUNNING || this.state == SHUTDOWN) { + return; + } + if (null != this.transIndexBuildService) { + this.transIndexBuildService.shutdown(); + } + this.state = SHUTDOWN; + log.info("TransMessageRocksDBStore shutdown success"); + } + + public void buildTransIndex(DispatchRequest dispatchRequest) { + if (null == dispatchRequest || dispatchRequest.getCommitLogOffset() < 0L || dispatchRequest.getMsgSize() <= 0 || state != RUNNING || null == this.originTransMsgQueue) { + logError.error("TransMessageRocksDBStore buildTransIndex error, dispatchRequest: {}, state: {}, originTransMsgQueue: {}", dispatchRequest, state, originTransMsgQueue); + return; + } + long reqOffsetPy = dispatchRequest.getCommitLogOffset(); + long endOffsetPy = messageRocksDBStorage.getLastOffsetPy(TRANS_COLUMN_FAMILY); + if (reqOffsetPy < endOffsetPy) { + if (System.currentTimeMillis() % 1000 == 0) { + log.warn("TransMessageRocksDBStore buildTransIndex recover, but ignore, reqOffsetPy: {}, endOffsetPy: {}", reqOffsetPy, endOffsetPy); + } + return; + } + int reqMsgSize = dispatchRequest.getMsgSize(); + try { + MessageExt msgInner = getMessage(reqOffsetPy, reqMsgSize); + if (null == msgInner) { + logError.error("TransMessageRocksDBStore buildTransIndex error, msgInner is not found, reqOffsetPy: {}, reqMsgSize: {}", reqOffsetPy, reqMsgSize); + return; + } + String topic = msgInner.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC); + String uniqKey = msgInner.getUserProperty(MessageConst.PROPERTY_TRANSACTION_ID); + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey)) { + logError.error("TransMessageRocksDBStore buildTransIndex error, uniqKey: {}, topic: {}", uniqKey, topic); + return; + } + TransRocksDBRecord transRocksDBRecord = null; + String reqTopic = dispatchRequest.getTopic(); + if (TopicValidator.RMQ_SYS_ROCKSDB_TRANS_HALF_TOPIC.equals(reqTopic)) { + transRocksDBRecord = new TransRocksDBRecord(reqOffsetPy, topic, uniqKey, reqMsgSize, 0); + } else if (TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC.equals(reqTopic)) { + long offsetPy = -1L; + String transOffsetPy = null; + try { + transOffsetPy = msgInner.getUserProperty(MessageConst.PROPERTY_TRANS_OFFSET); + if (!StringUtils.isEmpty(transOffsetPy)) { + offsetPy = Long.parseLong(transOffsetPy); + } + if (offsetPy >= 0L) { + transRocksDBRecord = new TransRocksDBRecord(offsetPy, topic, uniqKey, true); + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore buildTransIndex error, transOffsetPy: {}, error: {}", transOffsetPy, e.getMessage()); + } + } + if (null != transRocksDBRecord) { + while (!originTransMsgQueue.offer(transRocksDBRecord, 3, TimeUnit.SECONDS)) { + if (System.currentTimeMillis() % 1000 == 0) { + logError.error("TransMessageRocksDBStore buildTransStatus offer transRocksDBRecord error, topic: {}, uniqKey: {}, reqOffsetPy: {}", topic, uniqKey, reqOffsetPy); + } + } + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore buildTransStatus error: {}", e.getMessage()); + } + } + + public void deletePrepareMessage(MessageExt messageExt) { + if (null == messageExt) { + logError.error("TransMessageRocksDBStore deletePrepareMessage error, messageExt is null"); + return; + } + try { + MessageExtBrokerInner msgInner = makeOpMessageInner(messageExt); + if (null == msgInner) { + logError.error("TransMessageRocksDBStore deletePrepareMessage msgInner is null"); + return; + } + PutMessageResult result = messageStore.putMessage(msgInner); + if (result != null && result.getPutMessageStatus() == PutMessageStatus.PUT_OK) { + this.brokerStatsManager.incTopicPutNums(msgInner.getTopic()); + this.brokerStatsManager.incTopicPutSize(msgInner.getTopic(), result.getAppendMessageResult().getWroteBytes()); + this.brokerStatsManager.incBrokerPutNums(); + return; + } + logError.error("TransMessageRocksDBStore deletePrepareMessage put op msg failed, result: {}", result); + } catch (Exception e) { + logError.error("TransMessageRocksDBStore deletePrepareMessage error: {}", e.getMessage()); + } + } + + public MessageExt getMessage(long offsetPy, int sizePy) { + if (offsetPy < 0L || sizePy <= 0 || sizePy > storeConfig.getMaxMessageSize()) { + logError.error("TransMessageRocksDBStore getMessage param error, offsetPy: {}, sizePy: {}, maxMsgSizeConfig: {}", offsetPy, sizePy, storeConfig.getMaxMessageSize()); + return null; + } + ByteBuffer byteBuffer = bufferLocal.get(); + if (sizePy > byteBuffer.limit()) { + bufferLocal.remove(); + byteBuffer = ByteBuffer.allocate(sizePy); + bufferLocal.set(byteBuffer); + } + for (int i = 0; i < MAX_GET_MSG_TIMES; i++) { + try { + MessageExt msgExt = StoreUtil.getMessage(offsetPy, sizePy, messageStore, bufferLocal.get()); + if (null == msgExt) { + log.warn("Fail to read msg from commitLog offsetPy:{} sizePy:{}", offsetPy, sizePy); + } else { + return msgExt; + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore getMessage error, offsetPy: {}, sizePy: {}, error: {}", offsetPy, sizePy, e.getMessage()); + } + } + return null; + } + + public MessageRocksDBStorage getMessageRocksDBStorage() { + return messageRocksDBStorage; + } + + private MessageExtBrokerInner makeOpMessageInner(MessageExt messageExt) { + if (null == messageExt) { + logError.error("TransMessageRocksDBStore makeOpMessageInner messageExt is null"); + return null; + } + try { + MessageExtBrokerInner msgInner = new MessageExtBrokerInner(); + msgInner.setTopic(TopicValidator.RMQ_SYS_ROCKSDB_TRANS_OP_HALF_TOPIC); + msgInner.setBody(FILL_BYTE); + msgInner.setQueueId(0); + msgInner.setTags(REMOVE_TAG); + msgInner.setTagsCode(MessageExtBrokerInner.tagsString2tagsCode(msgInner.getTags())); + msgInner.setSysFlag(0); + msgInner.setBornTimestamp(System.currentTimeMillis()); + msgInner.setBornHost(this.storeHost); + msgInner.setStoreHost(this.storeHost); + msgInner.setWaitStoreMsgOK(false); + MessageClientIDSetter.setUniqID(msgInner); + String uniqKey = MessageClientIDSetter.getUniqID(messageExt); + if (!StringUtils.isEmpty(uniqKey)) { + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_TRANSACTION_ID, uniqKey); + } + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_TRANS_OFFSET, String.valueOf(messageExt.getCommitLogOffset())); + MessageAccessor.putProperty(msgInner, MessageConst.PROPERTY_REAL_TOPIC, messageExt.getUserProperty(MessageConst.PROPERTY_REAL_TOPIC)); + msgInner.setPropertiesString(MessageDecoder.messageProperties2String(msgInner.getProperties())); + return msgInner; + } catch (Exception e) { + logError.error("TransMessageRocksDBStore makeOpMessageInner error: {}", e.getMessage()); + return null; + } + } + + public Integer getCheckTimes(String topic, String uniqKey, Long offsetPy) { + if (StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey) || null == offsetPy || offsetPy < 0L) { + return null; + } + try { + TransRocksDBRecord record = messageRocksDBStorage.getRecordForTrans(TRANS_COLUMN_FAMILY, new TransRocksDBRecord(offsetPy, topic, uniqKey, false)); + if (null == record) { + return null; + } + return record.getCheckTimes(); + } catch (Exception e) { + logError.error("TransMessageRocksDBStore getCheckTimes error, topic: {}, uniqKey: {}, offsetPy: {}, error: {}", topic, uniqKey, offsetPy, e.getMessage()); + return null; + } + } + + public boolean isMappedFileMatchedRecover(long phyOffset) { + if (!storeConfig.isTransRocksDBEnable()) { + return true; + } + Long lastOffsetPy = messageRocksDBStorage.getLastOffsetPy(TRANS_COLUMN_FAMILY); + log.info("trans isMappedFileMatchedRecover lastOffsetPy: {}", lastOffsetPy); + if (null != lastOffsetPy && phyOffset <= lastOffsetPy) { + log.info("isMappedFileMatchedRecover TransMessageRocksDBStore recover form this offset, phyOffset: {}, lastOffsetPy: {}", phyOffset, lastOffsetPy); + return true; + } + return false; + } + + private String getServiceThreadName() { + String brokerIdentifier = ""; + if (TransMessageRocksDBStore.this.messageStore instanceof DefaultMessageStore) { + DefaultMessageStore messageStore = (DefaultMessageStore) TransMessageRocksDBStore.this.messageStore; + if (messageStore.getBrokerConfig().isInBrokerContainer()) { + brokerIdentifier = messageStore.getBrokerConfig().getIdentifier(); + } + } + return brokerIdentifier; + } + + public class TransIndexBuildService extends ServiceThread { + private final Logger log = TransMessageRocksDBStore.log; + private List trs; + @Override + public String getServiceName() { + return getServiceThreadName() + this.getClass().getSimpleName(); + } + + @Override + public void run() { + log.info(this.getServiceName() + "service start"); + trs = new ArrayList<>(BATCH_SIZE); + while (!this.isStopped() || !originTransMsgQueue.isEmpty()) { + try { + buildTransIndex(); + } catch (Exception e) { + trs.clear(); + logError.error("TransMessageRocksDBStore error occurred in: {}, error: {}", getServiceName(), e.getMessage()); + } + } + log.info(this.getServiceName() + " service end"); + } + + protected void buildTransIndex() { + pollTransMessageRecords(); + if (CollectionUtils.isEmpty(trs)) { + return; + } + try { + messageRocksDBStorage.writeRecordsForTrans(TRANS_COLUMN_FAMILY, trs); + } catch (Exception e) { + logError.error("TransMessageRocksDBStore pollAndPutTransRequest writeRecords error: {}", e.getMessage()); + } + trs.clear(); + } + + protected void pollTransMessageRecords() { + try { + TransRocksDBRecord firstReq = originTransMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null != firstReq) { + trs.add(firstReq); + while (true) { + TransRocksDBRecord tmpReq = originTransMsgQueue.poll(100, TimeUnit.MILLISECONDS); + if (null == tmpReq) { + break; + } + trs.add(tmpReq); + if (trs.size() >= BATCH_SIZE) { + break; + } + } + } + } catch (Exception e) { + logError.error("TransMessageRocksDBStore fetchTransMessageRecord error: {}", e.getMessage()); + } + } + } +} diff --git a/store/src/main/java/org/apache/rocketmq/store/transaction/TransRocksDBRecord.java b/store/src/main/java/org/apache/rocketmq/store/transaction/TransRocksDBRecord.java new file mode 100644 index 00000000000..099f6150939 --- /dev/null +++ b/store/src/main/java/org/apache/rocketmq/store/transaction/TransRocksDBRecord.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.store.transaction; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.constant.LoggerName; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.logging.org.slf4j.Logger; +import org.apache.rocketmq.logging.org.slf4j.LoggerFactory; + +public class TransRocksDBRecord { + private static final Logger logError = LoggerFactory.getLogger(LoggerName.STORE_ERROR_LOGGER_NAME); + public static final int VALUE_LENGTH = Integer.BYTES + Integer.BYTES; + private static final String KEY_SPLIT = "@"; + protected long offsetPy; + private String topic; + private String uniqKey; + private int checkTimes = 0; + private int sizePy; + private boolean isOp; + private boolean delete; + private MessageExt messageExt; + + public TransRocksDBRecord(long offsetPy, String topic, String uniqKey, int sizePy, int checkTimes) { + this.offsetPy = offsetPy; + this.topic = topic; + this.uniqKey = uniqKey; + this.sizePy = sizePy; + this.checkTimes = checkTimes; + } + + public TransRocksDBRecord(long offsetPy, String topic, String uniqKey, boolean isOp) { + this.offsetPy = offsetPy; + this.topic = topic; + this.uniqKey = uniqKey; + this.isOp = isOp; + } + + public TransRocksDBRecord() {} + + public byte[] getKeyBytes() { + if (offsetPy < 0L || StringUtils.isEmpty(topic) || StringUtils.isEmpty(uniqKey)) { + return null; + } + byte[] keySuffixBytes = (KEY_SPLIT + topic + KEY_SPLIT + uniqKey).getBytes(StandardCharsets.UTF_8); + int keyLength = Long.BYTES + keySuffixBytes.length; + return ByteBuffer.allocate(keyLength).putLong(offsetPy).put(keySuffixBytes).array(); + } + + public byte[] getValueBytes() { + if (checkTimes < 0 || sizePy <= 0) { + logError.error("TransRocksDBRecord getValueBytes error, checkTimes: {}, sizePy: {}", checkTimes, sizePy); + return null; + } + return ByteBuffer.allocate(VALUE_LENGTH).putInt(checkTimes).putInt(sizePy).array(); + } + + public static TransRocksDBRecord decode(byte[] key, byte[] value) { + if (null == key || key.length <= Long.BYTES || null == value || value.length != VALUE_LENGTH) { + logError.error("TransRocksDBRecord decode param error, key: {}, value: {}", key, value); + return null; + } + TransRocksDBRecord transRocksDBRecord = null; + try { + transRocksDBRecord = new TransRocksDBRecord(); + ByteBuffer keyByteBuffer = ByteBuffer.wrap(key); + transRocksDBRecord.setOffsetPy(keyByteBuffer.getLong()); + byte[] keySuffix = new byte[key.length - Long.BYTES]; + keyByteBuffer.get(keySuffix); + String[] keySuffixSplit = new String(keySuffix, StandardCharsets.UTF_8).split(KEY_SPLIT); + if (keySuffixSplit.length != 3) { + logError.error("TransRocksDBRecord decode keySuffixSplit parse error"); + return null; + } + transRocksDBRecord.setTopic(keySuffixSplit[1]); + transRocksDBRecord.setUniqKey(keySuffixSplit[2]); + ByteBuffer valueByteBuffer = ByteBuffer.wrap(value); + transRocksDBRecord.setCheckTimes(valueByteBuffer.getInt()); + transRocksDBRecord.setSizePy(valueByteBuffer.getInt()); + } catch (Exception e) { + logError.error("TransRocksDBRecord decode error, valueLength: {}, error: {}", value.length, e.getMessage()); + return null; + } + return transRocksDBRecord; + } + + public String getTopic() { + return topic; + } + + public void setTopic(String topic) { + this.topic = topic; + } + + public String getUniqKey() { + return uniqKey; + } + + public void setUniqKey(String uniqKey) { + this.uniqKey = uniqKey; + } + + public int getCheckTimes() { + return checkTimes; + } + + public void setCheckTimes(int checkTimes) { + this.checkTimes = checkTimes; + } + + public int getSizePy() { + return sizePy; + } + + public void setSizePy(int sizePy) { + this.sizePy = sizePy; + } + + public long getOffsetPy() { + return offsetPy; + } + + public void setOffsetPy(long offsetPy) { + this.offsetPy = offsetPy; + } + + public MessageExt getMessageExt() { + return messageExt; + } + + public void setMessageExt(MessageExt messageExt) { + this.messageExt = messageExt; + } + + public boolean isOp() { + return isOp; + } + + public void setOp(boolean op) { + isOp = op; + } + + public boolean isDelete() { + return delete; + } + + public void setDelete(boolean delete) { + this.delete = delete; + } +} diff --git a/store/src/test/java/org/apache/rocketmq/store/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/dledger/MixCommitlogTest.java b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java index 1bfc6f72eaa..93503c4ebd5 100644 --- a/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/dledger/MixCommitlogTest.java @@ -25,12 +25,14 @@ import org.apache.rocketmq.store.config.StorePathConfigHelper; import org.junit.Assert; import org.junit.Assume; +import org.junit.Ignore; import org.junit.Test; import static org.awaitility.Awaitility.await; public class MixCommitlogTest extends MessageStoreTestBase { + @Ignore @Test public void testFallBehindCQ() throws Exception { Assume.assumeFalse(MixAll.isWindows()); diff --git a/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java b/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java index 86371ea9006..519af441591 100644 --- a/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java +++ b/store/src/test/java/org/apache/rocketmq/store/ha/autoswitch/AutoSwitchHATest.java @@ -221,12 +221,12 @@ public void testConfirmOffset() throws Exception { // Step2, shutdown store2 this.messageStore2.shutdown(); - // Put message, which should put failed. + // Put message, which should succeed because slave is removed from syncStateSet, only master remains final PutMessageResult putMessageResult = this.messageStore1.putMessage(buildMessage()); - assertEquals(putMessageResult.getPutMessageStatus(), PutMessageStatus.FLUSH_SLAVE_TIMEOUT); + assertEquals(PutMessageStatus.PUT_OK,putMessageResult.getPutMessageStatus()); - // The confirmOffset still don't change, because syncStateSet contains broker2, but broker2 shutdown - assertEquals(confirmOffset, this.messageStore1.getConfirmOffset()); + // The confirmOffset should update because syncStateSet only contains master after slave shutdown + assertTrue(this.messageStore1.getConfirmOffset() >= confirmOffset); // Step3, shutdown store1, start store2, change store2 to master, epoch = 2 this.messageStore1.shutdown(); @@ -296,10 +296,19 @@ public void testOptionAllAckInSyncStateSet() throws Exception { this.messageStore2.shutdown(); this.messageStore2.destroy(); + // Wait for connection to be removed and syncStateSet to be updated by removeConnection + await().atMost(10, TimeUnit.SECONDS).until(() -> { + AutoSwitchHAService haService = (AutoSwitchHAService) this.messageStore1.getHaService(); + return haService.getConnectionCount().get() == 0 + && haService.getLocalSyncStateSet().size() == 1; + }); + + // Now manually set syncStateSet back to {1, 2} to test the scenario where + // syncStateSet contains a disconnected slave ((AutoSwitchHAService) this.messageStore1.getHaService()).setSyncStateSet(result); final PutMessageResult putMessageResult = this.messageStore1.putMessage(buildMessage()); - assertEquals(putMessageResult.getPutMessageStatus(), PutMessageStatus.FLUSH_SLAVE_TIMEOUT); + assertEquals(PutMessageStatus.FLUSH_SLAVE_TIMEOUT,putMessageResult.getPutMessageStatus()); } @Ignore diff --git a/store/src/test/java/org/apache/rocketmq/store/pop/AckMsgTest.java b/store/src/test/java/org/apache/rocketmq/store/pop/AckMsgTest.java index b5a3ff6381a..13df028e83d 100644 --- a/store/src/test/java/org/apache/rocketmq/store/pop/AckMsgTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/pop/AckMsgTest.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.store.pop; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import org.junit.Assert; import org.junit.Test; diff --git a/store/src/test/java/org/apache/rocketmq/store/pop/BatchAckMsgTest.java b/store/src/test/java/org/apache/rocketmq/store/pop/BatchAckMsgTest.java index 4bcfcf18be6..0a1bc714cfa 100644 --- a/store/src/test/java/org/apache/rocketmq/store/pop/BatchAckMsgTest.java +++ b/store/src/test/java/org/apache/rocketmq/store/pop/BatchAckMsgTest.java @@ -17,7 +17,7 @@ package org.apache.rocketmq.store.pop; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import org.junit.Assert; import org.junit.Test; 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/BUILD.bazel b/test/BUILD.bazel index a8a7a9d8bb3..d34456f3556 100644 --- a/test/BUILD.bazel +++ b/test/BUILD.bazel @@ -28,7 +28,6 @@ java_library( "//tools", "@maven//:ch_qos_logback_logback_classic", "@maven//:ch_qos_logback_logback_core", - "@maven//:com_alibaba_fastjson", "@maven//:com_github_luben_zstd_jni", "@maven//:com_google_guava_guava", "@maven//:com_google_protobuf_protobuf_java_util", 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 287e54d5617..0c4a8fc8353 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); @@ -141,6 +158,7 @@ public static BrokerController createAndStartBroker(String nsAddr) { brokerConfig.setRecallMessageEnable(true); storeConfig.setEnableConsumeQueueExt(true); brokerConfig.setLoadBalancePollNameServerInterval(500); + brokerConfig.setConfigManagerVersion(System.getProperty("configManagerVersion", "v1")); storeConfig.setStorePathRootDir(baseDir); storeConfig.setStorePathCommitLog(baseDir + SEP + "commitlog"); storeConfig.setMappedFileSizeCommitLog(commitLogSize); @@ -149,9 +167,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.intConfig(); + 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(); @@ -171,10 +247,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)) { @@ -183,16 +265,19 @@ 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); + return initTopic(topic, nsAddr, clusterName, BaseConf.QUEUE_NUMBERS, CQType.SimpleCQ, topicMessageType, null); } public static void deleteFile(File file) { @@ -210,4 +295,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 2d186373764..d1a6a5f44d6 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); 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/tieredstore/BUILD.bazel b/tieredstore/BUILD.bazel index 8822280ff87..0a943020860 100644 --- a/tieredstore/BUILD.bazel +++ b/tieredstore/BUILD.bazel @@ -39,7 +39,7 @@ java_library( "@maven//:io_opentelemetry_opentelemetry_exporter_logging_otlp", "@maven//:org_apache_commons_commons_lang3", "@maven//:org_apache_tomcat_annotations_api", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:org_apache_rocketmq_rocketmq_rocksdb", "@maven//:commons_collections_commons_collections", "@maven//:org_slf4j_slf4j_api", @@ -57,7 +57,7 @@ java_library( "//common", "//remoting", "//store", - "@maven//:com_alibaba_fastjson", + "@maven//:com_alibaba_fastjson2_fastjson2", "@maven//:commons_io_commons_io", "@maven//:io_opentelemetry_opentelemetry_api", "@maven//:io_opentelemetry_opentelemetry_context", diff --git a/tieredstore/README.md b/tieredstore/README.md index 6b5ecc8c8d4..1532fc3b5fd 100644 --- a/tieredstore/README.md +++ b/tieredstore/README.md @@ -45,12 +45,12 @@ Tiered storage provides some useful metrics, see [RIP-46](https://github.com/apa | Histogram | rocketmq_tiered_store_provider_upload_bytes | byte | | Histogram | rocketmq_tiered_store_provider_download_bytes | byte | | Gauge | rocketmq_tiered_store_dispatch_behind | | -| Gauge | rocketmq_tiered_store_dispatch_latency | byte | +| Gauge | rocketmq_tiered_store_dispatch_latency | milliseconds | | Counter | rocketmq_tiered_store_messages_dispatch_total | | | Counter | rocketmq_tiered_store_messages_out_total | | | Counter | rocketmq_tiered_store_get_message_fallback_total | | | Gauge | rocketmq_tiered_store_read_ahead_cache_count | | -| Gauge | rocketmq_tiered_store_read_ahead_cache_bytes | byte | +| Gauge | rocketmq_tiered_store_read_ahead_cache_bytes | bytes | | Counter | rocketmq_tiered_store_read_ahead_cache_access_total | | | Counter | rocketmq_tiered_store_read_ahead_cache_hit_total | | | Gauge | rocketmq_storage_message_reserve_time | milliseconds | diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/MessageStoreConfig.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/MessageStoreConfig.java index 10667566aa0..d22ab80dd82 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/MessageStoreConfig.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/MessageStoreConfig.java @@ -129,6 +129,7 @@ public boolean check(TieredStorageLevel targetLevel) { private String objectStoreBucket = ""; private String objectStoreAccessKey = ""; private String objectStoreSecretKey = ""; + private boolean writeWithoutMmap = false; public static String localHostName() { try { @@ -418,4 +419,12 @@ public void setObjectStoreSecretKey(String objectStoreSecretKey) { public String getObjectStoreEndpoint() { return objectStoreEndpoint; } + + public boolean isWriteWithoutMmap() { + return writeWithoutMmap; + } + + public void setWriteWithoutMmap(boolean writeWithoutMmap) { + this.writeWithoutMmap = writeWithoutMmap; + } } diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java index 19b587fa327..68ae8acc9ec 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/TieredMessageStore.java @@ -42,6 +42,9 @@ import org.apache.rocketmq.store.SelectMappedBufferResult; import org.apache.rocketmq.store.plugin.AbstractPluginMessageStore; import org.apache.rocketmq.store.plugin.MessageStorePluginContext; +import org.apache.rocketmq.store.rocksdb.MessageRocksDBStorage; +import org.apache.rocketmq.store.timer.rocksdb.TimerMessageRocksDBStore; +import org.apache.rocketmq.store.transaction.TransMessageRocksDBStore; import org.apache.rocketmq.tieredstore.core.MessageStoreDispatcher; import org.apache.rocketmq.tieredstore.core.MessageStoreDispatcherImpl; import org.apache.rocketmq.tieredstore.core.MessageStoreFetcher; @@ -76,6 +79,9 @@ public class TieredMessageStore extends AbstractPluginMessageStore { protected final MessageStoreFilter topicFilter; protected final MessageStoreFetcher fetcher; protected final MessageStoreDispatcher dispatcher; + protected final MessageRocksDBStorage messageRocksDBStorage; + protected TimerMessageRocksDBStore timerMessageRocksDBStore; + protected TransMessageRocksDBStore transMessageRocksDBStore; public TieredMessageStore(MessageStorePluginContext context, MessageStore next) { super(context, next); @@ -83,9 +89,10 @@ public TieredMessageStore(MessageStorePluginContext context, MessageStore next) this.storeConfig = new MessageStoreConfig(); this.context = context; this.context.registerConfiguration(this.storeConfig); + this.storeConfig.setWriteWithoutMmap(context.getMessageStoreConfig().isWriteWithoutMmap()); this.brokerName = this.storeConfig.getBrokerName(); this.defaultStore = next; - + this.messageRocksDBStorage = defaultStore.getMessageRocksDBStorage(); this.metadataStore = this.getMetadataStore(this.storeConfig); this.topicFilter = new MessageStoreTopicFilter(this.storeConfig); this.storeExecutor = new MessageStoreExecutor(); @@ -306,6 +313,26 @@ public long getMinOffsetInQueue(String topic, int queueId) { return Math.min(minOffsetInNextStore, minOffsetInTieredStore); } + @Override + public TimerMessageRocksDBStore getTimerRocksDBStore() { + return timerMessageRocksDBStore; + } + + @Override + public TransMessageRocksDBStore getTransRocksDBStore() { + return transMessageRocksDBStore; + } + + @Override + public void setTimerMessageRocksDBStore(TimerMessageRocksDBStore timerMessageRocksDBStore) { + this.timerMessageRocksDBStore = timerMessageRocksDBStore; + } + + @Override + public void setTransRocksDBStore(TransMessageRocksDBStore transMessageRocksDBStore) { + this.transMessageRocksDBStore = transMessageRocksDBStore; + } + @Override public long getEarliestMessageTime(String topic, int queueId) { return getEarliestMessageTimeAsync(topic, queueId).join(); @@ -338,7 +365,7 @@ public CompletableFuture getMessageStoreTimeStampAsync(String topic, int q .thenApply(time -> { Attributes latencyAttributes = TieredStoreMetricsManager.newAttributesBuilder() .put(TieredStoreMetricsConstant.LABEL_OPERATION, - TieredStoreMetricsConstant.OPERATION_API_GET_TIME_BY_OFFSET) + TieredStoreMetricsConstant.OPERATION_API_GET_TIME_BY_OFFSET) .put(TieredStoreMetricsConstant.LABEL_TOPIC, topic) .build(); TieredStoreMetricsManager.apiLatency.record(stopwatch.elapsed(TimeUnit.MILLISECONDS), latencyAttributes); @@ -382,6 +409,12 @@ public QueryMessageResult queryMessage(String topic, String key, int maxNum, lon return queryMessageAsync(topic, key, maxNum, begin, end).join(); } + @Override + public QueryMessageResult queryMessage(String topic, String key, int maxNum, long begin, + long end, String keyType, String lastKey) { + return queryMessageAsync(topic, key, maxNum, begin, end, keyType, lastKey).join(); + } + @Override public CompletableFuture queryMessageAsync(String topic, String key, int maxNum, long begin, long end) { @@ -417,6 +450,38 @@ public CompletableFuture queryMessageAsync(String topic, Str return CompletableFuture.completedFuture(result); } + @Override + public CompletableFuture queryMessageAsync(String topic, String key, int maxNum, long begin, long end, String indexType, String lastKey) { + long earliestTimeInNextStore = next.getEarliestMessageTime(); + if (earliestTimeInNextStore <= 0) { + log.warn("TieredMessageStore queryMessageAsync: get earliest message time in next store failed: {}", earliestTimeInNextStore); + } + boolean isForce = storeConfig.getTieredStorageLevel() == MessageStoreConfig.TieredStorageLevel.FORCE; + QueryMessageResult result = end < earliestTimeInNextStore || isForce ? new QueryMessageResult() : next.queryMessage(topic, key, maxNum, begin, end, indexType, lastKey); + int resultSize = result.getMessageBufferList().size(); + if (resultSize < maxNum && begin < earliestTimeInNextStore || isForce) { + Stopwatch stopwatch = Stopwatch.createStarted(); + try { + return fetcher.queryMessageAsync(topic, key, maxNum - resultSize, begin, isForce ? end : earliestTimeInNextStore) + .thenApply(tieredStoreResult -> { + Attributes latencyAttributes = TieredStoreMetricsManager.newAttributesBuilder() + .put(TieredStoreMetricsConstant.LABEL_OPERATION, TieredStoreMetricsConstant.OPERATION_API_QUERY_MESSAGE) + .put(TieredStoreMetricsConstant.LABEL_TOPIC, topic) + .build(); + TieredStoreMetricsManager.apiLatency.record(stopwatch.elapsed(TimeUnit.MILLISECONDS), latencyAttributes); + for (SelectMappedBufferResult msg : tieredStoreResult.getMessageMapedList()) { + result.addMessage(msg); + } + return result; + }); + } catch (Exception e) { + log.error("TieredMessageStore#queryMessageAsync: query message in tiered store failed", e); + return CompletableFuture.completedFuture(result); + } + } + return CompletableFuture.completedFuture(result); + } + @Override public List> getMetricsView() { List> res = super.getMetricsView(); @@ -430,6 +495,11 @@ public void initMetrics(Meter meter, Supplier attributesBuild TieredStoreMetricsManager.init(meter, attributesBuilderSupplier, storeConfig, fetcher, flatFileStore, next); } + @Override + public MessageRocksDBStorage getMessageRocksDBStorage() { + return messageRocksDBStorage; + } + @Override public int cleanUnusedTopic(Set retainTopics) { metadataStore.iterateTopic(topicMetadata -> { @@ -465,8 +535,13 @@ public synchronized void shutdown() { dispatcher.shutdown(); } if (indexService != null) { - indexService.shutdown(); + if (defaultStore.getRunningFlags() != null && defaultStore.getRunningFlags().isStoreWriteable()) { + indexService.shutdown(); + } else { + indexService.forceShutdown(); + } } + if (flatFileStore != null) { flatFileStore.shutdown(); } diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/FlatMessageFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/FlatMessageFile.java index b0e4dd6e3b0..a7505b4bf41 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/FlatMessageFile.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/file/FlatMessageFile.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.tieredstore.file; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; import com.google.common.annotations.VisibleForTesting; import java.nio.ByteBuffer; import java.util.ArrayList; diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexService.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexService.java index a4ea7e78a85..11fb1482c1f 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexService.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexService.java @@ -60,6 +60,13 @@ default void forceUpload() { */ void shutdown(); + /** + * Force shutdown the index service. + */ + default void forceShutdown() { + shutdown(); + }; + /** * Destroys the index service and releases all resources. */ diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreFile.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreFile.java index 528bce9bb8b..e0a3c5cd0af 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreFile.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreFile.java @@ -17,9 +17,11 @@ package org.apache.rocketmq.tieredstore.index; import com.google.common.base.Stopwatch; +import java.io.FileNotFoundException; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.MappedByteBuffer; +import java.nio.channels.FileChannel; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collections; @@ -90,20 +92,26 @@ public class IndexStoreFile implements IndexFile { private final AtomicInteger hashSlotCount = new AtomicInteger(0); private final AtomicInteger indexItemCount = new AtomicInteger(0); + private final boolean writeWithoutMmap; private MappedFile mappedFile; private ByteBuffer byteBuffer; private MappedFile compactMappedFile; private FileSegment fileSegment; + private FileChannel fileChannel; + public IndexStoreFile(MessageStoreConfig storeConfig, long timestamp) throws IOException { + this.writeWithoutMmap = storeConfig.isWriteWithoutMmap(); this.hashSlotMaxCount = storeConfig.getTieredStoreIndexFileMaxHashSlotNum(); this.indexItemMaxCount = storeConfig.getTieredStoreIndexFileMaxIndexNum(); this.fileStatus = new AtomicReference<>(UNSEALED); this.fileReadWriteLock = new ReentrantReadWriteLock(); this.mappedFile = new DefaultMappedFile( Paths.get(storeConfig.getStorePathRootDir(), FILE_DIRECTORY_NAME, String.valueOf(timestamp)).toString(), - this.getItemPosition(indexItemMaxCount)); + this.getItemPosition(indexItemMaxCount), + this.writeWithoutMmap); this.byteBuffer = this.mappedFile.getMappedByteBuffer(); + this.fileChannel = this.mappedFile.getFileChannel(); this.beginTimestamp.set(timestamp); this.endTimestamp.set(byteBuffer.getLong(INDEX_BEGIN_TIME_STAMP)); @@ -113,6 +121,7 @@ public IndexStoreFile(MessageStoreConfig storeConfig, long timestamp) throws IOE } public IndexStoreFile(MessageStoreConfig storeConfig, FileSegment fileSegment) { + this.writeWithoutMmap = storeConfig.isWriteWithoutMmap(); this.fileSegment = fileSegment; this.fileStatus = new AtomicReference<>(UPLOAD); this.fileReadWriteLock = new ReentrantReadWriteLock(); @@ -157,12 +166,31 @@ protected int hashCode(String keyStr) { return (keyHash < 0) ? -keyHash : keyHash; } - protected void flushNewMetadata(ByteBuffer byteBuffer, boolean end) { - byteBuffer.putInt(INDEX_MAGIC_CODE, !end ? BEGIN_MAGIC_CODE : END_MAGIC_CODE); - byteBuffer.putLong(INDEX_BEGIN_TIME_STAMP, this.beginTimestamp.get()); - byteBuffer.putLong(INDEX_END_TIME_STAMP, this.endTimestamp.get()); - byteBuffer.putInt(INDEX_SLOT_COUNT, this.hashSlotCount.get()); - byteBuffer.putInt(INDEX_ITEM_INDEX, this.indexItemCount.get()); + protected void flushNewMetadata(ByteBuffer byteBuffer, boolean end) throws IOException { + flushNewMetadata(byteBuffer, end, null); + } + + protected void flushNewMetadata(ByteBuffer byteBuffer, boolean end, FileChannel channel) throws IOException { + FileChannel targetChannel = channel != null ? channel : fileChannel; + if (writeWithoutMmap && targetChannel != null) { + // Use FileChannel for writing + ByteBuffer writeBuffer = ByteBuffer.allocate(INDEX_HEADER_SIZE); + writeBuffer.putInt(!end ? BEGIN_MAGIC_CODE : END_MAGIC_CODE); + writeBuffer.putLong(this.beginTimestamp.get()); + writeBuffer.putLong(this.endTimestamp.get()); + writeBuffer.putInt(this.hashSlotCount.get()); + writeBuffer.putInt(this.indexItemCount.get()); + writeBuffer.flip(); + targetChannel.position(INDEX_MAGIC_CODE); + targetChannel.write(writeBuffer); + } else { + // Use ByteBuffer for writing + byteBuffer.putInt(INDEX_MAGIC_CODE, !end ? BEGIN_MAGIC_CODE : END_MAGIC_CODE); + byteBuffer.putLong(INDEX_BEGIN_TIME_STAMP, this.beginTimestamp.get()); + byteBuffer.putLong(INDEX_END_TIME_STAMP, this.endTimestamp.get()); + byteBuffer.putInt(INDEX_SLOT_COUNT, this.hashSlotCount.get()); + byteBuffer.putInt(INDEX_ITEM_INDEX, this.indexItemCount.get()); + } } protected int getSlotPosition(int slotIndex) { @@ -215,9 +243,26 @@ public AppendResult putKey( IndexItem indexItem = new IndexItem( topicId, queueId, offset, size, hashCode, timeDiff, slotOldValue); int itemIndex = this.indexItemCount.incrementAndGet(); - this.byteBuffer.position(this.getItemPosition(itemIndex)); - this.byteBuffer.put(indexItem.getByteBuffer()); - this.byteBuffer.putInt(slotPosition, itemIndex); + int itemPosition = this.getItemPosition(itemIndex); + + if (writeWithoutMmap && fileChannel != null) { + // Use FileChannel for writing + ByteBuffer itemBuffer = indexItem.getByteBuffer(); + fileChannel.position(itemPosition); + fileChannel.write(itemBuffer); + + ByteBuffer slotBuffer = ByteBuffer.allocate(Integer.BYTES); + slotBuffer.putInt(0, itemIndex); + slotBuffer.position(0); + slotBuffer.limit(Integer.BYTES); + fileChannel.position(slotPosition); + fileChannel.write(slotBuffer); + } else { + // Use ByteBuffer for writing + this.byteBuffer.position(itemPosition); + this.byteBuffer.put(indexItem.getByteBuffer()); + this.byteBuffer.putInt(slotPosition, itemIndex); + } if (slotOldValue <= INVALID_INDEX) { this.hashSlotCount.incrementAndGet(); @@ -231,7 +276,7 @@ public AppendResult putKey( this.getTimestamp(), topic, key, hashCode % this.hashSlotMaxCount, itemIndex, slotOldValue, indexItem); } return AppendResult.SUCCESS; - } catch (Exception e) { + } catch (Throwable e) { log.error("IndexStoreFile put key error, topic: {}, topicId: {}, queueId: {}, keySet: {}, offset: {}, " + "size: {}, timestamp: {}", topic, topicId, queueId, keySet, offset, size, timestamp, e); } finally { @@ -392,7 +437,9 @@ public ByteBuffer doCompaction() { buffer = compactToNewFile(); log.debug("IndexStoreFile do compaction, timestamp: {}, file size: {}, cost: {}ms", this.getTimestamp(), buffer.capacity(), stopwatch.elapsed(TimeUnit.MICROSECONDS)); - } catch (Exception e) { + } catch (FileNotFoundException e) { + throw new RuntimeException(e); + } catch (Throwable e) { log.error("IndexStoreFile do compaction, timestamp: {}, cost: {}ms", this.getTimestamp(), stopwatch.elapsed(TimeUnit.MICROSECONDS), e); return null; @@ -423,8 +470,9 @@ protected ByteBuffer compactToNewFile() throws IOException { int writePosition = INDEX_HEADER_SIZE + (hashSlotMaxCount * HASH_SLOT_SIZE); int fileMaxLength = writePosition + COMPACT_INDEX_ITEM_SIZE * indexItemCount.get(); - compactMappedFile = new DefaultMappedFile(this.getCompactedFilePath(), fileMaxLength); + compactMappedFile = new DefaultMappedFile(this.getCompactedFilePath(), fileMaxLength, writeWithoutMmap); MappedByteBuffer newBuffer = compactMappedFile.getMappedByteBuffer(); + FileChannel compactFileChannel = compactMappedFile.getFileChannel(); for (int i = 0; i < hashSlotMaxCount; i++) { int slotPosition = this.getSlotPosition(i); @@ -437,24 +485,48 @@ protected ByteBuffer compactToNewFile() throws IOException { buffer.get(payload); int newSlotValue = payloadBuffer.getInt(COMPACT_INDEX_ITEM_SIZE); buffer.limit(COMPACT_INDEX_ITEM_SIZE); - newBuffer.position(writePosition); - newBuffer.put(payload, 0, COMPACT_INDEX_ITEM_SIZE); + + if (writeWithoutMmap && compactFileChannel != null) { + // Use FileChannel for writing + ByteBuffer writeBuffer = ByteBuffer.wrap(payload, 0, COMPACT_INDEX_ITEM_SIZE); + compactFileChannel.position(writePosition); + compactFileChannel.write(writeBuffer); + } else { + // Use ByteBuffer for writing + newBuffer.position(writePosition); + newBuffer.put(payload, 0, COMPACT_INDEX_ITEM_SIZE); + } log.trace("IndexStoreFile do compaction, write item, slot: {}, current: {}, next: {}", i, slotValue, newSlotValue); slotValue = newSlotValue; writePosition += COMPACT_INDEX_ITEM_SIZE; } int length = writePosition - writeBeginPosition; - newBuffer.putInt(slotPosition, writeBeginPosition); - newBuffer.putInt(slotPosition + Integer.BYTES, length); + if (writeWithoutMmap && compactFileChannel != null) { + // Use FileChannel for writing + ByteBuffer slotWriteBuffer = ByteBuffer.allocate(Integer.BYTES * 2); + slotWriteBuffer.putInt(0, writeBeginPosition); + slotWriteBuffer.putInt(Integer.BYTES, length); + slotWriteBuffer.position(0); + slotWriteBuffer.limit(Integer.BYTES * 2); + compactFileChannel.position(slotPosition); + compactFileChannel.write(slotWriteBuffer); + } else { + // Use ByteBuffer for writing + newBuffer.putInt(slotPosition, writeBeginPosition); + newBuffer.putInt(slotPosition + Integer.BYTES, length); + } if (length > 0) { log.trace("IndexStoreFile do compaction, write slot, slot: {}, begin: {}, length: {}", i, writeBeginPosition, length); } } - this.flushNewMetadata(newBuffer, true); - newBuffer.flip(); + this.flushNewMetadata(newBuffer, true, compactFileChannel); + + // Set position and limit for reading + newBuffer.position(0); + newBuffer.limit(fileMaxLength); return newBuffer; } @@ -468,11 +540,13 @@ public void shutdown() { } if (this.mappedFile != null) { this.mappedFile.shutdown(TimeUnit.SECONDS.toMillis(10)); + this.mappedFile.cleanResources(); } if (this.compactMappedFile != null) { this.compactMappedFile.shutdown(TimeUnit.SECONDS.toMillis(10)); + this.compactMappedFile.cleanResources(); } - } catch (Exception e) { + } catch (Throwable e) { log.error("IndexStoreFile shutdown failed, timestamp: {}, status: {}", this.getTimestamp(), fileStatus.get(), e); } finally { fileReadWriteLock.writeLock().unlock(); diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreService.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreService.java index 2385628ed45..132d2162f99 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreService.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/index/IndexStoreService.java @@ -414,18 +414,27 @@ public void shutdown() { } } + @Override + public void forceShutdown() { + super.shutdown(); + } + @Override public void run() { while (!this.isStopped()) { - long expireTimestamp = System.currentTimeMillis() - - TimeUnit.HOURS.toMillis(storeConfig.getTieredStoreFileReservedTime()); - this.destroyExpiredFile(expireTimestamp); - IndexFile indexFile = this.getNextSealedFile(); - if (indexFile != null) { - if (this.doCompactThenUploadFile(indexFile)) { - this.setCompactTimestamp(indexFile.getTimestamp()); - continue; + try { + long expireTimestamp = System.currentTimeMillis() + - TimeUnit.HOURS.toMillis(storeConfig.getTieredStoreFileReservedTime()); + this.destroyExpiredFile(expireTimestamp); + IndexFile indexFile = this.getNextSealedFile(); + if (indexFile != null) { + if (this.doCompactThenUploadFile(indexFile)) { + this.setCompactTimestamp(indexFile.getTimestamp()); + continue; + } } + } catch (Throwable e) { + log.error("IndexStoreService running error", e); } this.waitForRunning(TimeUnit.SECONDS.toMillis(10)); } @@ -434,13 +443,14 @@ public void run() { if (autoCreateNewFile) { this.forceUpload(); } - this.timeStoreTable.forEach((timestamp, file) -> file.shutdown()); - this.timeStoreTable.clear(); } catch (Exception e) { log.error("IndexStoreService shutdown error", e); } finally { + this.timeStoreTable.forEach((timestamp, file) -> file.shutdown()); + this.timeStoreTable.clear(); readWriteLock.writeLock().unlock(); } + log.info(this.getServiceName() + " service shutdown"); } } diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/DefaultMetadataStore.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/DefaultMetadataStore.java index 09500bf6da8..eb726543cfb 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/DefaultMetadataStore.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/DefaultMetadataStore.java @@ -16,17 +16,9 @@ */ package org.apache.rocketmq.tieredstore.metadata; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.serializer.SerializerFeature; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import com.google.common.annotations.VisibleForTesting; -import java.nio.file.Paths; -import java.util.HashMap; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; import org.apache.rocketmq.common.ConfigManager; import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.remoting.protocol.RemotingSerializable; @@ -36,6 +28,15 @@ import org.apache.rocketmq.tieredstore.metadata.entity.QueueMetadata; import org.apache.rocketmq.tieredstore.metadata.entity.TopicMetadata; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + public class DefaultMetadataStore extends ConfigManager implements MetadataStore { private static final int DEFAULT_CAPACITY = 1024; @@ -80,12 +81,9 @@ public String encode(boolean prettyFormat) { dataWrapper.setIndexFileSegmentTable(new ConcurrentHashMap<>(indexFileSegmentTable)); if (prettyFormat) { - return JSON.toJSONString( - dataWrapper, SerializerFeature.DisableCircularReferenceDetect, SerializerFeature.PrettyFormat); - } else { - return JSON.toJSONString( - dataWrapper, SerializerFeature.DisableCircularReferenceDetect); + return JSON.toJSONString(dataWrapper, JSONWriter.Feature.PrettyFormat); } + return JSON.toJSONString(dataWrapper); } @Override diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/FileSegmentMetadata.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/FileSegmentMetadata.java index 4f988ca2411..da2a0fd06ce 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/FileSegmentMetadata.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/FileSegmentMetadata.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.tieredstore.metadata.entity; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; import java.util.Objects; public class FileSegmentMetadata { diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/QueueMetadata.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/QueueMetadata.java index 6720f1d08ac..3f976f037dd 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/QueueMetadata.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/QueueMetadata.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.tieredstore.metadata.entity; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; import org.apache.rocketmq.common.message.MessageQueue; public class QueueMetadata { diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/TopicMetadata.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/TopicMetadata.java index 80e5230e7a3..72b994fc84e 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/TopicMetadata.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metadata/entity/TopicMetadata.java @@ -16,7 +16,7 @@ */ package org.apache.rocketmq.tieredstore.metadata.entity; -import com.alibaba.fastjson.annotation.JSONField; +import com.alibaba.fastjson2.annotation.JSONField; public class TopicMetadata { diff --git a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java index 4d083284834..e0ebff08cb0 100644 --- a/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java +++ b/tieredstore/src/main/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManager.java @@ -207,7 +207,7 @@ public static void init(Meter meter, Supplier attributesBuild dispatchLatency = meter.gaugeBuilder(GAUGE_DISPATCH_LATENCY) .setDescription("Tiered store dispatch latency") - .setUnit("seconds") + .setUnit("milliseconds") .ofLongs() .buildWithCallback(measurement -> { for (FlatMessageFile flatFile : flatFileStore.deepCopyFlatFileToList()) { @@ -261,7 +261,7 @@ public static void init(Meter meter, Supplier attributesBuild .ofLongs() .buildWithCallback(measurement -> { if (fetcher instanceof MessageStoreFetcherImpl) { - long count = ((MessageStoreFetcherImpl) fetcher).getFetcherCache().stats().loadCount(); + long count = ((MessageStoreFetcherImpl) fetcher).getFetcherCache().estimatedSize(); measurement.record(count, newAttributesBuilder().build()); } }); @@ -272,8 +272,10 @@ public static void init(Meter meter, Supplier attributesBuild .ofLongs() .buildWithCallback(measurement -> { if (fetcher instanceof MessageStoreFetcherImpl) { - long count = ((MessageStoreFetcherImpl) fetcher).getFetcherCache().estimatedSize(); - measurement.record(count, newAttributesBuilder().build()); + long bytes = ((MessageStoreFetcherImpl) fetcher).getFetcherCache().policy().eviction() + .map(eviction -> eviction.weightedSize().orElse(0L)) + .orElse(0L); + measurement.record(bytes, newAttributesBuilder().build()); } }); diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/index/IndexStoreFileTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/index/IndexStoreFileTest.java index d19b562463d..10014ba76a0 100644 --- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/index/IndexStoreFileTest.java +++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/index/IndexStoreFileTest.java @@ -39,7 +39,13 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import java.util.Arrays; +import java.util.Collection; + +@RunWith(Parameterized.class) public class IndexStoreFileTest { private static final String TOPIC_NAME = "TopicTest"; @@ -50,6 +56,17 @@ public class IndexStoreFileTest { private static final String KEY = "MessageKey"; private static final Set KEY_SET = Collections.singleton(KEY); + @Parameterized.Parameter + public boolean writeWithoutMmap; + + @Parameterized.Parameters(name = "writeWithoutMmap={0}") + public static Collection data() { + return Arrays.asList(new Object[][] { + { true }, + { false } + }); + } + private String filePath; private MessageStoreConfig storeConfig; private IndexStoreFile indexStoreFile; @@ -64,6 +81,7 @@ public void init() throws IOException { storeConfig.setTieredStoreIndexFileMaxHashSlotNum(5); storeConfig.setTieredStoreIndexFileMaxIndexNum(20); storeConfig.setTieredBackendServiceProvider("org.apache.rocketmq.tieredstore.provider.PosixFileSegment"); + storeConfig.setWriteWithoutMmap(writeWithoutMmap); indexStoreFile = new IndexStoreFile(storeConfig, System.currentTimeMillis()); } diff --git a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManagerTest.java b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManagerTest.java index cc4d9e2c68b..04341389610 100644 --- a/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManagerTest.java +++ b/tieredstore/src/test/java/org/apache/rocketmq/tieredstore/metrics/TieredStoreMetricsManagerTest.java @@ -16,13 +16,26 @@ */ package org.apache.rocketmq.tieredstore.metrics; +import com.github.benmanes.caffeine.cache.Cache; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.metrics.DoubleGaugeBuilder; +import io.opentelemetry.api.metrics.Meter; +import io.opentelemetry.api.metrics.ObservableLongGauge; +import io.opentelemetry.api.metrics.ObservableLongMeasurement; import io.opentelemetry.sdk.OpenTelemetrySdk; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import org.apache.rocketmq.common.message.MessageQueue; import org.apache.rocketmq.store.DefaultMessageStore; import org.apache.rocketmq.tieredstore.MessageStoreConfig; import org.apache.rocketmq.tieredstore.TieredMessageStore; +import org.apache.rocketmq.tieredstore.common.SelectBufferResult; import org.apache.rocketmq.tieredstore.core.MessageStoreFetcherImpl; import org.apache.rocketmq.tieredstore.file.FlatFileStore; import org.apache.rocketmq.tieredstore.provider.PosixFileSegment; +import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -52,4 +65,98 @@ public void init() { public void newAttributesBuilder() { TieredStoreMetricsManager.newAttributesBuilder(); } + + @Test + public void testCacheCountMetric() { + MessageStoreConfig storeConfig = new MessageStoreConfig(); + TieredMessageStore messageStore = Mockito.mock(TieredMessageStore.class); + Mockito.when(messageStore.getStoreConfig()).thenReturn(storeConfig); + Mockito.when(messageStore.getFlatFileStore()).thenReturn(Mockito.mock(FlatFileStore.class)); + // The fetcher will create real cache + MessageStoreFetcherImpl fetcher = new MessageStoreFetcherImpl(messageStore); + + AtomicLong capturedCacheCount = new AtomicLong(-1); + Meter mockMeter = createMockMeter(TieredStoreMetricsConstant.GAUGE_CACHE_COUNT, capturedCacheCount); + + // Prepare cache before init so the gauge callback sees a populated cache instead of an empty one. + int[] bufferSizes = prepareTestCache(fetcher); + + TieredStoreMetricsManager.init(mockMeter, + null, storeConfig, fetcher, + Mockito.mock(FlatFileStore.class), Mockito.mock(DefaultMessageStore.class)); + + // CacheCount gauge should report the number of cached entries. + Assert.assertEquals(bufferSizes.length, capturedCacheCount.get()); + } + + @Test + public void testCacheBytesMetric() { + MessageStoreConfig storeConfig = new MessageStoreConfig(); + TieredMessageStore messageStore = Mockito.mock(TieredMessageStore.class); + Mockito.when(messageStore.getStoreConfig()).thenReturn(storeConfig); + Mockito.when(messageStore.getFlatFileStore()).thenReturn(Mockito.mock(FlatFileStore.class)); + // The fetcher will create real cache + MessageStoreFetcherImpl fetcher = new MessageStoreFetcherImpl(messageStore); + + AtomicLong capturedCacheBytes = new AtomicLong(-1); + Meter mockMeter = createMockMeter(TieredStoreMetricsConstant.GAUGE_CACHE_BYTES, capturedCacheBytes); + + // Prepare cache before init so the gauge callback sees a populated cache instead of an empty one. + int[] bufferSizes = prepareTestCache(fetcher); + + TieredStoreMetricsManager.init(mockMeter, + null, storeConfig, fetcher, + Mockito.mock(FlatFileStore.class), Mockito.mock(DefaultMessageStore.class)); + + // CacheBytes gauge should report the sum of all cached buffer sizes. + int expectedSum = Arrays.stream(bufferSizes).sum(); + Assert.assertEquals(expectedSum, capturedCacheBytes.get()); + } + + private Meter createMockMeter(String targetMetricName, AtomicLong capturedValue) { + Meter mockMeter = Mockito.mock(Meter.class, Mockito.RETURNS_DEEP_STUBS); + + // Setup target gauge builder chain to capture the callback value + DoubleGaugeBuilder targetGaugeBuilder = Mockito.mock(DoubleGaugeBuilder.class, Mockito.RETURNS_DEEP_STUBS); + Mockito.when(mockMeter.gaugeBuilder(targetMetricName)).thenReturn(targetGaugeBuilder); + Mockito.when(targetGaugeBuilder.setDescription(Mockito.anyString())).thenReturn(targetGaugeBuilder); + Mockito.when(targetGaugeBuilder.setUnit(Mockito.anyString())).thenReturn(targetGaugeBuilder); + Mockito.when(targetGaugeBuilder.ofLongs().buildWithCallback(Mockito.any(Consumer.class))) + .thenAnswer(invocation -> { + Consumer callback = invocation.getArgument(0); + // Immediately invoke the callback to capture the current cache state + callback.accept(new ObservableLongMeasurement() { + @Override + public void record(long value) { + capturedValue.set(value); + } + + @Override + public void record(long value, Attributes attributes) { + capturedValue.set(value); + } + }); + return Mockito.mock(ObservableLongGauge.class); + }); + + return mockMeter; + } + + private int[] prepareTestCache(MessageStoreFetcherImpl fetcher) { + Cache cache = fetcher.getFetcherCache(); + String topic = "TestTopic"; + MessageQueue mq1 = new MessageQueue(topic, "broker", 0); + MessageQueue mq2 = new MessageQueue(topic, "broker", 1); + + int[] bufferSizes = {100, 200, 150, 300}; + for (int i = 0; i < bufferSizes.length; i++) { + SelectBufferResult result = new SelectBufferResult( + ByteBuffer.allocate(bufferSizes[i]), 0L, bufferSizes[i], 0L); + MessageQueue mq = i < 2 ? mq1 : mq2; + String key = String.format("%s@%d@%d", mq.getTopic(), mq.getQueueId(), (i + 1) * 100L); + cache.put(key, result); + } + return bufferSizes; + } + } diff --git a/tools/BUILD.bazel b/tools/BUILD.bazel index ec6fa1eaacb..a809a7a92e5 100644 --- a/tools/BUILD.bazel +++ b/tools/BUILD.bazel @@ -29,7 +29,6 @@ java_library( "@maven//:commons_validator_commons_validator", "@maven//:com_github_luben_zstd_jni", "@maven//:org_lz4_lz4_java", - "@maven//:com_alibaba_fastjson", "@maven//:io_netty_netty_all", "@maven//:commons_cli_commons_cli", "@maven//:org_slf4j_slf4j_api", diff --git a/tools/pom.xml b/tools/pom.xml index 8eea4977e6e..1d63ded9b86 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -44,10 +44,6 @@ ${project.groupId} rocketmq-srvutil - - com.alibaba - fastjson - org.apache.commons commons-lang3 diff --git a/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java b/tools/src/main/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExt.java index 2e72af13eed..b14b7966dc6 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 @@ -47,6 +47,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; @@ -160,6 +165,11 @@ public QueryResult queryMessage(String clusterName, String topic, String key, in return defaultMQAdminExtImpl.queryMessage(clusterName, topic, key, maxNum, begin, end); } + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, String keyType, String lastKey) + throws MQClientException, InterruptedException, RemotingException { + return defaultMQAdminExtImpl.queryMessage(clusterName, topic, key, maxNum, begin, end, keyType, lastKey); + } + @Override public void start() throws MQClientException { defaultMQAdminExtImpl.start(); @@ -986,4 +996,50 @@ public void exportPopRecords(String brokerAddr, long timeout) throws RemotingCon RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { defaultMQAdminExtImpl.exportPopRecords(brokerAddr, timeout); } + + @Override + public void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, + UnsupportedEncodingException, InterruptedException, MQBrokerException { + defaultMQAdminExtImpl.switchTimerEngine(brokerAddr, desTimerEngine); + } + + @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 7b268cf6947..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 @@ -16,7 +16,26 @@ */ package org.apache.rocketmq.tools.admin; -import com.alibaba.fastjson.JSON; +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,27 +127,6 @@ import org.apache.rocketmq.tools.admin.common.AdminToolsResultCodeEnum; import org.apache.rocketmq.tools.command.CommandUtil; -import java.io.UnsupportedEncodingException; -import java.text.MessageFormat; -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"; @@ -905,7 +908,7 @@ public void run() { resetOffsetByTimestampOld(addr, topicRouteMap.get(bd.getBrokerName()), group, topic, timestamp, true); successList.add(addr); } catch (Exception e2) { - logger.error(MessageFormat.format("resetOffsetByTimestampOld error. addr={0}, topic={1}, group={2},timestamp={3}", addr, topic, group, timestamp), e); + logger.error("resetOffsetByTimestampOld error. addr={}, topic={}, group={}, timestamp={}", addr, topic, group, timestamp, e); failureList.add(addr); } } else if (ResponseCode.SYSTEM_ERROR == e.getResponseCode()) { @@ -913,11 +916,11 @@ public void run() { successList.add(addr); } else { failureList.add(addr); - logger.error(MessageFormat.format("resetOffsetNewConcurrent error. addr={0}, topic={1}, group={2},timestamp={3}", addr, topic, group, timestamp), e); + logger.error("resetOffsetNewConcurrent error. addr={}, topic={}, group={}, timestamp={}", addr, topic, group, timestamp, e); } } catch (Exception e) { failureList.add(addr); - logger.error(MessageFormat.format("resetOffsetNewConcurrent error. addr={0}, topic={1}, group={2},timestamp={3}", addr, topic, group, timestamp), e); + logger.error("resetOffsetNewConcurrent error. addr={}, topic={}, group={}, timestamp={}", addr, topic, group, timestamp, e); } finally { latch.countDown(); } @@ -1739,9 +1742,12 @@ public QueryResult queryMessage(String topic, String key, int maxNum, long begin public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end) throws MQClientException, InterruptedException, RemotingException { - return this.mqClientInstance.getMQAdminImpl().queryMessage(clusterName, topic, key, maxNum, begin, end, false); + return this.mqClientInstance.getMQAdminImpl().queryMessage(clusterName, topic, key, maxNum, begin, end, false, MessageConst.INDEX_KEY_TYPE, null); } + public QueryResult queryMessage(String clusterName, String topic, String key, int maxNum, long begin, long end, String keyType, String lastKey) throws MQClientException, InterruptedException, RemotingException { + return this.mqClientInstance.getMQAdminImpl().queryMessage(clusterName, topic, key, maxNum, begin, end, false, keyType, lastKey); + } @Override public void updateConsumeOffset(String brokerAddr, String consumeGroup, MessageQueue mq, long offset) throws RemotingException, InterruptedException, MQBrokerException { @@ -2053,4 +2059,50 @@ public void exportPopRecords(String brokerAddr, long timeout) throws RemotingCon RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { this.mqClientInstance.getMQClientAPIImpl().exportPopRecord(brokerAddr, timeout); } + + @Override + public void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException { + this.mqClientInstance.getMQClientAPIImpl().switchTimerEngine(brokerAddr, desTimerEngine, timeoutMillis); + } + + + @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 46e2c066cb4..b8be2e232f0 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 @@ -43,6 +43,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; @@ -516,4 +521,27 @@ String setCommitLogReadAheadMode(final String brokerAddr, String mode) void exportPopRecords(String brokerAddr, long timeout) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException; + + void switchTimerEngine(String brokerAddr, String desTimerEngine) throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, UnsupportedEncodingException, InterruptedException, MQBrokerException; + + 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 b210e82b3c8..5d79822cde4 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/MQAdminStartup.java @@ -49,6 +49,7 @@ import org.apache.rocketmq.tools.command.broker.RemoveColdDataFlowCtrGroupConfigSubCommand; import org.apache.rocketmq.tools.command.broker.ResetMasterFlushOffsetSubCommand; import org.apache.rocketmq.tools.command.broker.SendMsgStatusCommand; +import org.apache.rocketmq.tools.command.broker.SwitchTimerEngineSubCommand; import org.apache.rocketmq.tools.command.broker.UpdateBrokerConfigSubCommand; import org.apache.rocketmq.tools.command.broker.UpdateColdDataFlowCtrGroupConfigSubCommand; import org.apache.rocketmq.tools.command.cluster.CLusterSendMsgRTCommand; @@ -77,6 +78,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; @@ -298,6 +305,15 @@ public static void initCommand() { initCommand(new CopyAclsSubCommand()); 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/broker/GetBrokerConfigCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java index c4762a29606..d9333f339cd 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommand.java @@ -25,6 +25,7 @@ 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.client.exception.MQBrokerException; import org.apache.rocketmq.remoting.RPCHook; @@ -50,13 +51,11 @@ public String commandDesc() { @Override public Options buildCommandlineOptions(final Options options) { - Option opt = new Option("b", "brokerAddr", true, "get which broker"); - opt.setRequired(false); - options.addOption(opt); - - opt = new Option("c", "clusterName", true, "get which cluster"); - opt.setRequired(false); - options.addOption(opt); + OptionGroup group = new OptionGroup(); + group.addOption(new Option("b", "brokerAddr", true, "get which broker")); + group.addOption(new Option("c", "clusterName", true, "get which cluster")); + group.setRequired(true); + options.addOptionGroup(group); return options; } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java index 34b3ba7d306..76c111b95c1 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/GetColdDataFlowCtrInfoSubCommand.java @@ -16,13 +16,9 @@ */ package org.apache.rocketmq.tools.command.broker; -import java.io.UnsupportedEncodingException; -import java.text.SimpleDateFormat; -import java.util.Date; -import java.util.List; -import java.util.Map; -import com.alibaba.fastjson.JSON; -import com.alibaba.fastjson.JSONObject; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -37,6 +33,12 @@ import org.apache.rocketmq.tools.command.SubCommand; import org.apache.rocketmq.tools.command.SubCommandException; +import java.io.UnsupportedEncodingException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.List; +import java.util.Map; + public class GetColdDataFlowCtrInfoSubCommand implements SubCommand { SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); @@ -116,7 +118,7 @@ protected void getAndPrint(final MQAdminExt defaultMQAdminExt, final String prin value.remove("createTimeMills"); }); - String formatStr = JSON.toJSONString(jsonObject, true); + String formatStr = JSON.toJSONString(jsonObject, JSONWriter.Feature.PrettyFormat); System.out.printf(formatStr); System.out.printf("%n"); } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/broker/SwitchTimerEngineSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/SwitchTimerEngineSubCommand.java new file mode 100644 index 00000000000..fbddca1b967 --- /dev/null +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/broker/SwitchTimerEngineSubCommand.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.rocketmq.tools.command.broker; +import java.util.Set; +import org.apache.commons.cli.CommandLine; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; +import org.apache.rocketmq.common.message.MessageConst; +import org.apache.rocketmq.remoting.RPCHook; +import org.apache.rocketmq.srvutil.ServerUtil; +import org.apache.rocketmq.tools.admin.DefaultMQAdminExt; +import org.apache.rocketmq.tools.command.CommandUtil; +import org.apache.rocketmq.tools.command.SubCommand; +import org.apache.rocketmq.tools.command.SubCommandException; + +public class SwitchTimerEngineSubCommand implements SubCommand { + private static final String ROCKSDB_TIMELINE = "ROCKSDB_TIMELINE"; + private static final String FILE_TIME_WHEEL = "FILE_TIME_WHEEL"; + + @Override + public String commandName() { + return "switchTimerEngine"; + } + + @Override + public String commandDesc() { + return "switch the engine of timer message in broker"; + } + + @Override + public Options buildCommandlineOptions(Options options) { + Option opt = new Option("b", "brokerAddr", true, "update which broker"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("c", "clusterName", true, "update which cluster"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("e", "engineType", true, "R/F, R for rocksdb timeline engine, F for file time wheel engine"); + opt.setRequired(true); + options.addOption(opt); + return options; + } + + @Override + public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) throws SubCommandException { + + DefaultMQAdminExt defaultMQAdminExt = new DefaultMQAdminExt(rpcHook); + defaultMQAdminExt.setInstanceName(Long.toString(System.currentTimeMillis())); + try { + String engineType = commandLine.getOptionValue('e').trim(); + if (StringUtils.isEmpty(engineType) || !MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType) && !MessageConst.TIMER_ENGINE_FILE_TIME_WHEEL.equals(engineType)) { + System.out.print("switchTimerEngine engineType must be R or F\n"); + return; + } + if (commandLine.hasOption('b')) { + String brokerAddr = commandLine.getOptionValue('b').trim(); + defaultMQAdminExt.start(); + defaultMQAdminExt.switchTimerEngine(brokerAddr, engineType); + System.out.printf("switchTimerEngine to %s success, %s\n", engineType, brokerAddr); + return; + } else if (commandLine.hasOption('c')) { + String clusterName = commandLine.getOptionValue('c').trim(); + defaultMQAdminExt.start(); + Set masterSet = CommandUtil.fetchMasterAddrByClusterName(defaultMQAdminExt, clusterName); + for (String brokerAddr : masterSet) { + try { + defaultMQAdminExt.switchTimerEngine(brokerAddr, engineType); + String engineName = MessageConst.TIMER_ENGINE_ROCKSDB_TIMELINE.equals(engineType) ? ROCKSDB_TIMELINE : FILE_TIME_WHEEL; + System.out.printf("switchTimerEngine to %s success, %s\n", engineName, brokerAddr); + } catch (Exception e) { + e.printStackTrace(); + } + } + return; + } + ServerUtil.printCommandLineHelp("mqadmin " + this.commandName(), options); + } catch (Exception e) { + throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); + } finally { + defaultMQAdminExt.shutdown(); + } + + } +} diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java index b17da4de45f..d4782a06848 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/consumer/UpdateSubGroupSubCommand.java @@ -16,9 +16,7 @@ */ package org.apache.rocketmq.tools.command.consumer; -import com.alibaba.fastjson.JSON; -import java.util.Map; -import java.util.Set; +import com.alibaba.fastjson2.JSON; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -32,6 +30,9 @@ import org.apache.rocketmq.tools.command.SubCommand; import org.apache.rocketmq.tools.command.SubCommandException; +import java.util.Map; +import java.util.Set; + public class UpdateSubGroupSubCommand implements SubCommand { @Override diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java index c3f96d59723..ebb1cf82baa 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportConfigsCommand.java @@ -16,15 +16,8 @@ */ package org.apache.rocketmq.tools.command.export; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Arrays; -import java.util.Properties; - -import com.alibaba.fastjson.JSON; - +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -35,6 +28,13 @@ import org.apache.rocketmq.tools.command.SubCommand; import org.apache.rocketmq.tools.command.SubCommandException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + public class ExportConfigsCommand implements SubCommand { @Override public String commandName() { @@ -98,7 +98,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) result.put("clusterScale", clusterScaleMap); String path = filePath + "/configs.json"; - MixAll.string2FileNotSafe(JSON.toJSONString(result, true), path); + MixAll.string2FileNotSafe(JSON.toJSONString(result, JSONWriter.Feature.PrettyFormat), path); System.out.printf("export %s success", path); } catch (Exception e) { throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java index 748f7b16e12..559936bcb8d 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataCommand.java @@ -16,10 +16,8 @@ */ package org.apache.rocketmq.tools.command.export; -import com.alibaba.fastjson.JSON; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -35,6 +33,10 @@ import org.apache.rocketmq.tools.command.SubCommand; import org.apache.rocketmq.tools.command.SubCommandException; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + public class ExportMetadataCommand implements SubCommand { private static final String DEFAULT_FILE_PATH = "/tmp/rocketmq/export"; @@ -99,13 +101,13 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) filePath = filePath + "/topic.json"; TopicConfigSerializeWrapper topicConfigSerializeWrapper = defaultMQAdminExt.getUserTopicConfig( brokerAddr, specialTopic, 10000L); - MixAll.string2FileNotSafe(JSON.toJSONString(topicConfigSerializeWrapper, true), filePath); + MixAll.string2FileNotSafe(JSON.toJSONString(topicConfigSerializeWrapper, JSONWriter.Feature.PrettyFormat), filePath); System.out.printf("export %s success", filePath); } else if (commandLine.hasOption('g')) { filePath = filePath + "/subscriptionGroup.json"; SubscriptionGroupWrapper subscriptionGroupWrapper = defaultMQAdminExt.getUserSubscriptionGroup( brokerAddr, 10000L); - MixAll.string2FileNotSafe(JSON.toJSONString(subscriptionGroupWrapper, true), filePath); + MixAll.string2FileNotSafe(JSON.toJSONString(subscriptionGroupWrapper, JSONWriter.Feature.PrettyFormat), filePath); System.out.printf("export %s success", filePath); } } else if (commandLine.hasOption('c')) { @@ -163,7 +165,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) exportPath = filePath + "/metadata.json"; } result.put("exportTime", System.currentTimeMillis()); - MixAll.string2FileNotSafe(JSON.toJSONString(result, true), exportPath); + MixAll.string2FileNotSafe(JSON.toJSONString(result, JSONWriter.Feature.PrettyFormat), exportPath); System.out.printf("export %s success%n", exportPath); } else { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java index 438d17d6689..1a3e5e5a6af 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetadataInRocksDBCommand.java @@ -17,11 +17,9 @@ package org.apache.rocketmq.tools.command.export; -import com.alibaba.fastjson.JSONObject; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.BiConsumer; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONObject; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -34,6 +32,11 @@ import org.apache.rocketmq.tools.command.SubCommandException; import org.rocksdb.RocksIterator; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; + public class ExportMetadataInRocksDBCommand implements SubCommand { private static final String TOPICS_JSON_CONFIG = "topics"; private static final String SUBSCRIPTION_GROUP_JSON_CONFIG = "subscriptionGroups"; @@ -118,8 +121,8 @@ private static void handleExportMetadata(ConfigRocksDBStorage kvStore, String co ); jsonConfig.put(configType.equalsIgnoreCase(TOPICS_JSON_CONFIG) ? "topicConfigTable" : "subscriptionGroupTable", - (JSONObject) JSONObject.toJSON(configTable)); - final String jsonConfigStr = JSONObject.toJSONString(jsonConfig, true); + (JSONObject) JSON.toJSON(configTable)); + final String jsonConfigStr = JSONObject.toJSONString(jsonConfig, JSONWriter.Feature.PrettyFormat); System.out.print(jsonConfigStr + "\n"); } else { AtomicLong count = new AtomicLong(0); diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java index 5d8bb37ba0f..1b7d5d44e38 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/export/ExportMetricsCommand.java @@ -16,12 +16,8 @@ */ package org.apache.rocketmq.tools.command.export; -import com.alibaba.fastjson.JSON; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Properties; -import java.util.Set; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -47,6 +43,12 @@ import org.apache.rocketmq.tools.command.SubCommandException; import org.apache.rocketmq.tools.command.stats.StatsAllSubCommand; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + public class ExportMetricsCommand implements SubCommand { @Override @@ -133,7 +135,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) result.put("evaluateReport", evaluateReportMap); result.put("totalData", totalData); - MixAll.string2FileNotSafe(JSON.toJSONString(result, true), path); + MixAll.string2FileNotSafe(JSON.toJSONString(result, JSONWriter.Feature.PrettyFormat), path); System.out.printf("export %s success", path); } catch (Exception e) { throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); 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/QueryMsgByKeySubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java index 02961c3bb50..ceef631c3f0 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByKeySubCommand.java @@ -19,8 +19,11 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.MixAll; +import org.apache.rocketmq.common.message.MessageConst; import org.apache.rocketmq.common.message.MessageExt; import org.apache.rocketmq.remoting.RPCHook; import org.apache.rocketmq.remoting.exception.RemotingException; @@ -67,6 +70,14 @@ public Options buildCommandlineOptions(Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("p", "keyType", true, "Index key type, default index key type is K, you can use K for keys OR T for tags"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("l", "lastKey", true, "Last Key"); + opt.setRequired(false); + options.addOption(opt); + return options; } @@ -79,7 +90,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t try { final String topic = commandLine.getOptionValue('t').trim(); final String key = commandLine.getOptionValue('k').trim(); - + String keyType = MessageConst.INDEX_KEY_TYPE; + String lastKey = null; long beginTimestamp = 0; long endTimestamp = Long.MAX_VALUE; int maxNum = 64; @@ -96,7 +108,17 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t if (commandLine.hasOption("c")) { clusterName = commandLine.getOptionValue("c").trim(); } - this.queryByKey(defaultMQAdminExt, clusterName, topic, key, maxNum, beginTimestamp, endTimestamp); + if (commandLine.hasOption("p")) { + keyType = commandLine.getOptionValue("p").trim(); + if (StringUtils.isEmpty(keyType) || !MessageConst.INDEX_KEY_TYPE.equals(keyType) && !MessageConst.INDEX_TAG_TYPE.equals(keyType)) { + System.out.printf("index type error, just support K for keys or T for tags"); + return; + } + } + if (commandLine.hasOption("l")) { + lastKey = commandLine.getOptionValue("l").trim(); + } + this.queryByKey(defaultMQAdminExt, clusterName, topic, key, maxNum, beginTimestamp, endTimestamp, keyType, lastKey); } catch (Exception e) { throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); } finally { @@ -105,18 +127,23 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t } private void queryByKey(final DefaultMQAdminExt admin, final String cluster, final String topic, final String key, int maxNum, long begin, - long end) + long end, String keyType, String lastKey) throws MQClientException, InterruptedException, RemotingException { admin.start(); - - QueryResult queryResult = admin.queryMessage(cluster, topic, key, maxNum, begin, end); - - System.out.printf("%-50s %4s %40s%n", + QueryResult queryResult = admin.queryMessage(cluster, topic, key, maxNum, begin, end, keyType, lastKey); + System.out.printf("%-50s %4s %40s %-200s%n", "#Message ID", "#QID", - "#Offset"); + "#Offset", + "#IndexKey"); for (MessageExt msg : queryResult.getMessageList()) { - System.out.printf("%-50s %4d %40d%n", msg.getMsgId(), msg.getQueueId(), msg.getQueueOffset()); + if (!StringUtils.isEmpty(keyType)) { + long storeTimestamp = MixAll.dealTimeToHourStamps(msg.getStoreTimestamp()); + String indexLastKey = storeTimestamp + "@" + topic + "@" + keyType + "@" + key + "@" + msg.getMsgId() + "@" + msg.getCommitLogOffset(); + System.out.printf("%-50s %4d %40d %-200s%n", msg.getMsgId(), msg.getQueueId(), msg.getQueueOffset(), indexLastKey); + } else { + System.out.printf("%-50s %4d %40d%n", msg.getMsgId(), msg.getQueueId(), msg.getQueueOffset()); + } } } } diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/message/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/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java index 5295d91cc30..8518a04e682 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/message/QueryMsgByUniqueKeySubCommand.java @@ -24,6 +24,7 @@ import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; +import org.apache.commons.lang3.StringUtils; import org.apache.rocketmq.client.QueryResult; import org.apache.rocketmq.client.exception.MQClientException; import org.apache.rocketmq.common.UtilAll; @@ -56,11 +57,17 @@ private DefaultMQAdminExt createMQAdminExt(RPCHook rpcHook) throws SubCommandExc } } - public static void queryById(final DefaultMQAdminExt admin, final String clusterName, final String topic, - final String msgId, - final boolean showAll) throws MQClientException, InterruptedException, IOException { - - QueryResult queryResult = admin.queryMessageByUniqKey(clusterName, topic, msgId, 32, 0, Long.MAX_VALUE); + public static void queryById(final DefaultMQAdminExt admin, final String clusterName, final String topic, final String msgId, final boolean showAll, final String startTime, final String endTime) throws MQClientException, InterruptedException, IOException { + QueryResult queryResult = null; + if (!StringUtils.isEmpty(startTime) && !StringUtils.isEmpty(endTime)) { + Long startTimelong = Long.valueOf(startTime); + Long endTimelong = Long.valueOf(endTime); + if (null != startTimelong && null != endTimelong) { + queryResult = admin.queryMessageByUniqKey(clusterName, topic, msgId, 32, startTimelong, endTimelong); + } + } else { + queryResult = admin.queryMessageByUniqKey(clusterName, topic, msgId, 32, System.currentTimeMillis() - 36 * 60 * 60 * 1000, System.currentTimeMillis() + 36 * 60 * 60 * 1000); + } assert queryResult != null; List list = queryResult.getMessageList(); if (list == null || list.size() == 0) { @@ -167,6 +174,14 @@ public Options buildCommandlineOptions(Options options) { opt.setRequired(false); options.addOption(opt); + opt = new Option("s", "startTime", true, "startTime"); + opt.setRequired(false); + options.addOption(opt); + + opt = new Option("e", "endTime", true, "endTime"); + opt.setRequired(false); + options.addOption(opt); + return options; } @@ -179,6 +194,8 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t final String msgId = commandLine.getOptionValue('i').trim(); final String topic = commandLine.getOptionValue('t').trim(); String clusterName = commandLine.hasOption('c') ? commandLine.getOptionValue('c').trim() : null; + String startTime = commandLine.hasOption('s') ? commandLine.getOptionValue('s').trim() : null; + String endTime = commandLine.hasOption('e') ? commandLine.getOptionValue('e').trim() : null; final boolean showAll = commandLine.hasOption('a'); if (commandLine.hasOption('g') && commandLine.hasOption('d')) { final String consumerGroup = commandLine.getOptionValue('g').trim(); @@ -198,7 +215,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) t } } else { - queryById(defaultMQAdminExt, clusterName, topic, msgId, showAll); + queryById(defaultMQAdminExt, clusterName, topic, msgId, showAll, startTime, endTime); } } catch (Exception e) { throw new SubCommandException(this.getClass().getSimpleName() + " command failed", e); diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java index 94899fce086..d65a04136b3 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/metadata/RocksDBConfigToJsonCommand.java @@ -17,22 +17,8 @@ package org.apache.rocketmq.tools.command.metadata; -import com.alibaba.fastjson.JSONObject; -import java.io.File; -import java.io.IOException; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicLong; +import com.alibaba.fastjson2.JSONObject; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.Option; import org.apache.commons.cli.Options; @@ -49,6 +35,22 @@ import org.apache.rocketmq.tools.command.SubCommandException; import org.rocksdb.RocksIterator; +import java.io.File; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicLong; + public class RocksDBConfigToJsonCommand implements SubCommand { @Override @@ -136,10 +138,10 @@ private void handleLocalMode(CommandLine commandLine) { if (commandLine.hasOption("jsonEnable") && "false".equalsIgnoreCase(commandLine.getOptionValue("jsonEnable").trim())) { printConfigMapJsonDisable(configMap); } else { - System.out.print(JSONObject.toJSONString(configMap, true) + "\n"); + System.out.print(JSONObject.toJSONString(configMap, JSONWriter.Feature.PrettyFormat) + "\n"); } } else { - String jsonString = JSONObject.toJSONString(configMap, true); + String jsonString = JSONObject.toJSONString(configMap, JSONWriter.Feature.PrettyFormat); try { MixAll.string2File(jsonString, exportFile); } catch (IOException e) { diff --git a/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java b/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java index 4902b8a881e..24d9900b611 100644 --- a/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java +++ b/tools/src/main/java/org/apache/rocketmq/tools/command/queue/QueryConsumeQueueCommand.java @@ -17,7 +17,8 @@ package org.apache.rocketmq.tools.command.queue; -import com.alibaba.fastjson.JSON; +import com.alibaba.fastjson2.JSON; +import com.alibaba.fastjson2.JSONWriter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Option; @@ -121,7 +122,7 @@ public void execute(CommandLine commandLine, Options options, RPCHook rpcHook) { ); if (queryConsumeQueueResponseBody.getSubscriptionData() != null) { - System.out.printf("Subscription data: \n%s\n", JSON.toJSONString(queryConsumeQueueResponseBody.getSubscriptionData(), true)); + System.out.printf("Subscription data: \n%s\n", JSON.toJSONString(queryConsumeQueueResponseBody.getSubscriptionData(), JSONWriter.Feature.PrettyFormat)); System.out.print("======================================\n"); } diff --git a/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImplTest.java b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImplTest.java new file mode 100644 index 00000000000..c5ea051ab46 --- /dev/null +++ b/tools/src/test/java/org/apache/rocketmq/tools/admin/DefaultMQAdminExtImplTest.java @@ -0,0 +1,825 @@ +/* + * 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.admin; + +import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.impl.MQAdminImpl; +import org.apache.rocketmq.client.impl.MQClientAPIImpl; +import org.apache.rocketmq.client.impl.factory.MQClientInstance; +import org.apache.rocketmq.common.TopicConfig; +import org.apache.rocketmq.common.constant.PermName; +import org.apache.rocketmq.common.message.MessageExt; +import org.apache.rocketmq.common.message.MessageQueue; +import org.apache.rocketmq.remoting.exception.RemotingConnectException; +import org.apache.rocketmq.remoting.exception.RemotingException; +import org.apache.rocketmq.remoting.exception.RemotingSendRequestException; +import org.apache.rocketmq.remoting.exception.RemotingTimeoutException; +import org.apache.rocketmq.remoting.protocol.ResponseCode; +import org.apache.rocketmq.remoting.protocol.admin.ConsumeStats; +import org.apache.rocketmq.remoting.protocol.admin.OffsetWrapper; +import org.apache.rocketmq.remoting.protocol.admin.RollbackStats; +import org.apache.rocketmq.remoting.protocol.admin.TopicOffset; +import org.apache.rocketmq.remoting.protocol.admin.TopicStatsTable; +import org.apache.rocketmq.remoting.protocol.body.ClusterInfo; +import org.apache.rocketmq.remoting.protocol.body.ConsumeMessageDirectlyResult; +import org.apache.rocketmq.remoting.protocol.body.ConsumerConnection; +import org.apache.rocketmq.remoting.protocol.body.GroupList; +import org.apache.rocketmq.remoting.protocol.body.QueueTimeSpan; +import org.apache.rocketmq.remoting.protocol.body.SubscriptionGroupWrapper; +import org.apache.rocketmq.remoting.protocol.body.TopicConfigSerializeWrapper; +import org.apache.rocketmq.remoting.protocol.body.TopicList; +import org.apache.rocketmq.remoting.protocol.header.UpdateConsumerOffsetRequestHeader; +import org.apache.rocketmq.remoting.protocol.header.UpdateGroupForbiddenRequestHeader; +import org.apache.rocketmq.remoting.protocol.heartbeat.SubscriptionData; +import org.apache.rocketmq.remoting.protocol.route.BrokerData; +import org.apache.rocketmq.remoting.protocol.route.QueueData; +import org.apache.rocketmq.remoting.protocol.route.TopicRouteData; +import org.apache.rocketmq.remoting.protocol.subscription.GroupForbidden; +import org.apache.rocketmq.remoting.protocol.subscription.SubscriptionGroupConfig; +import org.apache.rocketmq.tools.admin.api.BrokerOperatorResult; +import org.apache.rocketmq.tools.admin.api.MessageTrack; +import org.apache.rocketmq.tools.admin.common.AdminToolResult; +import org.apache.rocketmq.tools.admin.common.AdminToolsResultCodeEnum; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; + +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +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.assertThrows; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(MockitoJUnitRunner.class) +public class DefaultMQAdminExtImplTest { + + private DefaultMQAdminExtImpl defaultMQAdminExtImpl; + + @Mock + private DefaultMQAdminExt defaultMQAdminExt; + + @Mock + private MQClientInstance mqClientInstance; + + @Mock + private MQClientAPIImpl mqClientAPIImpl; + + @Mock + private MQAdminImpl mqAdminImpl; + + private final String defaultTopic = "defaultTopic"; + + private final String defaultCluster = "cluster"; + + private final String defaultBroker = "broker1"; + + private final String defaultGroup = "consumerGroup"; + + private final String defaultBrokerAddr = "127.0.0.1:10911"; + + private final long timeoutMillis = 3000L; + + private final String defaultMsgId = "AC1A43AC00002A9F00008F214319C26B"; + + @Before + public void init() throws IllegalAccessException, RemotingException, InterruptedException, MQClientException, MQBrokerException { + defaultMQAdminExtImpl = new DefaultMQAdminExtImpl(defaultMQAdminExt, timeoutMillis); + FieldUtils.writeDeclaredField(defaultMQAdminExtImpl, "mqClientInstance", mqClientInstance, true); + FieldUtils.writeDeclaredField(defaultMQAdminExtImpl, "mqClientInstance", mqClientInstance, true); + FieldUtils.writeDeclaredField(defaultMQAdminExtImpl, "threadPoolExecutor", Executors.newFixedThreadPool(1), true); + when(mqClientInstance.getMQClientAPIImpl()).thenReturn(mqClientAPIImpl); + when(mqClientInstance.getMQAdminImpl()).thenReturn(mqAdminImpl); + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(createTopicRouteData()); + } + + @Test + public void testExamineTopicStats() throws Exception { + TopicStatsTable topicStatsTable = mock(TopicStatsTable.class); + Map offsetTable = new ConcurrentHashMap<>(); + offsetTable.put(new MessageQueue(), new TopicOffset()); + when(topicStatsTable.getOffsetTable()).thenReturn(offsetTable); + when(mqClientAPIImpl.getTopicStatsInfo(any(), any(), anyLong())).thenReturn(topicStatsTable); + TopicStatsTable actual = defaultMQAdminExtImpl.examineTopicStats(defaultTopic); + assertNotNull(actual); + assertEquals(offsetTable.size(), actual.getOffsetTable().size()); + } + + @Test + public void testExamineTopicStatsConcurrentTopicRouteDataNull() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(null); + AdminToolResult actual = defaultMQAdminExtImpl.examineTopicStatsConcurrent(defaultTopic); + assertNotNull(actual); + assertEquals(200, actual.getCode()); + assertEquals(0, actual.getData().getOffsetTable().size()); + } + + @Test + public void testExamineTopicStatsConcurrentBrokerDataEmpty() throws Exception { + TopicRouteData topicRouteData = mock(TopicRouteData.class); + when(topicRouteData.getBrokerDatas()).thenReturn(new ArrayList<>()); + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(topicRouteData); + AdminToolResult actual = defaultMQAdminExtImpl.examineTopicStatsConcurrent(defaultTopic); + assertNotNull(actual); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), actual.getCode()); + assertEquals(0, actual.getData().getOffsetTable().size()); + } + + @Test + public void testExamineTopicStatsConcurrent() throws Exception { + CountDownLatch latch = new CountDownLatch(1); + doAnswer(invocation -> { + latch.countDown(); + return null; + }).when(mqClientAPIImpl).getTopicStatsInfo(any(), any(), anyLong()); + latch.await(1000, TimeUnit.MILLISECONDS); + AdminToolResult actual = defaultMQAdminExtImpl.examineTopicStatsConcurrent(defaultTopic); + assertNotNull(actual); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), actual.getCode()); + assertEquals(0, actual.getData().getOffsetTable().size()); + } + + @Test + public void testExamineTopicStatsConcurrentException() throws Exception { + doThrow(new MQBrokerException(ResponseCode.SYSTEM_ERROR, "Test Exception")).when(mqClientAPIImpl).getTopicStatsInfo(any(), any(), anyLong()); + assertNotNull(defaultMQAdminExtImpl.examineTopicStatsConcurrent(defaultTopic)); + } + + @Test + public void testExamineConsumeStatsConcurrentTopicRouteInfoNotExist() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(null); + AdminToolResult result = defaultMQAdminExtImpl.examineConsumeStatsConcurrent(defaultGroup, defaultTopic); + assertEquals(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST.getCode(), result.getCode()); + } + + @Test + public void testExamineConsumeStatsConcurrent() throws Exception { + AtomicInteger count = new AtomicInteger(0); + AtomicInteger success = new AtomicInteger(0); + AtomicInteger fail = new AtomicInteger(0); + CountDownLatch latch = new CountDownLatch(10); + ExecutorService executorService = Executors.newFixedThreadPool(10); + List brokerDataList = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + BrokerData bd = new BrokerData(); + bd.setBrokerName("brokerName" + i); + bd.setCluster(defaultCluster); + bd.setBrokerAddrs(createBrokerAddrs()); + brokerDataList.add(bd); + } + TopicRouteData topicRouteData = new TopicRouteData(); + topicRouteData.setBrokerDatas(brokerDataList); + for (int i = 0; i < 10; i++) { + executorService.submit(() -> { + try { + Thread.sleep(100); + if (count.incrementAndGet() % 2 == 0) { + success.incrementAndGet(); + } else { + throw new RemotingException("Test Exception"); + } + latch.countDown(); + } catch (Exception e) { + fail.incrementAndGet(); + } + }); + } + latch.await(3000, TimeUnit.MILLISECONDS); + executorService.shutdown(); + assertEquals(5, success.get()); + assertEquals(5, fail.get()); + assertEquals(10, count.get()); + } + + @Test + public void testExamineConsumeStatsConcurrentEmptyOffsetTable() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(createTopicRouteData()); + when(mqClientAPIImpl.getConsumeStats(anyString(), anyString(), anyString(), anyLong())).thenReturn(new ConsumeStats()); + AdminToolResult actual = defaultMQAdminExtImpl.examineConsumeStatsConcurrent(defaultGroup, defaultTopic); + assertEquals(AdminToolsResultCodeEnum.CONSUMER_NOT_ONLINE.getCode(), actual.getCode()); + } + + @Test + public void testViewMessageValidMsgIdReturnsMessageExt() throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + MessageExt expected = createMessageExt(); + when(mqAdminImpl.viewMessage(anyString(), anyString())).thenReturn(expected); + when(mqClientInstance.getMQAdminImpl()).thenReturn(mqAdminImpl); + MessageExt actual = defaultMQAdminExtImpl.viewMessage(expected.getTopic(), expected.getMsgId()); + assertNotNull(actual); + assertEquals(expected.getMsgId(), actual.getMsgId()); + assertEquals(expected.getTopic(), actual.getTopic()); + } + + @Test + public void testViewMessageInvalidMsgIdQueriesByUniqKey() throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + MessageExt expected = createMessageExt(); + expected.setMsgId("invalidMsgId"); + when(mqAdminImpl.queryMessageByUniqKey(anyString(), anyString())).thenReturn(expected); + when(mqClientInstance.getMQAdminImpl()).thenReturn(mqAdminImpl); + MessageExt actual = defaultMQAdminExtImpl.viewMessage(expected.getTopic(), expected.getMsgId()); + assertNotNull(actual); + assertEquals(expected.getMsgId(), actual.getMsgId()); + assertEquals(expected.getTopic(), actual.getTopic()); + } + + @Test + public void testViewMessageExceptionInDecodeLogsWarningAndQueriesByUniqKey() throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + MessageExt expected = createMessageExt(); + expected.setMsgId("exceptionMsgId"); + when(mqAdminImpl.queryMessageByUniqKey(anyString(), anyString())).thenReturn(expected); + MessageExt actual = defaultMQAdminExtImpl.viewMessage(expected.getTopic(), expected.getMsgId()); + assertNotNull(actual); + assertEquals(expected.getMsgId(), actual.getMsgId()); + assertEquals(expected.getTopic(), actual.getTopic()); + } + + @Test + public void testQueryMessageInvalidMsgIdReturnsMessageExt() throws Exception { + MessageExt expected = createMessageExt(); + expected.setMsgId("invalidMsgId"); + when(mqAdminImpl.queryMessageByUniqKey(anyString(), anyString(), anyString())).thenReturn(expected); + MessageExt actual = defaultMQAdminExtImpl.queryMessage(defaultCluster, expected.getTopic(), expected.getMsgId()); + assertNotNull(actual); + assertEquals(expected.getMsgId(), actual.getMsgId()); + assertEquals(expected.getTopic(), actual.getTopic()); + } + + @Test + public void testQueryMessageRemotingException() throws MQBrokerException, RemotingException, InterruptedException, MQClientException { + when(mqAdminImpl.viewMessage(anyString(), anyString())).thenThrow(new RemotingException("Test Exception")); + assertNull(defaultMQAdminExtImpl.queryMessage(null, defaultTopic, defaultMsgId)); + } + + @Test + public void testDeleteTopicValidInput() throws Exception { + ClusterInfo clusterInfo = mock(ClusterInfo.class); + when(defaultMQAdminExt.examineBrokerClusterInfo()).thenReturn(clusterInfo); + Map> clusterAddrTable = new HashMap<>(); + clusterAddrTable.put(defaultCluster, new HashSet<>(Arrays.asList("broker1", "broker2"))); + when(clusterInfo.getClusterAddrTable()).thenReturn(clusterAddrTable); + Map brokerAddrTable = new HashMap<>(); + BrokerData brokerData = new BrokerData(); + brokerData.setBrokerName(defaultBroker); + HashMap brokerAddrs = new HashMap<>(); + brokerAddrs.put(0L, defaultBrokerAddr); + brokerData.setBrokerAddrs(brokerAddrs); + brokerAddrTable.put(defaultBroker, brokerData); + when(clusterInfo.getBrokerAddrTable()).thenReturn(brokerAddrTable); + List nsAddrs = new ArrayList<>(); + nsAddrs.add("127.0.0.1:9876"); + when(mqClientAPIImpl.getNameServerAddressList()).thenReturn(nsAddrs); + List kvNamespaceToDeleteList = new ArrayList<>(); + kvNamespaceToDeleteList.add("namespace"); + FieldUtils.writeDeclaredField(defaultMQAdminExtImpl, "kvNamespaceToDeleteList", kvNamespaceToDeleteList, true); + defaultMQAdminExtImpl.deleteTopic(defaultTopic, defaultCluster); + verify(mqClientAPIImpl, times(1)).deleteTopicInBroker(any(), any(), anyLong()); + verify(mqClientAPIImpl, times(1)).deleteTopicInNameServer(any(), any(), anyLong()); + verify(mqClientAPIImpl, times(1)).deleteKVConfigValue(any(), any(), anyLong()); + } + + @Test + public void testDeleteTopicInBrokerConcurrent() throws InterruptedException, RemotingException, MQClientException { + Set addrs = Collections.singleton(defaultBrokerAddr); + doNothing().when(mqClientAPIImpl).deleteTopicInBroker(anyString(), anyString(), anyLong()); + AdminToolResult result = defaultMQAdminExtImpl.deleteTopicInBrokerConcurrent(addrs, defaultTopic); + assertNotNull(result); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), result.getCode()); + BrokerOperatorResult brokerResult = result.getData(); + List successList = brokerResult.getSuccessList(); + List failureList = brokerResult.getFailureList(); + assertEquals(1, successList.size()); + assertEquals(0, failureList.size()); + assertEquals(addrs.iterator().next(), successList.get(0)); + } + + @Test + public void testDeleteTopicInBrokerConcurrentAllFailures() throws InterruptedException, RemotingException, MQClientException { + Set addrs = new HashSet<>(Collections.singleton(defaultBrokerAddr)); + String anotherAddr = "anotherBrokerAddr:10911"; + addrs.add(anotherAddr); + doThrow(new RuntimeException("deleteTopic error")).when(mqClientAPIImpl).deleteTopicInBroker(anyString(), anyString(), anyLong()); + AdminToolResult result = defaultMQAdminExtImpl.deleteTopicInBrokerConcurrent(addrs, defaultTopic); + assertNotNull(result); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), result.getCode()); + BrokerOperatorResult brokerResult = result.getData(); + List successList = brokerResult.getSuccessList(); + List failureList = brokerResult.getFailureList(); + assertEquals(0, successList.size()); + assertEquals(2, failureList.size()); + assertTrue(failureList.contains(defaultBrokerAddr)); + assertTrue(failureList.contains(anotherAddr)); + } + + @Test + public void testResetOffsetByTimestampOldThrowException() { + String topic = "nonExistentTopic"; + long timestamp = System.currentTimeMillis(); + assertThrows(NullPointerException.class, () -> defaultMQAdminExtImpl.resetOffsetByTimestampOld(defaultGroup, topic, timestamp, false)); + } + + @Test + public void testResetOffsetByTimestampOldValidInputShouldProcessCorrectly() throws Exception { + long timestamp = System.currentTimeMillis(); + ConsumeStats consumeStats = mock(ConsumeStats.class); + Map offsetTable = new ConcurrentHashMap<>(); + OffsetWrapper offsetWrapper = new OffsetWrapper(); + offsetWrapper.setBrokerOffset(5L); + offsetWrapper.setConsumerOffset(5L); + offsetTable.put(new MessageQueue(defaultTopic, defaultBroker, 0), offsetWrapper); + when(consumeStats.getOffsetTable()).thenReturn(offsetTable); + when(mqClientAPIImpl.getConsumeStats(any(), any(), anyLong())).thenReturn(consumeStats); + List rollbackStatsList = defaultMQAdminExtImpl.resetOffsetByTimestampOld(defaultGroup, defaultTopic, timestamp, false); + assertNotNull(rollbackStatsList); + assertEquals(1, rollbackStatsList.size()); + RollbackStats rollbackStats = rollbackStatsList.get(0); + assertEquals(defaultBroker, rollbackStats.getBrokerName()); + assertEquals(0, rollbackStats.getQueueId()); + assertEquals(5L, rollbackStats.getBrokerOffset()); + assertEquals(5L, rollbackStats.getConsumerOffset()); + } + + @Test + public void testResetOffsetNew() throws Exception { + defaultMQAdminExtImpl.resetOffsetNew(defaultGroup, defaultTopic, timeoutMillis); + verify(mqClientAPIImpl, times(1)).invokeBrokerToResetOffset( + anyString(), + anyString(), + anyString(), + anyLong(), + anyBoolean(), + anyLong(), + anyBoolean()); + } + + @Test + public void testResetOffsetNewConcurrent() { + AdminToolResult actual = defaultMQAdminExtImpl.resetOffsetNewConcurrent(defaultGroup, defaultTopic, timeoutMillis); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), actual.getCode()); + } + + @Test + public void testResetOffsetNewConcurrentTopicRouteInfoNotExist() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(null); + AdminToolResult actual = defaultMQAdminExtImpl.resetOffsetNewConcurrent(defaultGroup, defaultTopic, timeoutMillis); + assertEquals(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST.getCode(), actual.getCode()); + } + + @Test + public void testResetOffsetNewConcurrentException() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenThrow(new MQClientException(ResponseCode.SYSTEM_ERROR, "Test Exception")); + AdminToolResult actual = defaultMQAdminExtImpl.resetOffsetNewConcurrent(defaultGroup, defaultTopic, timeoutMillis); + assertEquals(AdminToolsResultCodeEnum.MQ_CLIENT_ERROR.getCode(), actual.getCode()); + } + + @Test + public void testCreateOrUpdateOrderConfClusterConfig() throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + String key = "key1"; + String value = "value1"; + doNothing().when(mqClientAPIImpl).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + defaultMQAdminExtImpl.createOrUpdateOrderConf(key, value, true); + verify(mqClientAPIImpl, times(1)).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + } + + @Test + public void testCreateOrUpdateOrderConfNonClusterConfig() throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + String key = "key1"; + String value = "value1:value2"; + String oldOrderConfs = "key1:value1;key2:value2"; + when(mqClientAPIImpl.getKVConfigValue(anyString(), anyString(), anyLong())).thenReturn(oldOrderConfs); + doNothing().when(mqClientAPIImpl).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + defaultMQAdminExtImpl.createOrUpdateOrderConf(key, value, false); + verify(mqClientAPIImpl, times(1)).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + } + + @Test + public void testCreateOrUpdateOrderConfExceptionInPut() throws RemotingException, InterruptedException, MQClientException { + String key = "key1"; + String value = "value1:value2"; + String oldOrderConfs = "key1:value1;key2:value2"; + when(mqClientAPIImpl.getKVConfigValue(anyString(), anyString(), anyLong())).thenReturn(oldOrderConfs); + doThrow(new RemotingException("Test Exception")).when(mqClientAPIImpl).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + assertThrows(RemotingException.class, () -> defaultMQAdminExtImpl.createOrUpdateOrderConf(key, value, false)); + } + + @Test + public void testCreateOrUpdateOrderConfNoOldConfs() throws RemotingException, MQBrokerException, InterruptedException, MQClientException { + String key = "key1"; + String value = "value1:value2"; + when(mqClientAPIImpl.getKVConfigValue(anyString(), anyString(), anyLong())).thenReturn(null); + doNothing().when(mqClientAPIImpl).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + defaultMQAdminExtImpl.createOrUpdateOrderConf(key, value, false); + verify(mqClientAPIImpl, times(1)).putKVConfigValue(anyString(), anyString(), anyString(), anyLong()); + } + + @Test + public void testCreateOrUpdateOrderConfExceptionInGet() throws RemotingException, InterruptedException, MQClientException, MQBrokerException { + String key = "key1"; + String value = "value1:value2"; + when(mqClientAPIImpl.getKVConfigValue(anyString(), anyString(), anyLong())).thenThrow(new RemotingException("Test Exception")); + defaultMQAdminExtImpl.createOrUpdateOrderConf(key, value, false); + verify(mqClientAPIImpl, times(1)).getKVConfigValue(anyString(), anyString(), anyLong()); + } + + @Test + public void testQuerySubscriptionValidInput() throws InterruptedException, MQBrokerException, RemotingException, MQClientException { + when(mqClientAPIImpl.querySubscriptionByConsumer(anyString(), anyString(), anyString(), anyLong())).thenReturn(new SubscriptionData()); + assertNotNull(defaultMQAdminExtImpl.querySubscription("group", "topic")); + } + + @Test + public void testQueryTopicsByConsumer() throws Exception { + TopicList expected = new TopicList(); + expected.getTopicList().add(defaultTopic); + when(mqClientAPIImpl.queryTopicsByConsumer(anyString(), anyString(), anyLong())).thenReturn(expected); + TopicList actual = defaultMQAdminExtImpl.queryTopicsByConsumer(defaultGroup); + assertEquals(1, actual.getTopicList().size()); + assertEquals(expected.getTopicList().iterator().next(), actual.getTopicList().iterator().next()); + verify(mqClientAPIImpl, times(1)).queryTopicsByConsumer(anyString(), anyString(), anyLong()); + } + + @Test + public void testQueryTopicsByConsumerRemotingTimeoutException() throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { + when(mqClientAPIImpl.queryTopicsByConsumer(anyString(), anyString(), anyLong())).thenThrow(new RemotingTimeoutException("Test Exception")); + assertThrows(RemotingTimeoutException.class, () -> defaultMQAdminExtImpl.queryTopicsByConsumer(defaultGroup)); + verify(mqClientAPIImpl, times(1)).queryTopicsByConsumer(anyString(), anyString(), anyLong()); + } + + @Test + public void testQueryTopicsByConsumerMQBrokerException() throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { + when(mqClientAPIImpl.queryTopicsByConsumer(anyString(), anyString(), anyLong())).thenThrow(new MQBrokerException(ResponseCode.SYSTEM_ERROR, "Test Exception")); + assertThrows(MQBrokerException.class, () -> defaultMQAdminExtImpl.queryTopicsByConsumer(defaultGroup)); + verify(mqClientAPIImpl, times(1)).queryTopicsByConsumer(anyString(), anyString(), anyLong()); + } + + @Test + public void testQueryTopicsByConsumerMQClientException() throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { + when(mqClientAPIImpl.queryTopicsByConsumer(anyString(), anyString(), anyLong())).thenThrow(new MQBrokerException(ResponseCode.SYSTEM_ERROR, "Test Exception")); + assertThrows(MQBrokerException.class, () -> defaultMQAdminExtImpl.queryTopicsByConsumer(defaultGroup)); + verify(mqClientAPIImpl, times(1)).queryTopicsByConsumer(anyString(), anyString(), anyLong()); + } + + @Test + public void testQueryTopicsByConsumerNoBrokers() throws Exception { + TopicRouteData topicRouteData = new TopicRouteData(); + topicRouteData.setBrokerDatas(new ArrayList<>()); + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(topicRouteData); + TopicList actual = defaultMQAdminExtImpl.queryTopicsByConsumer(defaultGroup); + assertEquals(0, actual.getTopicList().size()); + } + + @Test + public void testQueryTopicsByConsumerConcurrentTopicRouteDataNull() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(null); + AdminToolResult actual = defaultMQAdminExtImpl.queryTopicsByConsumerConcurrent(defaultGroup); + assertEquals(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST.getCode(), actual.getCode()); + assertEquals("router info not found.", actual.getErrorMsg()); + } + + @Test + public void testQueryTopicsByConsumerConcurrentNoBrokers() throws Exception { + TopicRouteData topicRouteData = new TopicRouteData(); + topicRouteData.setBrokerDatas(new ArrayList<>()); + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(any(), anyLong())).thenReturn(topicRouteData); + AdminToolResult actual = defaultMQAdminExtImpl.queryTopicsByConsumerConcurrent(defaultGroup); + assertEquals(AdminToolsResultCodeEnum.TOPIC_ROUTE_INFO_NOT_EXIST.getCode(), actual.getCode()); + assertEquals("router info not found.", actual.getErrorMsg()); + } + + @Test + public void testQueryTopicsByConsumerConcurrent() throws Exception { + TopicList expectedTopicList = new TopicList(); + expectedTopicList.setTopicList(new HashSet<>(Arrays.asList(defaultTopic, "topic2"))); + when(mqClientAPIImpl.queryTopicsByConsumer(any(), any(), anyLong())).thenReturn(expectedTopicList); + AdminToolResult result = defaultMQAdminExtImpl.queryTopicsByConsumerConcurrent(defaultGroup); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), result.getCode()); + Set actual = result.getData().getTopicList(); + assertFalse(actual.isEmpty()); + assertTrue(actual.containsAll(expectedTopicList.getTopicList())); + } + + @Test + public void testQueryConsumeTimeSpanConcurrentTopicRouteDataNull() throws Exception { + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(null); + AdminToolResult> actual = defaultMQAdminExtImpl.queryConsumeTimeSpanConcurrent(defaultTopic, defaultGroup); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), actual.getCode()); + assertEquals(0, actual.getData().size()); + } + + @Test + public void testQueryConsumeTimeSpanConcurrentNoBrokers() throws Exception { + TopicRouteData topicRouteData = new TopicRouteData(); + topicRouteData.setBrokerDatas(new ArrayList<>()); + when(mqClientAPIImpl.getTopicRouteInfoFromNameServer(anyString(), anyLong())).thenReturn(topicRouteData); + AdminToolResult> actual = defaultMQAdminExtImpl.queryConsumeTimeSpanConcurrent(defaultTopic, defaultGroup); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), actual.getCode()); + assertEquals(0, actual.getData().size()); + } + + @Test + public void testQueryConsumeTimeSpanConcurrent() throws Exception { + List spans = new ArrayList<>(); + QueueTimeSpan queueTimeSpan = new QueueTimeSpan(); + queueTimeSpan.setMinTimeStamp(1000L); + queueTimeSpan.setMaxTimeStamp(2000L); + spans.add(queueTimeSpan); + when(mqClientAPIImpl.queryConsumeTimeSpan(anyString(), anyString(), anyString(), anyLong())).thenReturn(spans); + AdminToolResult> actual = defaultMQAdminExtImpl.queryConsumeTimeSpanConcurrent(defaultTopic, defaultGroup); + assertEquals(AdminToolsResultCodeEnum.SUCCESS.getCode(), actual.getCode()); + assertEquals(1, actual.getData().size()); + } + + @Test + public void testDeleteExpiredCommitLog() throws Exception { + ClusterInfo clusterInfo = mock(ClusterInfo.class); + when(clusterInfo.retrieveAllAddrByCluster(defaultCluster)).thenReturn(new String[]{"addr1", "addr2"}); + when(mqClientAPIImpl.getBrokerClusterInfo(anyLong())).thenReturn(clusterInfo); + when(mqClientAPIImpl.deleteExpiredCommitLog(anyString(), anyLong())).thenReturn(true); + boolean actual = defaultMQAdminExtImpl.deleteExpiredCommitLog(defaultCluster); + assertTrue(actual); + verify(mqClientAPIImpl, times(2)).deleteExpiredCommitLog(anyString(), anyLong()); + } + + @Test + public void testDeleteExpiredCommitLogByCluster() throws Exception { + ClusterInfo clusterInfo = mock(ClusterInfo.class); + when(clusterInfo.retrieveAllAddrByCluster(defaultCluster)).thenReturn(new String[]{"addr1", "addr2"}); + when(mqClientAPIImpl.deleteExpiredCommitLog(anyString(), anyLong())).thenReturn(true); + boolean actual = defaultMQAdminExtImpl.deleteExpiredCommitLogByCluster(clusterInfo, defaultCluster); + assertTrue(actual); + verify(mqClientAPIImpl, times(2)).deleteExpiredCommitLog(anyString(), anyLong()); + } + + @Test + public void testDeleteExpiredCommitLogByAddr() throws Exception { + when(mqClientAPIImpl.deleteExpiredCommitLog(defaultBrokerAddr, timeoutMillis)).thenReturn(true); + boolean actual = defaultMQAdminExtImpl.deleteExpiredCommitLogByAddr(defaultBrokerAddr); + assertTrue(actual); + verify(mqClientAPIImpl, times(1)).deleteExpiredCommitLog(defaultBrokerAddr, timeoutMillis); + } + + @Test + public void testConsumeMessageDirectly() throws Exception { + String clientId = "clientId"; + MessageExt messageExt = createMessageExt(); + when(mqAdminImpl.viewMessage(defaultTopic, defaultMsgId)).thenReturn(messageExt); + ConsumeMessageDirectlyResult consumeMessageDirectlyResult = mock(ConsumeMessageDirectlyResult.class); + when(mqClientAPIImpl.consumeMessageDirectly( + anyString(), + anyString(), + anyString(), + anyString(), + anyString(), + anyLong())) + .thenReturn(consumeMessageDirectlyResult); + ConsumeMessageDirectlyResult actual = defaultMQAdminExtImpl.consumeMessageDirectly(defaultGroup, clientId, defaultTopic, defaultMsgId); + assertNotNull(actual); + assertNull(actual.getRemark()); + assertFalse(actual.isAutoCommit()); + assertFalse(actual.isOrder()); + } + + @Test + public void testMessageTrackDetailConcurrent() throws Exception { + MessageExt messageExt = createMessageExt(); + GroupList groupList = mock(GroupList.class); + HashSet groupSet = new HashSet<>(); + groupSet.add(defaultGroup); + when(groupList.getGroupList()).thenReturn(groupSet); + when(mqClientAPIImpl.queryTopicConsumeByWho(anyString(), anyString(), anyLong())).thenReturn(groupList); + ConsumerConnection consumerConnection = mock(ConsumerConnection.class); + when(mqClientAPIImpl.getConsumerConnectionList(anyString(), anyString(), anyLong())).thenReturn(consumerConnection); + List actual = defaultMQAdminExtImpl.messageTrackDetailConcurrent(messageExt); + assertEquals(1, actual.size()); + } + +// @Test +// public void testConsumedConcurrent() throws Exception { +// ConsumeStats consumeStats = mock(ConsumeStats.class); +// ClusterInfo ci = mock(ClusterInfo.class); +// Map brokerAddrTable = new HashMap<>(); +// BrokerData brokerData = mock(BrokerData.class); +// HashMap brokerAddress = new HashMap<>(); +// brokerAddress.put(0L, defaultBrokerAddr); +// when(brokerData.getBrokerAddrs()).thenReturn(brokerAddress); +// brokerAddrTable.put(defaultBroker, brokerData); +// when(ci.getBrokerAddrTable()).thenReturn(brokerAddrTable); +// Map offsetTable = new HashMap<>(); +// OffsetWrapper offsetWrapper = new OffsetWrapper(); +// offsetWrapper.setConsumerOffset(1L); +// offsetTable.put(new MessageQueue(defaultTopic, defaultBroker, 0), offsetWrapper); +// when(consumeStats.getOffsetTable()).thenReturn(offsetTable); +//// when(mqClientAPIImpl.getConsumeStats(any(), any(), any(), anyLong())).thenReturn(consumeStats); +// when(mqClientAPIImpl.getBrokerClusterInfo(anyLong())).thenReturn(ci); +//// assertTrue(defaultMQAdminExtImpl.consumedConcurrent(createMessageExt(), defaultGroup)); +// } + + @Test + public void testCloneGroupOffsetValidInput() throws RemotingException, MQClientException, InterruptedException, MQBrokerException { + String srcGroup = "srcGroup"; + String destGroup = "destGroup"; + boolean isOffline = false; + defaultMQAdminExtImpl.cloneGroupOffset(srcGroup, destGroup, defaultTopic, isOffline); + verify(mqClientAPIImpl, times(1)).cloneGroupOffset( + anyString(), + anyString(), + anyString(), + anyString(), + anyBoolean(), + anyLong()); + } + + @Test + public void testGetUserSubscriptionGroup() throws Exception { + SubscriptionGroupWrapper subscriptionGroupWrapper = new SubscriptionGroupWrapper(); + ConcurrentMap subscriptionGroupTable = new ConcurrentHashMap<>(); + SubscriptionGroupConfig groupConfig1 = new SubscriptionGroupConfig(); + groupConfig1.setGroupName("CID_RMQ_SYS_GROUP"); + SubscriptionGroupConfig groupConfig2 = new SubscriptionGroupConfig(); + groupConfig2.setGroupName("DEFAULT_CONSUMER"); + SubscriptionGroupConfig groupConfig3 = new SubscriptionGroupConfig(); + groupConfig3.setGroupName("SYS_CONSUMER_GROUP"); + subscriptionGroupTable.put(groupConfig1.getGroupName(), groupConfig1); + subscriptionGroupTable.put(groupConfig2.getGroupName(), groupConfig2); + subscriptionGroupTable.put(groupConfig3.getGroupName(), groupConfig3); + subscriptionGroupWrapper.setSubscriptionGroupTable(subscriptionGroupTable); + when(mqClientAPIImpl.getAllSubscriptionGroup(any(), anyLong())).thenReturn(subscriptionGroupWrapper); + SubscriptionGroupWrapper actual = defaultMQAdminExtImpl.getUserSubscriptionGroup(defaultBrokerAddr, timeoutMillis); + assertEquals(1, actual.getSubscriptionGroupTable().size()); + assertTrue(actual.getSubscriptionGroupTable().containsKey("SYS_CONSUMER_GROUP")); + } + + @Test + public void testGetUserTopicConfig() throws Exception { + TopicConfigSerializeWrapper topicConfigSerializeWrapper = new TopicConfigSerializeWrapper(); + ConcurrentMap topicConfigMap = new ConcurrentHashMap<>(); + topicConfigMap.put("Topic1", new TopicConfig("Topic1", 1, 1, 0)); + topicConfigMap.put("Topic2", new TopicConfig("Topic2", 1, 1, 1)); + topicConfigSerializeWrapper.setTopicConfigTable(topicConfigMap); + TopicList topicList = new TopicList(); + Set topicSet = new HashSet<>(); + topicSet.add("Topic2"); + topicList.setTopicList(topicSet); + when(mqClientAPIImpl.getAllTopicConfig(any(), anyLong())).thenReturn(topicConfigSerializeWrapper); + when(mqClientAPIImpl.getSystemTopicListFromBroker(anyString(), anyLong())).thenReturn(topicList); + TopicConfigSerializeWrapper actual = defaultMQAdminExtImpl.getUserTopicConfig("brokerAddr", false, timeoutMillis); + assertEquals(1, actual.getTopicConfigTable().size()); + } + + @Test + public void testUpdateConsumeOffset() throws Exception { + doNothing().when(mqClientAPIImpl).updateConsumerOffset(any(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + defaultMQAdminExtImpl.updateConsumeOffset(defaultBrokerAddr, defaultGroup, createMessageQueue(), 1L); + verify(mqClientAPIImpl, times(1)).updateConsumerOffset(any(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + } + + @Test + public void testUpdateConsumeOffsetException() throws MQBrokerException, RemotingException, InterruptedException { + doThrow(new RemotingException("Test exception")).when(mqClientAPIImpl).updateConsumerOffset(anyString(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + assertThrows(RemotingException.class, + () -> defaultMQAdminExtImpl.updateConsumeOffset(defaultBrokerAddr, defaultGroup, createMessageQueue(), 1L)); + } + + @Test + public void testResetOffsetByQueueId() throws Exception { + long resetOffset = 100; + doNothing().when(mqClientAPIImpl).updateConsumerOffset(any(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + Map result = new HashMap<>(); + result.put(createMessageQueue(), resetOffset); + when(mqClientAPIImpl.invokeBrokerToResetOffset(anyString(), anyString(), anyString(), anyLong(), anyInt(), anyLong(), anyLong())).thenReturn(result); + defaultMQAdminExtImpl.resetOffsetByQueueId(defaultBrokerAddr, defaultGroup, defaultTopic, 0, resetOffset); + verify(mqClientAPIImpl, times(1)).updateConsumerOffset(any(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + verify(mqClientAPIImpl, times(1)).invokeBrokerToResetOffset( + anyString(), + anyString(), + anyString(), + anyLong(), + anyInt(), + anyLong(), + anyLong()); + } + + @Test + public void testResetOffsetByQueueIdThrowsException() throws MQBrokerException, RemotingException, InterruptedException, MQClientException { + doNothing().when(mqClientAPIImpl).updateConsumerOffset(any(), any(UpdateConsumerOffsetRequestHeader.class), anyLong()); + when(mqClientAPIImpl.invokeBrokerToResetOffset( + anyString(), + anyString(), + anyString(), + anyLong(), + anyInt(), + anyLong(), + anyLong())) + .thenThrow(new MQClientException(1, "Exception")); + assertThrows(MQBrokerException.class, + () -> defaultMQAdminExtImpl.resetOffsetByQueueId(defaultBrokerAddr, defaultGroup, defaultTopic, 0, 100)); + } + + @Test + public void testUpdateAndGetGroupReadForbidden() throws RemotingException, InterruptedException, MQBrokerException { + boolean readable = true; + GroupForbidden expectedResponse = new GroupForbidden(); + expectedResponse.setGroup(defaultGroup); + expectedResponse.setTopic(defaultTopic); + expectedResponse.setReadable(readable); + when(mqClientAPIImpl.updateAndGetGroupForbidden(any(), any(UpdateGroupForbiddenRequestHeader.class), anyLong())).thenReturn(expectedResponse); + GroupForbidden actual = defaultMQAdminExtImpl.updateAndGetGroupReadForbidden(defaultBrokerAddr, defaultGroup, defaultTopic, readable); + assertNotNull(actual); + assertEquals(defaultGroup, actual.getGroup()); + assertEquals(defaultTopic, actual.getTopic()); + assertEquals(readable, actual.getReadable()); + } + + @Test + public void testUpdateAndGetGroupReadForbiddenException() throws RemotingConnectException, RemotingSendRequestException, RemotingTimeoutException, MQBrokerException, InterruptedException { + when(mqClientAPIImpl.updateAndGetGroupForbidden(any(), any(UpdateGroupForbiddenRequestHeader.class), anyLong())) + .thenThrow(new MQBrokerException(ResponseCode.SYSTEM_ERROR, "Test Exception")); + assertThrows(MQBrokerException.class, + () -> defaultMQAdminExtImpl.updateAndGetGroupReadForbidden(defaultBrokerAddr, defaultGroup, defaultTopic, true)); + } + + private HashMap createBrokerAddrs() { + HashMap result = new HashMap<>(); + result.put(0L, defaultBrokerAddr); + return result; + } + + private TopicRouteData createTopicRouteData() { + BrokerData bd = new BrokerData(defaultCluster, defaultBroker, new HashMap<>()); + bd.setBrokerAddrs(new HashMap<>()); + bd.getBrokerAddrs().put(0L, defaultBrokerAddr); + QueueData qd = new QueueData(); + qd.setBrokerName(defaultBroker); + qd.setPerm(PermName.PERM_WRITE); + qd.setReadQueueNums(1); + qd.setTopicSysFlag(0); + qd.setWriteQueueNums(1); + TopicRouteData result = new TopicRouteData(); + result.getBrokerDatas().add(bd); + result.getQueueDatas().add(qd); + return result; + } + + private MessageQueue createMessageQueue() { + return new MessageQueue(defaultTopic, defaultBroker, 0); + } + + private MessageExt createMessageExt() { + MessageExt result = new MessageExt(); + result.setMsgId(defaultMsgId); + result.setTopic(defaultTopic); + result.setQueueId(0); + InetAddress inetAddress = mock(InetAddress.class); + InetSocketAddress address = new InetSocketAddress(inetAddress, 10911); + when(inetAddress.getHostAddress()).thenReturn("127.0.0.1"); + result.setStoreHost(address); + return result; + } +} diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java index 7d76dafa011..092d1f624ed 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/broker/GetBrokerConfigCommandTest.java @@ -48,7 +48,7 @@ protected byte[] getBody() { public void testExecute() throws SubCommandException { GetBrokerConfigCommand cmd = new GetBrokerConfigCommand(); Options options = ServerUtil.buildCommandlineOptions(new Options()); - String[] subargs = new String[] {"-b 127.0.0.1:" + listenPort(), "-c default-cluster"}; + String[] subargs = new String[] {"-b 127.0.0.1:" + listenPort()}; final CommandLine commandLine = ServerUtil.parseCmdLine("mqadmin " + cmd.commandName(), subargs, cmd.buildCommandlineOptions(options), new DefaultParser()); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java index f5967f5a303..a2ad8c5d850 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ConsumerConnectionSubCommandTest.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.tools.command.connection; -import java.util.HashSet; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; @@ -30,7 +29,7 @@ import org.junit.Before; import org.junit.Test; -import static org.mockito.Mockito.mock; +import java.util.HashSet; public class ConsumerConnectionSubCommandTest { private ServerResponseMocker brokerMocker; @@ -62,7 +61,7 @@ public void testExecute() throws SubCommandException { private ServerResponseMocker startOneBroker() { ConsumerConnection consumerConnection = new ConsumerConnection(); HashSet connectionSet = new HashSet<>(); - Connection connection = mock(Connection.class); + Connection connection = new Connection(); connectionSet.add(connection); consumerConnection.setConnectionSet(connectionSet); // start broker diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java index 672e4113d6c..812edde9e04 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/connection/ProducerConnectionSubCommandTest.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.tools.command.connection; -import java.util.HashSet; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; @@ -30,7 +29,7 @@ import org.junit.Before; import org.junit.Test; -import static org.mockito.Mockito.mock; +import java.util.HashSet; public class ProducerConnectionSubCommandTest { @@ -64,7 +63,7 @@ public void testExecute() throws SubCommandException { private ServerResponseMocker startOneBroker() { ProducerConnection producerConnection = new ProducerConnection(); HashSet connectionSet = new HashSet<>(); - Connection connection = mock(Connection.class); + Connection connection = new Connection(); connectionSet.add(connection); producerConnection.setConnectionSet(connectionSet); diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java index 4651113c689..9f4da15fa3a 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/ConsumerStatusSubCommandTest.java @@ -16,7 +16,6 @@ */ package org.apache.rocketmq.tools.command.consumer; -import java.util.HashSet; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; @@ -30,7 +29,7 @@ import org.junit.Before; import org.junit.Test; -import static org.mockito.Mockito.mock; +import java.util.HashSet; public class ConsumerStatusSubCommandTest { @@ -65,7 +64,7 @@ public void testExecute() throws SubCommandException { private ServerResponseMocker startOneBroker() { ConsumerConnection consumerConnection = new ConsumerConnection(); HashSet connectionSet = new HashSet<>(); - Connection connection = mock(Connection.class); + Connection connection = new Connection(); connectionSet.add(connection); consumerConnection.setConnectionSet(connectionSet); // start broker diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java index e4e5e970c6e..7ff4c42851a 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/consumer/GetConsumerConfigSubCommandTest.java @@ -16,9 +16,6 @@ */ package org.apache.rocketmq.tools.command.consumer; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Set; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; @@ -33,7 +30,9 @@ import org.junit.Before; import org.junit.Test; -import static org.mockito.Mockito.mock; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Set; public class GetConsumerConfigSubCommandTest { @@ -91,7 +90,7 @@ private ServerResponseMocker startNameServer() { private ServerResponseMocker startOneBroker() { ConsumerConnection consumerConnection = new ConsumerConnection(); HashSet connectionSet = new HashSet<>(); - Connection connection = mock(Connection.class); + Connection connection = new Connection(); connectionSet.add(connection); consumerConnection.setConnectionSet(connectionSet); // start broker 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..dcb27ab6ee2 --- /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.jupiter.api.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..af97396729e --- /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.jupiter.api.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(); + } +} diff --git a/tools/src/test/java/org/apache/rocketmq/tools/command/producer/ProducerSubCommandTest.java b/tools/src/test/java/org/apache/rocketmq/tools/command/producer/ProducerSubCommandTest.java index 7039c05fbad..b1ebb924e77 100644 --- a/tools/src/test/java/org/apache/rocketmq/tools/command/producer/ProducerSubCommandTest.java +++ b/tools/src/test/java/org/apache/rocketmq/tools/command/producer/ProducerSubCommandTest.java @@ -17,13 +17,12 @@ package org.apache.rocketmq.tools.command.producer; -import java.util.HashMap; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.DefaultParser; import org.apache.commons.cli.Options; -import org.apache.rocketmq.common.message.MessageQueue; -import org.apache.rocketmq.remoting.protocol.admin.ConsumeStats; -import org.apache.rocketmq.remoting.protocol.admin.OffsetWrapper; +import org.apache.rocketmq.remoting.protocol.LanguageCode; +import org.apache.rocketmq.remoting.protocol.body.ProducerInfo; +import org.apache.rocketmq.remoting.protocol.body.ProducerTableInfo; import org.apache.rocketmq.srvutil.ServerUtil; import org.apache.rocketmq.tools.command.SubCommandException; import org.apache.rocketmq.tools.command.server.NameServerMocker; @@ -32,6 +31,10 @@ import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + public class ProducerSubCommandTest { private ServerResponseMocker brokerMocker; @@ -61,21 +64,16 @@ public void testExecute() throws SubCommandException { } private ServerResponseMocker startOneBroker() { - ConsumeStats consumeStats = new ConsumeStats(); - HashMap offsetTable = new HashMap<>(); - MessageQueue messageQueue = new MessageQueue(); - messageQueue.setBrokerName("mockBrokerName"); - messageQueue.setQueueId(1); - messageQueue.setBrokerName("mockTopicName"); - - OffsetWrapper offsetWrapper = new OffsetWrapper(); - offsetWrapper.setBrokerOffset(1); - offsetWrapper.setConsumerOffset(1); - offsetWrapper.setLastTimestamp(System.currentTimeMillis()); + ProducerTableInfo producerTableInfo = new ProducerTableInfo(new HashMap<>()); + List producerInfo = new ArrayList<>(); + producerInfo.add(new ProducerInfo( + "xxxx-client-id", + "127.0.0.1:18978", + LanguageCode.JAVA, + 400, + System.currentTimeMillis())); - offsetTable.put(messageQueue, offsetWrapper); - consumeStats.setOffsetTable(offsetTable); - // start broker - return ServerResponseMocker.startServer(consumeStats.encode()); + producerTableInfo.getData().put("mockTopicName", producerInfo); + return ServerResponseMocker.startServer(producerTableInfo.encode()); } }