From 201ceeb30cfc2257b8f7c6f1e0e132c9ac50fcfb Mon Sep 17 00:00:00 2001 From: Yonny Hao Date: Wed, 17 Sep 2025 11:24:37 +0800 Subject: [PATCH 1/2] Improved tenant route cache refresh efficiency and reduced memory overhead accordingly --- .../bifromq-dist-worker-schema/pom.xml | 4 + .../dist/worker/schema/GroupMatching.java | 5 +- .../dist/worker/schema/KVSchemaUtil.java | 41 +- .../bifromq/dist/worker/schema/Matching.java | 2 +- .../dist/worker/schema/NormalMatching.java | 2 +- .../dist/worker/schema/KVSchemaUtilTest.java | 45 +- .../bifromq/dist/worker/DistWorkerCoProc.java | 119 ++-- .../dist/worker/cache/IMatchedRoutes.java | 51 +- .../dist/worker/cache/ISubscriptionCache.java | 11 +- .../dist/worker/cache/ITenantRouteCache.java | 11 +- .../dist/worker/cache/MatchedRoutes.java | 204 +++++++ .../dist/worker/cache/RouteCacheKey.java | 37 ++ .../dist/worker/cache/SubscriptionCache.java | 5 +- .../dist/worker/cache/TenantRouteCache.java | 274 +++++++--- .../dist/worker/cache/TenantRouteMatcher.java | 115 +--- .../dist/worker/cache/task/AddRoutesTask.java | 42 ++ .../dist/worker/cache/task/CacheTaskType.java | 24 + .../dist/worker/cache/task/LoadEntryTask.java | 44 ++ .../worker/cache/task/RefreshEntriesTask.java | 33 ++ .../worker/cache/task/ReloadEntryTask.java | 52 ++ .../worker/cache/task/RemoveRoutesTask.java | 42 ++ .../cache/task/TenantRouteCacheTask.java | 24 + .../dist/worker/DistWorkerCoProcTest.java | 29 +- .../dist/worker/cache/MatchedRoutesTest.java | 302 +++++++++++ .../worker/cache/SubscriptionCacheTest.java | 31 +- .../worker/cache/TenantRouteCacheTest.java | 442 ++++++++------- .../worker/cache/TenantRouteMatcherTest.java | 507 ++++++++++++++++++ .../bifromq/dist/worker/schema/Matchings.java | 88 +++ .../apache/bifromq/metrics/ITenantMeter.java | 15 +- .../metrics/TenantFunctionCounters.java | 70 +++ .../apache/bifromq/metrics/TenantMeter.java | 69 +-- .../apache/bifromq/metrics/TenantMetric.java | 13 +- .../mqtt/integration/v3/MQTTLastWillTest.java | 57 +- .../distservice/GroupFanoutThrottled.java | 1 + .../PersistentFanoutThrottled.java | 1 + .../bifromq/util/index/TopicLevelTrie.java | 15 +- 36 files changed, 2247 insertions(+), 580 deletions(-) create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/MatchedRoutes.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/RouteCacheKey.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/AddRoutesTask.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/CacheTaskType.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/LoadEntryTask.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RefreshEntriesTask.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/ReloadEntryTask.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RemoveRoutesTask.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/TenantRouteCacheTask.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/MatchedRoutesTest.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcherTest.java create mode 100644 bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/schema/Matchings.java create mode 100644 bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantFunctionCounters.java diff --git a/bifromq-dist/bifromq-dist-worker-schema/pom.xml b/bifromq-dist/bifromq-dist-worker-schema/pom.xml index 1c2cf307d..91bcbf872 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/pom.xml +++ b/bifromq-dist/bifromq-dist-worker-schema/pom.xml @@ -30,6 +30,10 @@ bifromq-dist-worker-schema + + com.github.ben-manes.caffeine + caffeine + org.apache.bifromq bifromq-util diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java index ff745eeef..58d4622db 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java @@ -29,7 +29,7 @@ /** * Represent a group matching route. */ -@EqualsAndHashCode(callSuper = true) +@EqualsAndHashCode(callSuper = true, cacheStrategy = EqualsAndHashCode.CacheStrategy.LAZY) @ToString public final class GroupMatching extends Matching { @EqualsAndHashCode.Exclude @@ -41,8 +41,9 @@ public final class GroupMatching extends Matching { GroupMatching(String tenantId, RouteMatcher matcher, Map members) { super(tenantId, matcher); assert matcher.getType() != RouteMatcher.Type.Normal; - this.ordered = matcher.getType() == RouteMatcher.Type.OrderedShare; this.receivers = members; + + this.ordered = matcher.getType() == RouteMatcher.Type.OrderedShare; this.receiverList = members.entrySet().stream() .map(e -> new NormalMatching(tenantId, matcher, e.getKey(), e.getValue())) .collect(Collectors.toList()); diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java index f1c0117c4..a46090e38 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java @@ -14,11 +14,13 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker.schema; +import static com.google.protobuf.ByteString.copyFromUtf8; +import static com.google.protobuf.UnsafeByteOperations.unsafeWrap; import static org.apache.bifromq.util.BSUtil.toByteString; import static org.apache.bifromq.util.BSUtil.toShort; import static org.apache.bifromq.util.TopicConst.DELIMITER; @@ -27,15 +29,14 @@ import static org.apache.bifromq.util.TopicConst.UNORDERED_SHARE; import static org.apache.bifromq.util.TopicUtil.parse; import static org.apache.bifromq.util.TopicUtil.unescape; -import static com.google.protobuf.ByteString.copyFromUtf8; -import static com.google.protobuf.UnsafeByteOperations.unsafeWrap; +import com.github.benmanes.caffeine.cache.Interner; +import com.google.protobuf.ByteString; +import java.util.List; import org.apache.bifromq.dist.rpc.proto.MatchRoute; import org.apache.bifromq.dist.rpc.proto.RouteGroup; import org.apache.bifromq.type.RouteMatcher; import org.apache.bifromq.util.BSUtil; -import com.google.protobuf.ByteString; -import java.util.List; /** * Utility for working with the data stored in dist worker. @@ -50,6 +51,8 @@ public class KVSchemaUtil { private static final ByteString FLAG_NORMAL_VAL = ByteString.copyFrom(new byte[] {FLAG_NORMAL}); private static final ByteString FLAG_UNORDERED_VAL = ByteString.copyFrom(new byte[] {FLAG_UNORDERED}); private static final ByteString FLAG_ORDERED_VAL = ByteString.copyFrom(new byte[] {FLAG_ORDERED}); + private static final Interner MATCHING_INTERNER = Interner.newWeakInterner(); + private static final Interner RECEIVER_INTERNER = Interner.newWeakInterner(); public static String toReceiverUrl(MatchRoute route) { return toReceiverUrl(route.getBrokerId(), route.getReceiverId(), route.getDelivererKey()); @@ -61,26 +64,35 @@ public static String toReceiverUrl(int subBrokerId, String receiverId, String de public static Receiver parseReceiver(String receiverUrl) { String[] parts = receiverUrl.split(NUL); - return new Receiver(Integer.parseInt(parts[0]), parts[1], parts[2]); + return RECEIVER_INTERNER.intern(new Receiver(Integer.parseInt(parts[0]), parts[1], parts[2])); } public static Matching buildMatchRoute(ByteString routeKey, ByteString routeValue) { RouteDetail routeDetail = parseRouteDetail(routeKey); try { if (routeDetail.matcher().getType() == RouteMatcher.Type.Normal) { - return new NormalMatching(routeDetail.tenantId(), - routeDetail.matcher(), - routeDetail.receiverUrl(), - BSUtil.toLong(routeValue)); + return buildNormalMatchRoute(routeDetail, BSUtil.toLong(routeValue)); } - return new GroupMatching(routeDetail.tenantId(), - routeDetail.matcher(), - RouteGroup.parseFrom(routeValue).getMembersMap()); + return buildGroupMatchRoute(routeDetail, RouteGroup.parseFrom(routeValue)); } catch (Exception e) { throw new IllegalStateException("Unable to parse matching record", e); } } + public static Matching buildNormalMatchRoute(RouteDetail routeDetail, long incarnation) { + assert routeDetail.matcher().getType() == RouteMatcher.Type.Normal; + return MATCHING_INTERNER.intern(new NormalMatching(routeDetail.tenantId(), + routeDetail.matcher(), + routeDetail.receiverUrl(), + incarnation)); + } + + public static Matching buildGroupMatchRoute(RouteDetail routeDetail, RouteGroup group) { + assert routeDetail.matcher().getType() != RouteMatcher.Type.Normal; + return MATCHING_INTERNER.intern(new GroupMatching(routeDetail.tenantId(), routeDetail.matcher(), + group.getMembersMap())); + } + public static ByteString tenantBeginKey(String tenantId) { ByteString tenantIdBytes = copyFromUtf8(tenantId); return SCHEMA_VER.concat(toByteString((short) tenantIdBytes.size()).concat(tenantIdBytes)); @@ -107,8 +119,7 @@ public static ByteString toNormalRouteKey(String tenantId, RouteMatcher routeMat public static ByteString toGroupRouteKey(String tenantId, RouteMatcher routeMatcher) { assert routeMatcher.getType() != RouteMatcher.Type.Normal; - return tenantRouteBucketStartKey(tenantId, routeMatcher.getFilterLevelList(), - bucket(routeMatcher.getGroup())) + return tenantRouteBucketStartKey(tenantId, routeMatcher.getFilterLevelList(), bucket(routeMatcher.getGroup())) .concat(routeMatcher.getType() == RouteMatcher.Type.OrderedShare ? FLAG_ORDERED_VAL : FLAG_UNORDERED_VAL) .concat(toReceiverBytes(routeMatcher.getGroup())); } diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Matching.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Matching.java index 60b8d9aa5..56d659fe1 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Matching.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Matching.java @@ -26,7 +26,7 @@ /** * The abstract class of matching route. */ -@EqualsAndHashCode +@EqualsAndHashCode(cacheStrategy = EqualsAndHashCode.CacheStrategy.LAZY) @ToString public abstract class Matching { @EqualsAndHashCode.Exclude diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java index 07b95f223..f7650af30 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java @@ -27,7 +27,7 @@ /** * Represent a normal matching route. */ -@EqualsAndHashCode(callSuper = true) +@EqualsAndHashCode(callSuper = true, cacheStrategy = EqualsAndHashCode.CacheStrategy.LAZY) @ToString public class NormalMatching extends Matching { private final String receiverUrl; diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtilTest.java b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtilTest.java index 2c52d5e57..9d469cffe 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtilTest.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtilTest.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker.schema; @@ -24,15 +24,16 @@ import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toNormalRouteKey; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; +import com.google.protobuf.ByteString; import org.apache.bifromq.dist.rpc.proto.MatchRoute; import org.apache.bifromq.dist.rpc.proto.RouteGroup; import org.apache.bifromq.type.MatchInfo; import org.apache.bifromq.type.RouteMatcher; import org.apache.bifromq.util.BSUtil; import org.apache.bifromq.util.TopicUtil; -import com.google.protobuf.ByteString; import org.testng.annotations.Test; public class KVSchemaUtilTest { @@ -40,6 +41,46 @@ public class KVSchemaUtilTest { private static final int InboxService = 1; private static final int RuleEngine = 2; + @Test + public void testNormalMatchingIntern() { + String tenantId = "tenantId"; + String topicFilter = "/a/b/c"; + RouteMatcher matcher = TopicUtil.from(topicFilter); + MatchRoute route = MatchRoute.newBuilder() + .setMatcher(matcher) + .setBrokerId(MqttBroker) + .setReceiverId("inbox1") + .setDelivererKey("delivererKey1") + .setIncarnation(1L) + .build(); + ByteString key = toNormalRouteKey(tenantId, matcher, toReceiverUrl(route)); + Matching matching1 = buildMatchRoute(key, BSUtil.toByteString(route.getIncarnation())); + Matching matching2 = buildMatchRoute(key, BSUtil.toByteString(route.getIncarnation())); + assertSame(matching1, matching2); + } + + @Test + public void testGroupMatchingIntern() { + String origTopicFilter = "$share/group//a/b/c"; + RouteMatcher matcher = TopicUtil.from(origTopicFilter); + MatchRoute route = MatchRoute.newBuilder() + .setMatcher(matcher) + .setBrokerId(MqttBroker) + .setReceiverId("inbox1") + .setDelivererKey("server1") + .setIncarnation(1L) + .build(); + + String scopedReceiverId = toReceiverUrl(MqttBroker, "inbox1", "server1"); + ByteString key = toGroupRouteKey("tenantId", matcher); + RouteGroup groupMembers = RouteGroup.newBuilder() + .putMembers(scopedReceiverId, route.getIncarnation()) + .build(); + Matching matching1 = buildMatchRoute(key, groupMembers.toByteString()); + Matching matching2 = buildMatchRoute(key, groupMembers.toByteString()); + assertSame(matching1, matching2); + } + @Test public void testParseNormalMatchRecord() { String tenantId = "tenantId"; diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/DistWorkerCoProc.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/DistWorkerCoProc.java index 79baeaacd..f0837aecc 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/DistWorkerCoProc.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/DistWorkerCoProc.java @@ -14,24 +14,49 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker; +import static java.util.Collections.singletonList; import static org.apache.bifromq.basekv.utils.BoundaryUtil.intersect; import static org.apache.bifromq.basekv.utils.BoundaryUtil.isNULLRange; import static org.apache.bifromq.basekv.utils.BoundaryUtil.toBoundary; import static org.apache.bifromq.basekv.utils.BoundaryUtil.upperBound; import static org.apache.bifromq.dist.worker.Comparators.FilterLevelsComparator; import static org.apache.bifromq.dist.worker.Comparators.RouteMatcherComparator; +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.buildGroupMatchRoute; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.buildMatchRoute; +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.buildNormalMatchRoute; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.tenantBeginKey; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toGroupRouteKey; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toNormalRouteKey; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; -import static java.util.Collections.singletonList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.protobuf.Any; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Optional; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; import org.apache.bifromq.basekv.proto.Boundary; import org.apache.bifromq.basekv.proto.KVRangeId; import org.apache.bifromq.basekv.store.api.IKVCloseableReader; @@ -62,6 +87,8 @@ import org.apache.bifromq.dist.rpc.proto.RouteGroup; import org.apache.bifromq.dist.rpc.proto.TopicFanout; import org.apache.bifromq.dist.worker.cache.ISubscriptionCache; +import org.apache.bifromq.dist.worker.cache.task.AddRoutesTask; +import org.apache.bifromq.dist.worker.cache.task.RemoveRoutesTask; import org.apache.bifromq.dist.worker.schema.GroupMatching; import org.apache.bifromq.dist.worker.schema.KVSchemaUtil; import org.apache.bifromq.dist.worker.schema.Matching; @@ -71,30 +98,6 @@ import org.apache.bifromq.type.RouteMatcher; import org.apache.bifromq.type.TopicMessagePack; import org.apache.bifromq.util.BSUtil; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.protobuf.Any; -import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.NavigableSet; -import java.util.Optional; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; -import lombok.SneakyThrows; -import lombok.extern.slf4j.Slf4j; @Slf4j class DistWorkerCoProc implements IKVRangeCoProc { @@ -154,7 +157,7 @@ public Supplier mutate(RWCoProcInput input, IKVReader reader, IK DistServiceRWCoProcInput coProcInput = input.getDistService(); log.trace("Receive rw co-proc request\n{}", coProcInput); // tenantId -> topicFilter - NavigableMap> updatedMatches = Maps.newTreeMap(); + NavigableMap>> updatedMatches = Maps.newTreeMap(); DistServiceRWCoProcOutput.Builder outputBuilder = DistServiceRWCoProcOutput.newBuilder(); AtomicReference afterMutate = new AtomicReference<>(); switch (coProcInput.getTypeCase()) { @@ -176,8 +179,13 @@ public Supplier mutate(RWCoProcInput input, IKVReader reader, IK } RWCoProcOutput output = RWCoProcOutput.newBuilder().setDistService(outputBuilder.build()).build(); return () -> { - routeCache.refresh(updatedMatches); - updatedMatches.forEach((tenantId, topicFilters) -> topicFilters.forEach((topicFilter) -> { + routeCache.refresh(Maps.transformValues(updatedMatches, v -> { + if (coProcInput.getTypeCase() == DistServiceRWCoProcInput.TypeCase.BATCHMATCH) { + return AddRoutesTask.of(v); + } + return RemoveRoutesTask.of(v); + })); + updatedMatches.forEach((tenantId, topicFilters) -> topicFilters.forEach((topicFilter, matchings) -> { if (topicFilter.getType() == RouteMatcher.Type.OrderedShare) { deliverExecutorGroup.refreshOrderedShareSubRoutes(tenantId, topicFilter); } @@ -190,7 +198,7 @@ public Supplier mutate(RWCoProcInput input, IKVReader reader, IK } private void refreshFact(IKVReader reader, - NavigableMap> mutations, + NavigableMap>> mutations, boolean isAdd) { if (mutations.isEmpty()) { return; @@ -199,10 +207,11 @@ private void refreshFact(IKVReader reader, if (!fact.hasFirstGlobalFilterLevels() || !fact.hasLastGlobalFilterLevels()) { needRefresh = true; } else { - Map.Entry> firstMutation = mutations.firstEntry(); - Map.Entry> lastMutation = mutations.lastEntry(); - Iterable firstRoute = toGlobalTopicLevels(firstMutation.getKey(), firstMutation.getValue().first()); - Iterable lastRoute = toGlobalTopicLevels(lastMutation.getKey(), lastMutation.getValue().last()); + Map.Entry>> firstMutation = mutations.firstEntry(); + Map.Entry>> lastMutation = mutations.lastEntry(); + Iterable firstRoute = toGlobalTopicLevels(firstMutation.getKey(), + firstMutation.getValue().firstKey()); + Iterable lastRoute = toGlobalTopicLevels(lastMutation.getKey(), lastMutation.getValue().lastKey()); if (isAdd) { if (FilterLevelsComparator .compare(firstRoute, fact.getFirstGlobalFilterLevels().getFilterLevelList()) < 0 @@ -265,7 +274,7 @@ public void close() { } private Runnable batchAddRoute(BatchMatchRequest request, IKVReader reader, IKVWriter writer, - Map> newMatches, + Map>> newMatches, BatchMatchReply.Builder replyBuilder) { replyBuilder.setReqId(request.getReqId()); Map normalRoutesAdded = new HashMap<>(); @@ -281,16 +290,21 @@ private Runnable batchAddRoute(BatchMatchRequest request, IKVReader reader, IKVW long incarnation = route.getIncarnation(); RouteMatcher routeMatcher = route.getMatcher(); if (routeMatcher.getType() == RouteMatcher.Type.Normal) { - ByteString normalRouteKey = toNormalRouteKey(tenantId, routeMatcher, toReceiverUrl(route)); + String receiverUrl = toReceiverUrl(route); + ByteString normalRouteKey = toNormalRouteKey(tenantId, routeMatcher, receiverUrl); + Optional incarOpt = reader.get(normalRouteKey).map(BSUtil::toLong); if (incarOpt.isEmpty() || incarOpt.get() < incarnation) { + RouteDetail routeDetail = new RouteDetail(tenantId, routeMatcher, receiverUrl); + Matching normalMatching = buildNormalMatchRoute(routeDetail, incarnation); writer.put(normalRouteKey, BSUtil.toByteString(incarnation)); // match record may be duplicated in the request if (!addedMatches.contains(normalRouteKey)) { normalRoutesAdded.computeIfAbsent(tenantId, k -> new AtomicInteger()).incrementAndGet(); } - newMatches.computeIfAbsent(tenantId, k -> new TreeSet<>(RouteMatcherComparator)) - .add(routeMatcher); + newMatches.computeIfAbsent(tenantId, k -> new TreeMap<>(RouteMatcherComparator)) + .computeIfAbsent(routeMatcher, k -> new HashSet<>()) + .add(normalMatching); addedMatches.add(normalRouteKey); } codes[i] = BatchMatchReply.TenantBatch.Code.OK; @@ -338,8 +352,13 @@ private Runnable batchAddRoute(BatchMatchRequest request, IKVReader reader, IKVW } } if (updated) { - writer.put(groupMatchRecordKey, matchGroup.build().toByteString()); - newMatches.computeIfAbsent(tenantId, k -> new TreeSet<>(RouteMatcherComparator)).add(origRouteMatcher); + RouteDetail routeDetail = new RouteDetail(tenantId, origRouteMatcher, null); + RouteGroup routeGroup = matchGroup.build(); + Matching groupMatching = buildGroupMatchRoute(routeDetail, routeGroup); + writer.put(groupMatchRecordKey, routeGroup.toByteString()); + newMatches.computeIfAbsent(tenantId, k -> new TreeMap<>(RouteMatcherComparator)) + .computeIfAbsent(origRouteMatcher, k -> new HashSet<>()) + .add(groupMatching); } }); resultMap.forEach((tenantId, codes) -> { @@ -358,7 +377,7 @@ private Runnable batchAddRoute(BatchMatchRequest request, IKVReader reader, IKVW private Runnable batchRemoveRoute(BatchUnmatchRequest request, IKVReader reader, IKVWriter writer, - Map> removedMatches, + Map>> removedMatches, BatchUnmatchReply.Builder replyBuilder) { replyBuilder.setReqId(request.getReqId()); Map normalRoutesRemoved = new HashMap<>(); @@ -373,15 +392,19 @@ private Runnable batchRemoveRoute(BatchUnmatchRequest request, MatchRoute route = tenantUnmatchRequest.getRoute(i); RouteMatcher routeMatcher = route.getMatcher(); if (routeMatcher.getType() == RouteMatcher.Type.Normal) { - ByteString normalRouteKey = toNormalRouteKey(tenantId, routeMatcher, toReceiverUrl(route)); + String receiverUrl = toReceiverUrl(route); + ByteString normalRouteKey = toNormalRouteKey(tenantId, routeMatcher, receiverUrl); Optional incarOpt = reader.get(normalRouteKey).map(BSUtil::toLong); if (incarOpt.isPresent() && incarOpt.get() <= route.getIncarnation()) { + RouteDetail routeDetail = new RouteDetail(tenantId, routeMatcher, receiverUrl); + Matching normalMatching = buildNormalMatchRoute(routeDetail, incarOpt.get()); writer.delete(normalRouteKey); if (!delMatches.contains(normalRouteKey)) { normalRoutesRemoved.computeIfAbsent(tenantId, k -> new AtomicInteger()).incrementAndGet(); } - removedMatches.computeIfAbsent(tenantId, k -> new TreeSet<>(RouteMatcherComparator)) - .add(routeMatcher); + removedMatches.computeIfAbsent(tenantId, k -> new TreeMap<>(RouteMatcherComparator)) + .computeIfAbsent(routeMatcher, k -> new HashSet<>()) + .add(normalMatching); delMatches.add(normalRouteKey); codes[i] = BatchUnmatchReply.TenantBatch.Code.OK; } else { @@ -420,8 +443,12 @@ private Runnable batchRemoveRoute(BatchUnmatchRequest request, writer.put(groupRouteKey, RouteGroup.newBuilder().putAllMembers(existing).build().toByteString()); } - removedMatches.computeIfAbsent(tenantId, k -> new TreeSet<>(RouteMatcherComparator)) - .add(origRouteMatcher); + RouteDetail routeDetail = new RouteDetail(tenantId, origRouteMatcher, null); + RouteGroup routeGroup = RouteGroup.newBuilder().putAllMembers(existing).build(); + Matching newGroupMatching = buildGroupMatchRoute(routeDetail, routeGroup); + removedMatches.computeIfAbsent(tenantId, k -> new TreeMap<>(RouteMatcherComparator)) + .computeIfAbsent(origRouteMatcher, k -> new HashSet<>()) + .add(newGroupMatching); } } else { delGroupMembers.forEach((detail, resultIdx) -> resultMap.get(tenantId)[resultIdx] = diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/IMatchedRoutes.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/IMatchedRoutes.java index 0a1756daa..248e8084c 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/IMatchedRoutes.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/IMatchedRoutes.java @@ -20,10 +20,12 @@ package org.apache.bifromq.dist.worker.cache; import java.util.Set; +import org.apache.bifromq.dist.worker.schema.GroupMatching; import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; /** - * The result of matching a topic within a boundary. + * A mutable matched routes for a topic. */ public interface IMatchedRoutes { /** @@ -61,4 +63,51 @@ public interface IMatchedRoutes { */ Set routes(); + /** + * Add a normal matching to the matched routes. + * + * @param matching the normal matching to add + */ + AddResult addNormalMatching(NormalMatching matching); + + /** + * Remove a normal matching from the matched routes. + * + * @param matching the normal matching to remove + */ + void removeNormalMatching(NormalMatching matching); + + /** + * Add a new or override an existing group matching to the matched routes. + * + * @param matching the group matching to add or override + */ + AddResult putGroupMatching(GroupMatching matching); + + /** + * Remove an existing group matching. + * + * @param matching the group matching to remove + */ + void removeGroupMatching(GroupMatching matching); + + /** + * Adjust matched routes to new fanout limits. If the current fanout exceeds the new limits, the matched routes + * will be clamped to the new limits by removing excess persistent fanout matchings. + * If the new limits are higher than the previous and the current fanout is already at the previous limits, + * ReloadNeeded is returned. + * + * @param newMaxPersistentFanout the new maximum persistent fanout + * @param newMaxGroupFanout the new maximum group fanout + * @return the adjust result + */ + AdjustResult adjust(int newMaxPersistentFanout, int newMaxGroupFanout); + + enum AdjustResult { + Clamped, Adjusted, ReloadNeeded + } + + enum AddResult { + Added, Exists, ExceedFanoutLimit + } } diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ISubscriptionCache.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ISubscriptionCache.java index 17f8cecdd..b58c2a0e7 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ISubscriptionCache.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ISubscriptionCache.java @@ -14,26 +14,25 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker.cache; -import org.apache.bifromq.basekv.proto.Boundary; -import org.apache.bifromq.dist.worker.schema.Matching; -import org.apache.bifromq.type.RouteMatcher; import java.util.List; import java.util.Map; -import java.util.NavigableSet; import java.util.Set; import java.util.concurrent.CompletableFuture; +import org.apache.bifromq.basekv.proto.Boundary; +import org.apache.bifromq.dist.worker.cache.task.RefreshEntriesTask; +import org.apache.bifromq.dist.worker.schema.Matching; public interface ISubscriptionCache { CompletableFuture> get(String tenantId, String topic); boolean isCached(String tenantId, List filterLevels); - void refresh(Map> topicFiltersByTenant); + void refresh(Map tenantRefreshTasks); void reset(Boundary boundary); diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ITenantRouteCache.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ITenantRouteCache.java index 236c8cfeb..981657304 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ITenantRouteCache.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/ITenantRouteCache.java @@ -14,18 +14,17 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker.cache; -import org.apache.bifromq.basekv.proto.Boundary; -import org.apache.bifromq.dist.worker.schema.Matching; -import org.apache.bifromq.type.RouteMatcher; import java.util.List; -import java.util.NavigableSet; import java.util.Set; import java.util.concurrent.CompletableFuture; +import org.apache.bifromq.basekv.proto.Boundary; +import org.apache.bifromq.dist.worker.cache.task.RefreshEntriesTask; +import org.apache.bifromq.dist.worker.schema.Matching; /** * Cache for matched routes for given tenant. @@ -33,7 +32,7 @@ public interface ITenantRouteCache { boolean isCached(List filterLevels); - void refresh(NavigableSet routeMatchers); + void refresh(RefreshEntriesTask task); CompletableFuture> getMatch(String topic, Boundary currentTenantRange); diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/MatchedRoutes.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/MatchedRoutes.java new file mode 100644 index 000000000..ecc37bb9d --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/MatchedRoutes.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.bifromq.dist.worker.cache; + +import static org.apache.bifromq.plugin.eventcollector.ThreadLocalEventPool.getLocal; + +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.bifromq.dist.worker.schema.GroupMatching; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; +import org.apache.bifromq.plugin.eventcollector.IEventCollector; +import org.apache.bifromq.plugin.eventcollector.distservice.GroupFanoutThrottled; +import org.apache.bifromq.plugin.eventcollector.distservice.PersistentFanoutThrottled; + +public class MatchedRoutes implements IMatchedRoutes { + private final String tenantId; + private final String topic; + private final IEventCollector eventCollector; + private final Set allMatchings = Sets.newConcurrentHashSet(); + private final Map groupMatchings = Maps.newConcurrentMap(); + private final AtomicInteger persistentFanout = new AtomicInteger(); + private int maxPersistentFanout; + private int maxGroupFanout; + + public MatchedRoutes(String tenantId, + String topic, + IEventCollector eventCollector, + int maxPersistentFanout, + int maxGroupFanout) { + this.tenantId = tenantId; + this.topic = topic; + this.eventCollector = eventCollector; + this.maxPersistentFanout = maxPersistentFanout; + this.maxGroupFanout = maxGroupFanout; + } + + + @Override + public int maxPersistentFanout() { + return maxPersistentFanout; + } + + @Override + public int maxGroupFanout() { + return maxGroupFanout; + } + + @Override + public int persistentFanout() { + return persistentFanout.get(); + } + + @Override + public int groupFanout() { + return groupMatchings.size(); + } + + @Override + public Set routes() { + return allMatchings; + } + + @Override + public AddResult addNormalMatching(NormalMatching matching) { + if (allMatchings.add(matching)) { + if (matching.subBrokerId() == 1) { + if (persistentFanout.get() < maxPersistentFanout) { + persistentFanout.incrementAndGet(); + return AddResult.Added; + } else { + allMatchings.remove(matching); + eventCollector.report(getLocal(PersistentFanoutThrottled.class) + .tenantId(tenantId) + .topic(topic) + .mqttTopicFilter(matching.mqttTopicFilter()) + .maxCount(maxPersistentFanout) + ); + return AddResult.ExceedFanoutLimit; + } + } else { + return AddResult.Added; + } + } + return AddResult.Exists; + } + + @Override + public void removeNormalMatching(NormalMatching matching) { + if (allMatchings.remove(matching)) { + if (matching.subBrokerId() == 1) { + persistentFanout.decrementAndGet(); + } + } + } + + @Override + public AddResult putGroupMatching(GroupMatching matching) { + GroupMatching prev = groupMatchings.put(matching.mqttTopicFilter(), matching); + if (prev == null) { + if (groupMatchings.size() <= maxGroupFanout) { + allMatchings.add(matching); + return AddResult.Added; + } else { + groupMatchings.remove(matching.mqttTopicFilter()); + eventCollector.report(getLocal(GroupFanoutThrottled.class) + .tenantId(tenantId) + .topic(topic) + .mqttTopicFilter(matching.mqttTopicFilter()) + .maxCount(maxGroupFanout) + ); + return AddResult.ExceedFanoutLimit; + } + } else { + allMatchings.remove(prev); + allMatchings.add(matching); + return AddResult.Exists; + } + } + + @Override + public void removeGroupMatching(GroupMatching matching) { + assert matching.receivers().isEmpty(); + removeGroupMatching(matching.mqttTopicFilter()); + } + + private void removeGroupMatching(String mqttTopicFilter) { + GroupMatching existing = groupMatchings.remove(mqttTopicFilter); + if (existing != null) { + allMatchings.remove(existing); + } + } + + @Override + public AdjustResult adjust(int newMaxPersistentFanout, int newMaxGroupFanout) { + // current limit increases and cachedRoutes reached the previous limit, need refresh + if (maxPersistentFanout < newMaxPersistentFanout && persistentFanout.get() == maxPersistentFanout) { + return AdjustResult.ReloadNeeded; + } + if (maxGroupFanout < newMaxGroupFanout && groupMatchings.size() == maxGroupFanout) { + return AdjustResult.ReloadNeeded; + } + // current limit decreases and cachedRoutes exceeded the current limit, need refresh + boolean clamped = false; + if (maxPersistentFanout > newMaxPersistentFanout && persistentFanout.get() > newMaxPersistentFanout) { + // clamp persistent fanout + int toRemove = persistentFanout.get() - newMaxPersistentFanout; + List toRemoved = new ArrayList<>(toRemove); + for (Matching matching : allMatchings) { + if (toRemove == 0) { + break; + } + if (matching.type() == Matching.Type.Normal && ((NormalMatching) matching).subBrokerId() == 1) { + toRemoved.add((NormalMatching) matching); + toRemove--; + } + } + toRemoved.forEach(this::removeNormalMatching); + clamped = true; + } + if (maxGroupFanout > newMaxGroupFanout && groupMatchings.size() > newMaxGroupFanout) { + // clamp persistent fanout + int toRemove = groupMatchings.size() - newMaxGroupFanout; + List toRemoved = new ArrayList<>(toRemove); + for (String mqttTopicFilter : groupMatchings.keySet()) { + if (toRemove == 0) { + break; + } + toRemoved.add(mqttTopicFilter); + toRemove--; + } + toRemoved.forEach(this::removeGroupMatching); + clamped = true; + } + maxPersistentFanout = newMaxPersistentFanout; + maxGroupFanout = newMaxGroupFanout; + if (clamped) { + return AdjustResult.Clamped; + } + return AdjustResult.Adjusted; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/RouteCacheKey.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/RouteCacheKey.java new file mode 100644 index 000000000..9a1f187bc --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/RouteCacheKey.java @@ -0,0 +1,37 @@ +/* + * 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.bifromq.dist.worker.cache; + +import java.util.concurrent.atomic.AtomicReference; +import lombok.EqualsAndHashCode; +import org.apache.bifromq.basekv.proto.Boundary; + +@EqualsAndHashCode(cacheStrategy = EqualsAndHashCode.CacheStrategy.LAZY) +public final class RouteCacheKey { + public final String topic; + public final Boundary matchRecordBoundary; + @EqualsAndHashCode.Exclude + public final AtomicReference cachedMatchedRoutes = new AtomicReference<>(); + + public RouteCacheKey(String topic, Boundary matchRecordBoundary) { + this.topic = topic; + this.matchRecordBoundary = matchRecordBoundary; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/SubscriptionCache.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/SubscriptionCache.java index 5ff3804db..cf15ab2e4 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/SubscriptionCache.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/SubscriptionCache.java @@ -35,7 +35,6 @@ import java.time.Duration; import java.util.List; import java.util.Map; -import java.util.NavigableSet; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; @@ -46,12 +45,12 @@ import org.apache.bifromq.basekv.proto.KVRangeId; import org.apache.bifromq.basekv.store.api.IKVCloseableReader; import org.apache.bifromq.basekv.utils.KVRangeIdUtil; +import org.apache.bifromq.dist.worker.cache.task.RefreshEntriesTask; import org.apache.bifromq.dist.worker.schema.Matching; import org.apache.bifromq.plugin.eventcollector.IEventCollector; import org.apache.bifromq.plugin.settingprovider.ISettingProvider; import org.apache.bifromq.sysprops.props.DistCachedRoutesFanoutCheckIntervalSeconds; import org.apache.bifromq.sysprops.props.DistTopicMatchExpirySeconds; -import org.apache.bifromq.type.RouteMatcher; /** * Cache for subscription matching. @@ -125,7 +124,7 @@ public boolean isCached(String tenantId, List filterLevels) { } @Override - public void refresh(Map> topicFiltersByTenant) { + public void refresh(Map topicFiltersByTenant) { topicFiltersByTenant.forEach((tenantId, topicFilters) -> { ITenantRouteCache cache = tenantCache.getIfPresent(noRefreshExpiry(tenantId)); if (cache != null) { diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteCache.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteCache.java index e11d4be34..c0e1d693e 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteCache.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteCache.java @@ -23,35 +23,49 @@ import static org.apache.bifromq.plugin.settingprovider.Setting.MaxGroupFanout; import static org.apache.bifromq.plugin.settingprovider.Setting.MaxPersistentFanout; +import com.github.benmanes.caffeine.cache.AsyncCacheLoader; import com.github.benmanes.caffeine.cache.AsyncLoadingCache; -import com.github.benmanes.caffeine.cache.CacheLoader; import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.Scheduler; import com.github.benmanes.caffeine.cache.Ticker; import com.github.benmanes.caffeine.cache.Weigher; import java.time.Duration; -import java.util.HashMap; +import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.NavigableSet; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.bifromq.basekv.proto.Boundary; import org.apache.bifromq.dist.worker.TopicIndex; +import org.apache.bifromq.dist.worker.cache.task.AddRoutesTask; +import org.apache.bifromq.dist.worker.cache.task.LoadEntryTask; +import org.apache.bifromq.dist.worker.cache.task.RefreshEntriesTask; +import org.apache.bifromq.dist.worker.cache.task.ReloadEntryTask; +import org.apache.bifromq.dist.worker.cache.task.RemoveRoutesTask; +import org.apache.bifromq.dist.worker.cache.task.TenantRouteCacheTask; +import org.apache.bifromq.dist.worker.schema.GroupMatching; import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; import org.apache.bifromq.metrics.ITenantMeter; import org.apache.bifromq.metrics.TenantMetric; import org.apache.bifromq.plugin.settingprovider.ISettingProvider; import org.apache.bifromq.sysprops.props.DistMaxCachedRoutesPerTenant; import org.apache.bifromq.type.RouteMatcher; import org.checkerframework.checker.index.qual.NonNegative; -import org.checkerframework.checker.nullness.qual.Nullable; +import org.jspecify.annotations.NonNull; class TenantRouteCache implements ITenantRouteCache { private final String tenantId; + private final ISettingProvider settingProvider; + private final ITenantRouteMatcher matcher; private final AsyncLoadingCache routesCache; private final TopicIndex index; + private final Executor matchExecutor; + private final ConcurrentLinkedDeque tasks = new ConcurrentLinkedDeque<>(); + private final AtomicBoolean taskRunning = new AtomicBoolean(false); TenantRouteCache(String tenantId, ITenantRouteMatcher matcher, @@ -71,6 +85,9 @@ class TenantRouteCache implements ITenantRouteCache { Ticker ticker, Executor matchExecutor) { this.tenantId = tenantId; + this.matcher = matcher; + this.matchExecutor = matchExecutor; + this.settingProvider = settingProvider; index = new TopicIndex<>(); routesCache = Caffeine.newBuilder() .scheduler(Scheduler.systemScheduler()) @@ -90,106 +107,187 @@ class TenantRouteCache implements ITenantRouteCache { index.remove(key.topic, key); } }) - .buildAsync(new CacheLoader<>() { + .recordStats() + .buildAsync(new AsyncCacheLoader<>() { @Override - public @Nullable IMatchedRoutes load(RouteCacheKey key) { - int maxPersistentFanouts = settingProvider.provide(MaxPersistentFanout, tenantId); - int maxGroupFanouts = settingProvider.provide(MaxGroupFanout, tenantId); - ITenantMeter.get(tenantId).recordCount(TenantMetric.MqttRouteCacheMissCount); - Map results = matcher.matchAll(singleton(key.topic), - maxPersistentFanouts, maxGroupFanouts); - index.add(key.topic, key); - return results.get(key.topic); + public CompletableFuture asyncLoad(RouteCacheKey key, Executor executor) { + LoadEntryTask task = LoadEntryTask.of(key.topic, key); + submitCacheTask(task); + return task.future; } @Override - public Map loadAll(Set keys) { - ITenantMeter.get(tenantId).recordCount(TenantMetric.MqttRouteCacheMissCount, keys.size()); - Map topicToKeyMap = new HashMap<>(); - keys.forEach(k -> topicToKeyMap.put(k.topic(), k)); + public CompletableFuture asyncReload(RouteCacheKey key, + @NonNull IMatchedRoutes oldValue, + Executor executor) { int maxPersistentFanouts = settingProvider.provide(MaxPersistentFanout, tenantId); int maxGroupFanouts = settingProvider.provide(MaxGroupFanout, tenantId); - Map resultMap = matcher.matchAll(topicToKeyMap.keySet(), - maxPersistentFanouts, maxGroupFanouts); - Map result = new HashMap<>(); - for (Map.Entry entry : resultMap.entrySet()) { - RouteCacheKey key = topicToKeyMap.get(entry.getKey()); - result.put(key, entry.getValue()); - index.add(key.topic, key); - } - return result; - } - - @Override - public @Nullable IMatchedRoutes reload(RouteCacheKey key, IMatchedRoutes oldValue) { - int maxPersistentFanouts = settingProvider.provide(MaxPersistentFanout, tenantId); - int maxGroupFanouts = settingProvider.provide(MaxGroupFanout, tenantId); - if (needRefresh(oldValue, maxPersistentFanouts, maxGroupFanouts)) { - Map results = matcher.matchAll(singleton(key.topic), - maxPersistentFanouts, maxGroupFanouts); - return results.get(key.topic); - } else if (oldValue.maxPersistentFanout() != maxPersistentFanouts - || oldValue.maxGroupFanout() != maxGroupFanouts) { - return new IMatchedRoutes() { - @Override - public int maxPersistentFanout() { - return maxPersistentFanouts; - } - - @Override - public int maxGroupFanout() { - return maxGroupFanouts; - } - - @Override - public int persistentFanout() { - return oldValue.persistentFanout(); - } - - @Override - public int groupFanout() { - return oldValue.groupFanout(); - } - - @Override - public Set routes() { - return oldValue.routes(); - } - }; - } else { - return oldValue; + if (oldValue.adjust(maxPersistentFanouts, maxGroupFanouts) + == IMatchedRoutes.AdjustResult.ReloadNeeded) { + ReloadEntryTask task = ReloadEntryTask.of(key.topic, key, maxPersistentFanouts, + maxGroupFanouts); + submitCacheTask(task); + return task.future; } + return CompletableFuture.completedFuture(oldValue); } }); + ITenantMeter.counting(tenantId, TenantMetric.MqttRouteCacheHitCount, routesCache.synchronous(), + cache -> cache.stats().hitCount()); + ITenantMeter.counting(tenantId, TenantMetric.MqttRouteCacheMissCount, routesCache.synchronous(), + cache -> cache.stats().missCount()); + ITenantMeter.counting(tenantId, TenantMetric.MqttRouteCacheEvictCount, routesCache.synchronous(), + cache -> cache.stats().evictionCount()); ITenantMeter.gauging(tenantId, TenantMetric.MqttRouteCacheSize, routesCache.synchronous()::estimatedSize); } - private boolean needRefresh(IMatchedRoutes cachedRoutes, int maxPersistentFanouts, int maxGroupFanouts) { - return needRefresh(maxPersistentFanouts, cachedRoutes.maxPersistentFanout(), cachedRoutes.persistentFanout()) - || needRefresh(maxGroupFanouts, cachedRoutes.maxGroupFanout(), cachedRoutes.groupFanout()); - } - - private boolean needRefresh(int currentLimit, int previousLimit, int currentFanout) { - // current limit increases and cachedRoutes reached the previous limit, need refresh - // current limit decreases and cachedRoutes exceeded the current limit, need refresh - return (previousLimit < currentLimit && currentFanout == previousLimit) - || (previousLimit > currentLimit && currentFanout > currentLimit); - } - @Override public boolean isCached(List filterLevels) { return !index.match(filterLevels).isEmpty(); } @Override - public void refresh(NavigableSet routeMatchers) { - routeMatchers.forEach(topicFilter -> { - for (RouteCacheKey cacheKey : index.match(topicFilter.getFilterLevelList())) { - // we must invalidate the cache entry first to ensure the refresh will trigger a load - routesCache.synchronous().invalidate(cacheKey); - routesCache.synchronous().refresh(cacheKey); + public void refresh(RefreshEntriesTask task) { + // Enqueue and schedule an async refresh task; ensure only one runner active + submitCacheTask(task); + } + + private void submitCacheTask(TenantRouteCacheTask task) { + tasks.add(task); + tryScheduleTask(); + } + + private void tryScheduleTask() { + if (taskRunning.compareAndSet(false, true)) { + matchExecutor.execute(this::runTaskLoop); + } + } + + private void runTaskLoop() { + TenantRouteCacheTask task; + List> loadFutures = new ArrayList<>(tasks.size()); + outer: + while ((task = tasks.poll()) != null) { + switch (task.type()) { + case Load -> { + LoadEntryTask loadEntryTask = (LoadEntryTask) task; + loadFutures.add(CompletableFuture.runAsync(() -> { + int maxPersistentFanouts = settingProvider.provide(MaxPersistentFanout, tenantId); + int maxGroupFanouts = settingProvider.provide(MaxGroupFanout, tenantId); + String topic = loadEntryTask.topic; + RouteCacheKey cacheKey = loadEntryTask.cacheKey; + Map results = matcher.matchAll(singleton(topic), maxPersistentFanouts, + maxGroupFanouts); + + IMatchedRoutes matchedRoutes = results.get(loadEntryTask.topic); + // update reference + cacheKey.cachedMatchedRoutes.set(matchedRoutes); + // sync index + index.add(topic, cacheKey); + loadEntryTask.future.complete(matchedRoutes); + }, matchExecutor)); + } + case Reload -> { + ReloadEntryTask reloadEntryTask = (ReloadEntryTask) task; + loadFutures.add(CompletableFuture.runAsync(() -> { + String topic = reloadEntryTask.topic; + RouteCacheKey cacheKey = reloadEntryTask.cacheKey; + Map results = matcher.matchAll(singleton(topic), + reloadEntryTask.maxPersistentFanouts, reloadEntryTask.maxGroupFanouts); + IMatchedRoutes matchedRoutes = results.get(topic); + // update reference + cacheKey.cachedMatchedRoutes.set(matchedRoutes); + reloadEntryTask.future.complete(matchedRoutes); + }, matchExecutor)); + } + case AddRoutes, RemoveRoutes -> { + if (!loadFutures.isEmpty()) { + tasks.addFirst(task); + break outer; + } else { + switch (task.type()) { + case AddRoutes -> { + AddRoutesTask addTask = (AddRoutesTask) task; + for (RouteMatcher topicFilter : addTask.routes.keySet()) { + Set newMatchings = addTask.routes.get(topicFilter); + List filterLevels = topicFilter.getFilterLevelList(); + Set keys = index.match(filterLevels); + switch (topicFilter.getType()) { + case Normal -> { + for (RouteCacheKey cacheKey : keys) { + for (Matching matching : newMatchings) { + cacheKey.cachedMatchedRoutes.get() + .addNormalMatching((NormalMatching) matching); + } + } + } + case OrderedShare, UnorderedShare -> { + for (RouteCacheKey cacheKey : keys) { + for (Matching matching : newMatchings) { + cacheKey.cachedMatchedRoutes.get() + .putGroupMatching((GroupMatching) matching); + } + } + } + default -> { + // do nothing + } + } + } + } + case RemoveRoutes -> { + RemoveRoutesTask removeTask = (RemoveRoutesTask) task; + for (RouteMatcher topicFilter : removeTask.routes.keySet()) { + Set removedMatchings = removeTask.routes.get(topicFilter); + List filterLevels = topicFilter.getFilterLevelList(); + Set keys = index.match(filterLevels); + switch (topicFilter.getType()) { + case Normal -> { + for (RouteCacheKey cacheKey : keys) { + for (Matching matching : removedMatchings) { + cacheKey.cachedMatchedRoutes.get() + .removeNormalMatching((NormalMatching) matching); + } + } + } + case OrderedShare, UnorderedShare -> { + for (RouteCacheKey cacheKey : keys) { + for (Matching matching : removedMatchings) { + GroupMatching groupMatching = (GroupMatching) matching; + if (groupMatching.receivers().isEmpty()) { + cacheKey.cachedMatchedRoutes.get() + .removeGroupMatching(groupMatching); + } else { + cacheKey.cachedMatchedRoutes.get() + .putGroupMatching(groupMatching); + } + } + } + } + default -> { + // do nothing + } + } + } + } + default -> { + // do nothing + } + } + } + } + default -> { + // do nothing + } } - }); + } + CompletableFuture.allOf(loadFutures.toArray(CompletableFuture[]::new)) + .whenComplete((v, e) -> { + taskRunning.set(false); + if (!tasks.isEmpty()) { + tryScheduleTask(); + } + }); } @Override @@ -199,9 +297,9 @@ public CompletableFuture> getMatch(String topic, Boundary currentT @Override public void destroy() { + ITenantMeter.stopCounting(tenantId, TenantMetric.MqttRouteCacheMissCount); + ITenantMeter.stopCounting(tenantId, TenantMetric.MqttRouteCacheHitCount); + ITenantMeter.stopCounting(tenantId, TenantMetric.MqttRouteCacheEvictCount); ITenantMeter.stopGauging(tenantId, TenantMetric.MqttRouteCacheSize); } - - private record RouteCacheKey(String topic, Boundary matchRecordBoundary) { - } } diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java index 32a18def1..2fb3ae79d 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java @@ -27,7 +27,6 @@ import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.buildMatchRoute; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.tenantBeginKey; import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.tenantRouteStartKey; -import static org.apache.bifromq.plugin.eventcollector.ThreadLocalEventPool.getLocal; import com.google.protobuf.ByteString; import io.micrometer.core.instrument.Timer; @@ -36,18 +35,16 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import org.apache.bifromq.basekv.proto.Boundary; import org.apache.bifromq.basekv.store.api.IKVIterator; import org.apache.bifromq.basekv.store.api.IKVReader; import org.apache.bifromq.dist.trie.TopicFilterIterator; import org.apache.bifromq.dist.trie.TopicTrieNode; +import org.apache.bifromq.dist.worker.schema.GroupMatching; import org.apache.bifromq.dist.worker.schema.Matching; import org.apache.bifromq.dist.worker.schema.NormalMatching; import org.apache.bifromq.plugin.eventcollector.IEventCollector; -import org.apache.bifromq.plugin.eventcollector.distservice.GroupFanoutThrottled; -import org.apache.bifromq.plugin.eventcollector.distservice.PersistentFanoutThrottled; import org.apache.bifromq.util.TopicUtil; class TenantRouteMatcher implements ITenantRouteMatcher { @@ -72,12 +69,11 @@ public Map matchAll(Set topics, int maxGroupFanoutCount) { final Timer.Sample sample = Timer.start(); Map matchedRoutes = new HashMap<>(); - Set pFanoutTrimedTopic = new HashSet<>(); - Set gFanoutTrimedTopic = new HashSet<>(); TopicTrieNode.Builder topicTrieBuilder = TopicTrieNode.builder(false); topics.forEach(topic -> { topicTrieBuilder.addTopic(TopicUtil.parse(topic, false), topic); - matchedRoutes.put(topic, new MatchResult(maxPersistentFanoutCount, maxGroupFanoutCount)); + matchedRoutes.put(topic, + new MatchedRoutes(tenantId, topic, eventCollector, maxPersistentFanoutCount, maxGroupFanoutCount)); }); IKVReader rangeReader = kvReaderSupplier.get(); @@ -96,6 +92,7 @@ public Map matchAll(Set topics, // track seek itr.seek(tenantBoundary.getStartKey()); int probe = 0; + while (itr.isValid() && compare(itr.key(), tenantBoundary.getEndKey()) < 0) { // track itr.key() Matching matching = buildMatchRoute(itr.key(), itr.value()); @@ -110,30 +107,12 @@ public Map matchAll(Set topics, Set backingTopics = new HashSet<>(); for (Set topicSet : expansionSetItr.value().values()) { for (String topic : topicSet) { - MatchResult matchResult = (MatchResult) matchedRoutes.computeIfAbsent(topic, - k -> new MatchResult(maxPersistentFanoutCount, maxGroupFanoutCount)); + MatchedRoutes matchResult = (MatchedRoutes) matchedRoutes.computeIfAbsent(topic, + k -> new MatchedRoutes(tenantId, k, eventCollector, maxPersistentFanoutCount, + maxGroupFanoutCount)); switch (matching.type()) { - case Normal -> { - NormalMatching normalRoute = (NormalMatching) matching; - if (normalRoute.subBrokerId() == 1) { - if (matchResult.persistentFanoutCount.get() < maxPersistentFanoutCount) { - matchResult.persistentFanoutCount.incrementAndGet(); - matchResult.routes.add(matching); - } else { - pFanoutTrimedTopic.add(topic); - } - } else { - matchResult.routes.add(matching); - } - } - case Group -> { - if (matchResult.groupFanoutCount.get() < maxGroupFanoutCount) { - matchResult.groupFanoutCount.incrementAndGet(); - matchResult.routes.add(matching); - } else { - gFanoutTrimedTopic.add(topic); - } - } + case Normal -> matchResult.addNormalMatching((NormalMatching) matching); + case Group -> matchResult.putGroupMatching((GroupMatching) matching); default -> { // never happen } @@ -161,30 +140,12 @@ public Map matchAll(Set topics, } else { itr.next(); for (String topic : matchedTopics) { - MatchResult matchResult = (MatchResult) matchedRoutes.computeIfAbsent(topic, - k -> new MatchResult(maxPersistentFanoutCount, maxGroupFanoutCount)); + MatchedRoutes matchResult = (MatchedRoutes) matchedRoutes.computeIfAbsent(topic, + k -> new MatchedRoutes(tenantId, k, eventCollector, maxPersistentFanoutCount, + maxGroupFanoutCount)); switch (matching.type()) { - case Normal -> { - NormalMatching normalRoute = (NormalMatching) matching; - if (normalRoute.subBrokerId() == 1) { - if (matchResult.persistentFanoutCount.get() < maxPersistentFanoutCount) { - matchResult.persistentFanoutCount.incrementAndGet(); - matchResult.routes.add(matching); - } else { - pFanoutTrimedTopic.add(topic); - } - } else { - matchResult.routes.add(matching); - } - } - case Group -> { - if (matchResult.groupFanoutCount.get() < maxGroupFanoutCount) { - matchResult.groupFanoutCount.incrementAndGet(); - matchResult.routes.add(matching); - } else { - gFanoutTrimedTopic.add(topic); - } - } + case Normal -> matchResult.addNormalMatching((NormalMatching) matching); + case Group -> matchResult.putGroupMatching((GroupMatching) matching); default -> { // never happen } @@ -193,54 +154,6 @@ public Map matchAll(Set topics, } } sample.stop(timer); - pFanoutTrimedTopic.forEach(topic -> eventCollector.report(getLocal(PersistentFanoutThrottled.class) - .tenantId(tenantId) - .topic(topic) - .maxCount(maxPersistentFanoutCount) - )); - gFanoutTrimedTopic.forEach(topic -> eventCollector.report(getLocal(GroupFanoutThrottled.class) - .tenantId(tenantId) - .topic(topic) - .maxCount(maxGroupFanoutCount) - )); return matchedRoutes; } - - private static class MatchResult implements IMatchedRoutes { - private final Set routes = new HashSet<>(); - private final int maxPersistentFanoutCount; - private final int maxGroupFanoutCount; - private final AtomicInteger persistentFanoutCount = new AtomicInteger(0); - private final AtomicInteger groupFanoutCount = new AtomicInteger(0); - - private MatchResult(int maxPersistentFanoutCount, int maxGroupFanoutCount) { - this.maxPersistentFanoutCount = maxPersistentFanoutCount; - this.maxGroupFanoutCount = maxGroupFanoutCount; - } - - @Override - public int maxPersistentFanout() { - return maxPersistentFanoutCount; - } - - @Override - public int maxGroupFanout() { - return maxGroupFanoutCount; - } - - @Override - public int persistentFanout() { - return persistentFanoutCount.get(); - } - - @Override - public int groupFanout() { - return groupFanoutCount.get(); - } - - @Override - public Set routes() { - return routes; - } - } } diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/AddRoutesTask.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/AddRoutesTask.java new file mode 100644 index 000000000..28bfeae83 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/AddRoutesTask.java @@ -0,0 +1,42 @@ +/* + * 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.bifromq.dist.worker.cache.task; + +import java.util.NavigableMap; +import java.util.Set; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.type.RouteMatcher; + +public class AddRoutesTask extends RefreshEntriesTask { + + private AddRoutesTask( + NavigableMap> removed) { + super(removed); + } + + public static AddRoutesTask of(NavigableMap> added) { + return new AddRoutesTask(added); + } + + @Override + public CacheTaskType type() { + return CacheTaskType.AddRoutes; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/CacheTaskType.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/CacheTaskType.java new file mode 100644 index 000000000..1194d2b85 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/CacheTaskType.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.bifromq.dist.worker.cache.task; + +public enum CacheTaskType { + Load, Reload, AddRoutes, RemoveRoutes +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/LoadEntryTask.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/LoadEntryTask.java new file mode 100644 index 000000000..ead1fc6b5 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/LoadEntryTask.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.bifromq.dist.worker.cache.task; + +import java.util.concurrent.CompletableFuture; +import org.apache.bifromq.dist.worker.cache.IMatchedRoutes; +import org.apache.bifromq.dist.worker.cache.RouteCacheKey; + +public class LoadEntryTask extends TenantRouteCacheTask { + public final String topic; + public final RouteCacheKey cacheKey; + public final CompletableFuture future = new CompletableFuture<>(); + + private LoadEntryTask(String topic, RouteCacheKey cacheKey) { + this.topic = topic; + this.cacheKey = cacheKey; + } + + public static LoadEntryTask of(String topic, RouteCacheKey cacheKey) { + return new LoadEntryTask(topic, cacheKey); + } + + @Override + public final CacheTaskType type() { + return CacheTaskType.Load; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RefreshEntriesTask.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RefreshEntriesTask.java new file mode 100644 index 000000000..9a360081a --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RefreshEntriesTask.java @@ -0,0 +1,33 @@ +/* + * 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.bifromq.dist.worker.cache.task; + +import java.util.NavigableMap; +import java.util.Set; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.type.RouteMatcher; + +public abstract class RefreshEntriesTask extends TenantRouteCacheTask { + public final NavigableMap> routes; + + protected RefreshEntriesTask(NavigableMap> removed) { + this.routes = removed; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/ReloadEntryTask.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/ReloadEntryTask.java new file mode 100644 index 000000000..99ec5690f --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/ReloadEntryTask.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.bifromq.dist.worker.cache.task; + +import java.util.concurrent.CompletableFuture; +import org.apache.bifromq.dist.worker.cache.IMatchedRoutes; +import org.apache.bifromq.dist.worker.cache.RouteCacheKey; + +public class ReloadEntryTask extends TenantRouteCacheTask { + public final String topic; + public final RouteCacheKey cacheKey; + public final int maxPersistentFanouts; + public final int maxGroupFanouts; + public final CompletableFuture future = new CompletableFuture<>(); + + + private ReloadEntryTask(String topic, RouteCacheKey cacheKey, int maxPersistentFanouts, int maxGroupFanouts) { + this.topic = topic; + this.cacheKey = cacheKey; + this.maxPersistentFanouts = maxPersistentFanouts; + this.maxGroupFanouts = maxGroupFanouts; + } + + public static ReloadEntryTask of(String topic, + RouteCacheKey cacheKey, + int maxPersistentFanouts, + int maxGroupFanouts) { + return new ReloadEntryTask(topic, cacheKey, maxPersistentFanouts, maxGroupFanouts); + } + + @Override + public CacheTaskType type() { + return CacheTaskType.Reload; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RemoveRoutesTask.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RemoveRoutesTask.java new file mode 100644 index 000000000..0ea5819f1 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/RemoveRoutesTask.java @@ -0,0 +1,42 @@ +/* + * 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.bifromq.dist.worker.cache.task; + +import java.util.NavigableMap; +import java.util.Set; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.type.RouteMatcher; + +public class RemoveRoutesTask extends RefreshEntriesTask { + + private RemoveRoutesTask( + NavigableMap> removed) { + super(removed); + } + + public static RemoveRoutesTask of(NavigableMap> added) { + return new RemoveRoutesTask(added); + } + + @Override + public CacheTaskType type() { + return CacheTaskType.RemoveRoutes; + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/TenantRouteCacheTask.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/TenantRouteCacheTask.java new file mode 100644 index 000000000..061291b18 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/task/TenantRouteCacheTask.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.bifromq.dist.worker.cache.task; + +public abstract class TenantRouteCacheTask { + public abstract CacheTaskType type(); +} diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/DistWorkerCoProcTest.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/DistWorkerCoProcTest.java index c342b640f..2cee6cd59 100644 --- a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/DistWorkerCoProcTest.java +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/DistWorkerCoProcTest.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker; @@ -36,6 +36,14 @@ import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; +import lombok.SneakyThrows; import org.apache.bifromq.basekv.proto.Boundary; import org.apache.bifromq.basekv.proto.KVRangeId; import org.apache.bifromq.basekv.store.api.IKVCloseableReader; @@ -67,14 +75,6 @@ import org.apache.bifromq.type.TopicMessagePack; import org.apache.bifromq.util.BSUtil; import org.apache.bifromq.util.TopicUtil; -import com.google.protobuf.ByteString; -import com.google.protobuf.InvalidProtocolBufferException; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.function.Supplier; -import lombok.SneakyThrows; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -165,7 +165,8 @@ public void testMutateBatchRemoveRoute() { // Verify that matches are removed from the cache verify(routeCache, times(1)).refresh( - argThat(m -> m.containsKey("tenant1") && m.get("tenant1").contains(TopicUtil.from("topicFilter1")))); + argThat(m -> m.containsKey("tenant1") && m.get("tenant1").routes.keySet() + .contains(TopicUtil.from("topicFilter1")))); // Verify that tenant state is updated verify(tenantsState, times(1)).decNormalRoutes(eq("tenant1"), eq(1)); @@ -240,7 +241,8 @@ public void testMutateBatchRemoveRouteReturnsCorrectFact() { verify(writer, times(1)).delete(any(ByteString.class)); verify(routeCache, times(1)).refresh( - argThat(m -> m.containsKey("tenant1") && m.get("tenant1").contains(TopicUtil.from("topicFilter1")))); + argThat(m -> m.containsKey("tenant1") && m.get("tenant1").routes.keySet() + .contains(TopicUtil.from("topicFilter1")))); verify(tenantsState, times(1)).decNormalRoutes(eq("tenant1"), eq(1)); try { @@ -371,7 +373,8 @@ public void testRemoveRouteAffectFirstRouteTriggersRefresh() { verify(writer, times(1)).delete(any(ByteString.class)); verify(routeCache, times(1)).refresh(argThat(m -> m.containsKey("tenantA") - && m.get("tenantA").contains(TopicUtil.from("topicA")))); + && m.get("tenantA").routes.keySet() + .contains(TopicUtil.from("topicA")))); verify(tenantsState, times(1)).decNormalRoutes(eq("tenantA"), eq(1)); verify(reader, times(1)).refresh(); } @@ -411,7 +414,7 @@ public void testRemoveRouteAffectLastRouteTriggersRefresh() { verify(writer, times(1)).delete(any(ByteString.class)); verify(routeCache, times(1)).refresh(argThat(m -> m.containsKey("tenantB") - && m.get("tenantB").contains(TopicUtil.from("topicB")))); + && m.get("tenantB").routes.keySet().contains(TopicUtil.from("topicB")))); verify(tenantsState, times(1)).decNormalRoutes(eq("tenantB"), eq(1)); verify(reader, times(1)).refresh(); } diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/MatchedRoutesTest.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/MatchedRoutesTest.java new file mode 100644 index 000000000..078917e02 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/MatchedRoutesTest.java @@ -0,0 +1,302 @@ +/* + * 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.bifromq.dist.worker.cache; + +import static org.apache.bifromq.dist.worker.schema.Matchings.normalMatching; +import static org.apache.bifromq.dist.worker.schema.Matchings.receiverUrl; +import static org.apache.bifromq.dist.worker.schema.Matchings.unorderedGroupMatching; +import static org.apache.bifromq.plugin.eventcollector.EventType.GROUP_FANOUT_THROTTLED; +import static org.apache.bifromq.plugin.eventcollector.EventType.PERSISTENT_FANOUT_THROTTLED; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +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.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +import java.util.Map; +import org.apache.bifromq.dist.worker.schema.GroupMatching; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; +import org.apache.bifromq.plugin.eventcollector.IEventCollector; +import org.apache.bifromq.plugin.eventcollector.distservice.GroupFanoutThrottled; +import org.apache.bifromq.plugin.eventcollector.distservice.PersistentFanoutThrottled; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class MatchedRoutesTest { + private static final String TENANT_ID = "tenantA"; + private static final String TOPIC = "sensor/temperature"; + private static final String GROUP_FILTER = "sensor/+"; + + private IEventCollector eventCollector; + + @BeforeMethod + public void setUp() { + eventCollector = mock(IEventCollector.class); + } + + @Test + public void addPersistentNormalMatchingWithinLimit() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + NormalMatching persistent = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + + IMatchedRoutes.AddResult result = routes.addNormalMatching(persistent); + + assertEquals(result, IMatchedRoutes.AddResult.Added); + assertEquals(routes.persistentFanout(), 1); + assertEquals(routes.maxPersistentFanout(), 2); + assertTrue(routes.routes().contains(persistent)); + verify(eventCollector, never()).report(any()); + } + + @Test + public void addPersistentNormalMatchingExceedLimit() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 1, 2); + NormalMatching first = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + NormalMatching second = normalMatching(TENANT_ID, TOPIC, 1, "receiverB", "delivererB", 2); + + assertEquals(routes.addNormalMatching(first), IMatchedRoutes.AddResult.Added); + IMatchedRoutes.AddResult result = routes.addNormalMatching(second); + + assertEquals(result, IMatchedRoutes.AddResult.ExceedFanoutLimit); + assertEquals(routes.persistentFanout(), 1); + assertFalse(routes.routes().contains(second)); + verify(eventCollector, times(1)).report(argThat(e -> { + if (e.type() != PERSISTENT_FANOUT_THROTTLED) { + return false; + } + PersistentFanoutThrottled event = (PersistentFanoutThrottled) e; + return event.tenantId().equals(TENANT_ID) + && event.topic().equals(TOPIC) + && event.mqttTopicFilter().equals(second.mqttTopicFilter()) + && event.maxCount() == 1; + })); + } + + @Test + public void addNonPersistentNormalMatchingDoesNotAffectPersistentFanout() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 1, 2); + NormalMatching nonPersistent = normalMatching(TENANT_ID, TOPIC, 2, "receiverC", "delivererC", 1); + + IMatchedRoutes.AddResult result = routes.addNormalMatching(nonPersistent); + + assertEquals(result, IMatchedRoutes.AddResult.Added); + assertEquals(routes.persistentFanout(), 0); + assertTrue(routes.routes().contains(nonPersistent)); + } + + @Test + public void addDuplicateNormalMatchingReturnsExists() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + NormalMatching matching = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + + assertEquals(routes.addNormalMatching(matching), IMatchedRoutes.AddResult.Added); + IMatchedRoutes.AddResult second = routes.addNormalMatching(matching); + + assertEquals(second, IMatchedRoutes.AddResult.Exists); + assertEquals(routes.persistentFanout(), 1); + assertEquals(routes.routes().stream() + .filter(m -> m.type() == Matching.Type.Normal && ((NormalMatching) m).subBrokerId() == 1) + .count(), 1); + } + + @Test + public void removePersistentNormalMatching() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + NormalMatching persistent = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + routes.addNormalMatching(persistent); + + routes.removeNormalMatching(persistent); + + assertEquals(routes.persistentFanout(), 0); + assertFalse(routes.routes().contains(persistent)); + + // second removal should be a no-op + routes.removeNormalMatching(persistent); + assertEquals(routes.persistentFanout(), 0); + } + + @Test + public void removeNonPersistentNormalMatching() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + NormalMatching nonPersistent = normalMatching(TENANT_ID, TOPIC, 2, "receiverB", "delivererB", 1); + routes.addNormalMatching(nonPersistent); + + routes.removeNormalMatching(nonPersistent); + + assertEquals(routes.persistentFanout(), 0); + assertFalse(routes.routes().contains(nonPersistent)); + } + + @Test + public void putGroupMatchingWithinLimit() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + GroupMatching group = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 1L)); + + IMatchedRoutes.AddResult result = routes.putGroupMatching(group); + + assertEquals(result, IMatchedRoutes.AddResult.Added); + assertEquals(routes.groupFanout(), 1); + assertTrue(routes.routes().contains(group)); + verify(eventCollector, never()).report(any()); + } + + @Test + public void putGroupMatchingExceedLimit() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 1); + GroupMatching first = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 1L)); + GroupMatching second = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupB", + Map.of(receiverUrl(1, "receiverB", "delivererB"), 1L)); + + assertEquals(routes.putGroupMatching(first), IMatchedRoutes.AddResult.Added); + IMatchedRoutes.AddResult result = routes.putGroupMatching(second); + + assertEquals(result, IMatchedRoutes.AddResult.ExceedFanoutLimit); + assertEquals(routes.groupFanout(), 1); + assertFalse(routes.routes().contains(second)); + verify(eventCollector, times(1)).report(argThat(e -> { + if (e.type() != GROUP_FANOUT_THROTTLED) { + return false; + } + GroupFanoutThrottled event = (GroupFanoutThrottled) e; + return event.tenantId().equals(TENANT_ID) + && event.topic().equals(TOPIC) + && event.mqttTopicFilter().equals(second.mqttTopicFilter()) + && event.maxCount() == 1; + })); + } + + @Test + public void putGroupMatchingReplacesExistingGroup() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 3); + GroupMatching original = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 1L)); + GroupMatching replacement = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", + Map.of(receiverUrl(1, "receiverB", "delivererB"), 2L)); + + assertEquals(routes.putGroupMatching(original), IMatchedRoutes.AddResult.Added); + IMatchedRoutes.AddResult result = routes.putGroupMatching(replacement); + + assertEquals(result, IMatchedRoutes.AddResult.Exists); + assertEquals(routes.groupFanout(), 1); + assertTrue(routes.routes().contains(replacement)); + assertFalse(routes.routes().contains(original)); + } + + @Test + public void removeGroupMatching() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + GroupMatching group = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", Map.of()); + routes.putGroupMatching(group); + + routes.removeGroupMatching(group); + + assertEquals(routes.groupFanout(), 0); + assertFalse(routes.routes().contains(group)); + + routes.removeGroupMatching(group); + assertEquals(routes.groupFanout(), 0); + } + + @Test + public void adjustReturnsReloadNeededWhenPersistentLimitIncreases() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 1, 2); + routes.addNormalMatching(normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1)); + + IMatchedRoutes.AdjustResult result = routes.adjust(2, routes.maxGroupFanout()); + + assertEquals(result, IMatchedRoutes.AdjustResult.ReloadNeeded); + } + + @Test + public void adjustReturnsReloadNeededWhenGroupLimitIncreases() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 1); + routes.putGroupMatching(unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 1L))); + + IMatchedRoutes.AdjustResult result = routes.adjust(routes.maxPersistentFanout(), 2); + + assertEquals(result, IMatchedRoutes.AdjustResult.ReloadNeeded); + } + + @Test + public void adjustClampsPersistentFanoutWhenLimitDecreases() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 3, 2); + NormalMatching first = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + NormalMatching second = normalMatching(TENANT_ID, TOPIC, 1, "receiverB", "delivererB", 2); + NormalMatching third = normalMatching(TENANT_ID, TOPIC, 1, "receiverC", "delivererC", 3); + routes.addNormalMatching(first); + routes.addNormalMatching(second); + routes.addNormalMatching(third); + + IMatchedRoutes.AdjustResult result = routes.adjust(1, routes.maxGroupFanout()); + + assertEquals(result, IMatchedRoutes.AdjustResult.Clamped); + assertEquals(routes.persistentFanout(), 1); + long persistentCount = routes.routes().stream() + .filter(m -> m.type() == Matching.Type.Normal && ((NormalMatching) m).subBrokerId() == 1) + .count(); + assertEquals(persistentCount, 1); + + IMatchedRoutes.AdjustResult secondResult = routes.adjust(1, routes.maxGroupFanout()); + assertEquals(secondResult, IMatchedRoutes.AdjustResult.Adjusted); + } + + @Test + public void adjustClampsGroupFanoutWhenLimitDecreases() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 3); + GroupMatching g1 = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupA", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 1L)); + GroupMatching g2 = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupB", + Map.of(receiverUrl(1, "receiverB", "delivererB"), 1L)); + GroupMatching g3 = unorderedGroupMatching(TENANT_ID, GROUP_FILTER, "groupC", + Map.of(receiverUrl(1, "receiverC", "delivererC"), 1L)); + routes.putGroupMatching(g1); + routes.putGroupMatching(g2); + routes.putGroupMatching(g3); + + IMatchedRoutes.AdjustResult result = routes.adjust(routes.maxPersistentFanout(), 1); + + assertEquals(result, IMatchedRoutes.AdjustResult.Clamped); + assertEquals(routes.groupFanout(), 1); + long groupCount = routes.routes().stream().filter(m -> m.type() == Matching.Type.Group).count(); + assertEquals(groupCount, 1); + + IMatchedRoutes.AdjustResult secondResult = routes.adjust(routes.maxPersistentFanout(), 1); + assertEquals(secondResult, IMatchedRoutes.AdjustResult.Adjusted); + } + + @Test + public void adjustUpdatesLimitsWhenWithinBounds() { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, 2, 2); + + IMatchedRoutes.AdjustResult result = routes.adjust(4, 3); + + assertEquals(result, IMatchedRoutes.AdjustResult.Adjusted); + assertEquals(routes.maxPersistentFanout(), 4); + assertEquals(routes.maxGroupFanout(), 3); + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/SubscriptionCacheTest.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/SubscriptionCacheTest.java index 9c26c3c0c..edf624685 100644 --- a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/SubscriptionCacheTest.java +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/SubscriptionCacheTest.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.worker.cache; @@ -33,13 +33,6 @@ import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; -import org.apache.bifromq.basekv.proto.Boundary; -import org.apache.bifromq.basekv.proto.KVRangeId; -import org.apache.bifromq.basekv.store.api.IKVCloseableReader; -import org.apache.bifromq.dist.worker.Comparators; -import org.apache.bifromq.dist.worker.schema.Matching; -import org.apache.bifromq.type.RouteMatcher; -import org.apache.bifromq.util.TopicUtil; import com.github.benmanes.caffeine.cache.Ticker; import com.google.protobuf.ByteString; import java.time.Duration; @@ -47,13 +40,22 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.NavigableSet; +import java.util.NavigableMap; import java.util.Set; -import java.util.TreeSet; +import java.util.TreeMap; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executor; import java.util.concurrent.Executors; import java.util.function.Supplier; +import org.apache.bifromq.basekv.proto.Boundary; +import org.apache.bifromq.basekv.proto.KVRangeId; +import org.apache.bifromq.basekv.store.api.IKVCloseableReader; +import org.apache.bifromq.dist.worker.Comparators; +import org.apache.bifromq.dist.worker.cache.task.AddRoutesTask; +import org.apache.bifromq.dist.worker.cache.task.RefreshEntriesTask; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.type.RouteMatcher; +import org.apache.bifromq.util.TopicUtil; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -120,14 +122,15 @@ public void isCached() { @Test public void refresh() { String tenantId = "tenant1"; - NavigableSet routeMatchers = new TreeSet<>(Comparators.RouteMatcherComparator); - Map> matchesByTenant = new HashMap<>(); - matchesByTenant.put(tenantId, routeMatchers); + NavigableMap> routeMatchers = new TreeMap<>(Comparators.RouteMatcherComparator); + RefreshEntriesTask refreshEntriesTask = AddRoutesTask.of(routeMatchers); + Map matchesByTenant = new HashMap<>(); + matchesByTenant.put(tenantId, refreshEntriesTask); when(tenantRouteCacheFactoryMock.create(tenantId)).thenReturn(tenantRouteCacheMock); cache.refresh(matchesByTenant); - verify(tenantRouteCacheMock, never()).refresh(routeMatchers); + verify(tenantRouteCacheMock, never()).refresh(refreshEntriesTask); } @Test diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteCacheTest.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteCacheTest.java index 0efa00f44..0231e8751 100644 --- a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteCacheTest.java +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteCacheTest.java @@ -21,26 +21,43 @@ import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static org.apache.bifromq.basekv.utils.BoundaryUtil.FULL_BOUNDARY; +import static org.apache.bifromq.dist.worker.schema.Matchings.normalMatching; +import static org.apache.bifromq.dist.worker.schema.Matchings.receiverUrl; +import static org.apache.bifromq.dist.worker.schema.Matchings.unorderedGroupMatching; import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; import com.github.benmanes.caffeine.cache.Ticker; -import com.google.common.collect.Sets; import java.time.Duration; import java.util.Map; -import java.util.NavigableSet; +import java.util.NavigableMap; import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; import org.apache.bifromq.dist.worker.Comparators; import org.apache.bifromq.dist.worker.MeterTest; +import org.apache.bifromq.dist.worker.cache.task.AddRoutesTask; +import org.apache.bifromq.dist.worker.cache.task.RemoveRoutesTask; +import org.apache.bifromq.dist.worker.schema.GroupMatching; import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; import org.apache.bifromq.metrics.TenantMetric; +import org.apache.bifromq.plugin.eventcollector.IEventCollector; import org.apache.bifromq.plugin.settingprovider.ISettingProvider; import org.apache.bifromq.plugin.settingprovider.Setting; import org.apache.bifromq.type.RouteMatcher; @@ -50,290 +67,263 @@ import org.testng.annotations.Test; public class TenantRouteCacheTest extends MeterTest { + private static final String TENANT_ID = "tenantA"; + private static final String TOPIC = "sensor/temperature"; + private static final Duration EXPIRY = Duration.ofMinutes(1); + private static final Duration FANOUT_CHECK = Duration.ofMillis(200); - private TenantRouteCache cache; - private ManualTicker mockTicker; - private ITenantRouteMatcher mockMatcher; - private ISettingProvider mockSettingProvider; - private String tenantId; - private Duration expiryDuration; - private Duration fanoutCheckDuration; + private final Set caches = ConcurrentHashMap.newKeySet(); + + private ManualTicker ticker; + private ITenantRouteMatcher matcher; + private IEventCollector eventCollector; + private ISettingProvider settingProvider; + private ExecutorService executorToShutdown; @BeforeMethod - public void setup() { + public void setUp() { super.setup(); - tenantId = "tenant1"; - mockTicker = new ManualTicker(); - mockMatcher = mock(ITenantRouteMatcher.class); - mockSettingProvider = mock(ISettingProvider.class); - expiryDuration = Duration.ofMinutes(1); - fanoutCheckDuration = Duration.ofSeconds(2); - cache = new TenantRouteCache(tenantId, mockMatcher, mockSettingProvider, expiryDuration, - fanoutCheckDuration, directExecutor()); + ticker = new ManualTicker(); + matcher = mock(ITenantRouteMatcher.class); + settingProvider = mock(ISettingProvider.class); + eventCollector = mock(IEventCollector.class); + executorToShutdown = null; } @AfterMethod public void tearDown() { - cache.destroy(); + caches.forEach(TenantRouteCache::destroy); + caches.clear(); + if (executorToShutdown != null) { + executorToShutdown.shutdownNow(); + executorToShutdown = null; + } super.tearDown(); } - @SuppressWarnings("checkstyle:WhitespaceAround") @Test - public void getMatch() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(1, 0, 0))); - - Set cachedMatchings = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertNotNull(cachedMatchings); - assertEquals(cachedMatchings.size(), 1); - } + public void shouldLoadAndIndexRoutesOnFirstAccess() { + TenantRouteCache cache = newCache(directExecutor()); + NormalMatching existing = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + IMatchedRoutes matchedRoutes = mockRoutesWithBackingSet(10, 5, Set.of(existing)); - @Test - public void isCached() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 0, 0))); - cache.getMatch(topic, FULL_BOUNDARY).join(); - - assertTrue(cache.isCached(TopicUtil.from(topic).getFilterLevelList())); - } + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(10); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(10), eq(5))).thenReturn(Map.of(TOPIC, matchedRoutes)); - @Test - public void refreshToAddMatch() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn( - Map.of(topic, mockResult(1, 0, 0)), - Map.of(topic, mockResult(3, 0, 0)) - ); - - Set cachedMatchings = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertEquals(cachedMatchings.size(), 1); - - NavigableSet routeMatchers = Sets.newTreeSet(Comparators.RouteMatcherComparator); - routeMatchers.add(TopicUtil.from("#")); - cache.refresh(routeMatchers); - - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 3); + Set result = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + + assertEquals(result, Set.of(existing)); + assertTrue(cache.isCached(TopicUtil.from(TOPIC).getFilterLevelList())); + verify(matcher, times(1)).matchAll(eq(Set.of(TOPIC)), eq(10), eq(5)); } @Test - public void refreshToRemoveMatch() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(2, 0, 0))); + public void shouldReuseCachedValueWithoutReload() { + TenantRouteCache cache = newCache(directExecutor()); + NormalMatching existing = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + IMatchedRoutes matchedRoutes = mockRoutesWithBackingSet(10, 5, Set.of(existing)); - Set cachedMatchings = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertEquals(cachedMatchings.size(), 2); + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(10); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(10), eq(5))).thenReturn(Map.of(TOPIC, matchedRoutes)); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 0, 0))); + Set first = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + Set second = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); - NavigableSet routeMatchers = Sets.newTreeSet(Comparators.RouteMatcherComparator); - routeMatchers.add(TopicUtil.from("#")); - cache.refresh(routeMatchers); - - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().isEmpty()); + assertEquals(first, Set.of(existing)); + assertEquals(second, Set.of(existing)); + verify(matcher, times(1)).matchAll(eq(Set.of(TOPIC)), eq(10), eq(5)); } @Test - public void testPersistentFanoutCheck() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10, 5, 10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 10, 0))); - - Set cachedMatchings = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertEquals(cachedMatchings.size(), 10); + public void shouldReloadWhensReloadReturned() { + TenantRouteCache cache = newCache(directExecutor()); + NormalMatching existing = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + NormalMatching extra = normalMatching(TENANT_ID, TOPIC, 1, "receiverB", "delivererB", 2); + IMatchedRoutes initial = mockRoutesWithBackingSet(1, 5, Set.of(existing)); + IMatchedRoutes reloaded = mockRoutesWithBackingSet(2, 5, Set.of(existing, extra)); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(5), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 5, 0))); + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(1, 2); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5, 5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(1), eq(5))).thenReturn(Map.of(TOPIC, initial)); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(2), eq(5))).thenReturn(Map.of(TOPIC, reloaded)); - NavigableSet routeMatchers = Sets.newTreeSet(Comparators.RouteMatcherComparator); - routeMatchers.add(TopicUtil.from("#")); - // set maxPersistentFanout to 5 - cache.refresh(routeMatchers); - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 5); + Set first = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + assertEquals(first, Set.of(existing)); - // set maxPersistentFanout to 10 - cache.refresh(routeMatchers); - - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 10); - } + ticker.advance(FANOUT_CHECK.plusMillis(1)); + await().atMost(Duration.ofSeconds(5)).until(() -> cache.getMatch(TOPIC, FULL_BOUNDARY).join().contains(extra)); - @Test - public void testGroupFanoutCheck() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5, 3, 5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 0, 5))); - - Set cachedMatchings = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertEquals(cachedMatchings.size(), 5); - - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(3))) - .thenReturn(Map.of(topic, mockResult(0, 0, 3))); - - NavigableSet routeMatchers = Sets.newTreeSet(Comparators.RouteMatcherComparator); - routeMatchers.add(TopicUtil.from("#")); - // set maxGroupFanout to 3 - cache.refresh(routeMatchers); - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 3); - - // set maxGroupFanout to 5 - cache.refresh(routeMatchers); - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 5); + verify(matcher, times(2)).matchAll(eq(Set.of(TOPIC)), anyInt(), anyInt()); } @Test - public void reloadOnPersistentFanoutLimitDecrease() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10, 5); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); + public void shouldNotReloadWhenAdjustedReturned() { + TenantRouteCache cache = newCache(directExecutor()); + NormalMatching existing = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + IMatchedRoutes matchedRoutes = mockRoutesWithBackingSet(10, 5, Set.of(existing)); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 10, 0))); + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(10, 20); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5, 5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(10), eq(5))).thenReturn(Map.of(TOPIC, matchedRoutes)); - assertEquals(cache.getMatch(topic, FULL_BOUNDARY).join().size(), 10); + Set first = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + assertEquals(first, Set.of(existing)); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(5), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 5, 0))); + ticker.advance(FANOUT_CHECK.plusMillis(1)); + Set second = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + assertEquals(second, Set.of(existing)); - mockTicker.advance(Duration.ofMillis(10)); - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 5); + verify(matcher, times(1)).matchAll(eq(Set.of(TOPIC)), anyInt(), anyInt()); } @Test - public void reloadOnPersistentFanoutLimitIncreaseWhenClamped() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(5, 10); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(5), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 5, 0))); + public void shouldApplyAddRoutesTask() { + NormalMatching existing = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + IMatchedRoutes matchedRoutes = mockRoutesWithBackingSet(10, 5, Set.of(existing)); - assertEquals(cache.getMatch(topic, FULL_BOUNDARY).join().size(), 5); + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(10); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(10), eq(5))).thenReturn(Map.of(TOPIC, matchedRoutes)); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 10, 0))); - - mockTicker.advance(Duration.ofMillis(10)); - await().until(() -> cache.getMatch(topic, FULL_BOUNDARY).join().size() == 10); - } + TenantRouteCache cache = newCache(directExecutor()); - @Test - public void reloadDoesNotRematchWhenIncreaseWithoutClamp() { - String topic = "home/sensor/temperature"; - when(mockSettingProvider.provide(eq(Setting.MaxPersistentFanout), eq(tenantId))).thenReturn(10, 20); - when(mockSettingProvider.provide(eq(Setting.MaxGroupFanout), eq(tenantId))).thenReturn(5); - when(mockMatcher.matchAll(eq(Set.of(topic)), eq(10), eq(5))) - .thenReturn(Map.of(topic, mockResult(0, 3, 0))); + assertTrue(cache.getMatch(TOPIC, FULL_BOUNDARY).join().contains(existing)); - Set first = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertEquals(first.size(), 3); + NormalMatching newNormal = normalMatching(TENANT_ID, TOPIC, 1, "receiverB", "delivererB", 2); + GroupMatching newGroup = unorderedGroupMatching(TENANT_ID, "sensor/#", "groupA", + Map.of(receiverUrl(1, "receiverC", "delivererC"), 1L)); - mockTicker.advance(Duration.ofMillis(10)); + NavigableMap> additions = new TreeMap<>(Comparators.RouteMatcherComparator); + additions.put(newNormal.matcher, Set.of(newNormal)); + additions.put(newGroup.matcher, Set.of(newGroup)); - Set second = cache.getMatch(topic, FULL_BOUNDARY).join(); - assertEquals(second.size(), 3); + cache.refresh(AddRoutesTask.of(additions)); - verify(mockMatcher, times(1)).matchAll(eq(Set.of(topic)), eq(10), eq(5)); + await().atMost(Duration.ofSeconds(5)).until(() -> { + Set current = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + return current.contains(newNormal) && current.contains(newGroup); + }); } @Test - public void testDestroy() { - TenantRouteCache cache = new TenantRouteCache(tenantId, mockMatcher, mockSettingProvider, - expiryDuration, fanoutCheckDuration, directExecutor()); - assertGauge(tenantId, TenantMetric.MqttRouteCacheSize); - cache.destroy(); - assertNoGauge(tenantId, TenantMetric.MqttRouteCacheSize); + public void shouldApplyRemoveRoutesTask() { + TenantRouteCache cache = newCache(directExecutor()); + NormalMatching normal = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + GroupMatching removableGroup = unorderedGroupMatching(TENANT_ID, "sensor/#", "groupRemove", + Map.of(receiverUrl(1, "receiverB", "delivererB"), 1L)); + GroupMatching updatableGroup = unorderedGroupMatching(TENANT_ID, "sensor/+", "groupUpdate", + Map.of(receiverUrl(1, "receiverC", "delivererC"), 1L, + receiverUrl(1, "receiverD", "delivererD"), 1L)); + IMatchedRoutes matchedRoutes = mockRoutesWithBackingSet(10, 5, Set.of(normal, removableGroup, updatableGroup)); + + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(10); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(10), eq(5))).thenReturn(Map.of(TOPIC, matchedRoutes)); + + cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + + GroupMatching emptyGroup = unorderedGroupMatching(TENANT_ID, "sensor/#", "groupRemove", Map.of()); + GroupMatching reducedGroup = unorderedGroupMatching(TENANT_ID, "sensor/+", + "groupUpdate", Map.of(receiverUrl(1, "receiverC", "delivererC"), 1L)); + + NavigableMap> removals = new TreeMap<>(Comparators.RouteMatcherComparator); + removals.put(normal.matcher, Set.of(normal)); + removals.put(removableGroup.matcher, Set.of(emptyGroup)); + removals.put(updatableGroup.matcher, Set.of(reducedGroup)); + + cache.refresh(RemoveRoutesTask.of(removals)); + + await().atMost(Duration.ofSeconds(5)).until(() -> { + Set current = cache.getMatch(TOPIC, FULL_BOUNDARY).join(); + boolean normalRemoved = !current.contains(normal); + boolean groupRemoved = !current.contains(removableGroup); + boolean groupUpdated = current.contains(reducedGroup) + && current.stream().noneMatch(m -> m instanceof GroupMatching gm + && gm.matcher.equals(updatableGroup.matcher) + && gm.receivers().equals(updatableGroup.receivers())); + return normalRemoved && groupRemoved && groupUpdated; + }); } - private IMatchedRoutes mockResult(int transientFanout, int persistentFanout, int groupFanout) { - Set matchings = Sets.newHashSet(); - for (int i = 0; i < transientFanout; i++) { - Matching transientMatching = new Matching(tenantId, RouteMatcher.newBuilder() - .setMqttTopicFilter("transient/topic/" + i) - .build()) { - @Override - public Type type() { - return Matching.Type.Normal; - } - }; - matchings.add(transientMatching); - } - for (int i = 0; i < persistentFanout; i++) { - Matching transientMatching = new Matching(tenantId, RouteMatcher.newBuilder() - .setMqttTopicFilter("transient/topic/" + i) - .build()) { - @Override - public Type type() { - return Matching.Type.Normal; - } - }; - matchings.add(transientMatching); - } - for (int i = 0; i < groupFanout; i++) { - Matching transientMatching = new Matching(tenantId, RouteMatcher.newBuilder() - .setMqttTopicFilter("transient/topic/" + i) - .build()) { - @Override - public Type type() { - return Type.Group; - } - }; - matchings.add(transientMatching); - } - return new IMatchedRoutes() { - @Override - public int maxPersistentFanout() { - return persistentFanout; - } + @Test + public void shouldQueueTasksUntilLoadCompletes() throws Exception { + executorToShutdown = Executors.newSingleThreadExecutor(); + TenantRouteCache cache = newCache(executorToShutdown); + NormalMatching existing = normalMatching(TENANT_ID, TOPIC, 1, "receiverA", "delivererA", 1); + IMatchedRoutes matchedRoutes = mockRoutesWithBackingSet(10, 5, Set.of(existing)); + + CountDownLatch loadStarted = new CountDownLatch(1); + CountDownLatch allowLoad = new CountDownLatch(1); + + when(settingProvider.provide(eq(Setting.MaxPersistentFanout), eq(TENANT_ID))).thenReturn(10); + when(settingProvider.provide(eq(Setting.MaxGroupFanout), eq(TENANT_ID))).thenReturn(5); + when(matcher.matchAll(eq(Set.of(TOPIC)), eq(10), eq(5))).thenAnswer(invocation -> { + loadStarted.countDown(); + allowLoad.await(5, TimeUnit.SECONDS); + return Map.of(TOPIC, matchedRoutes); + }); + + CompletableFuture> future = cache.getMatch(TOPIC, FULL_BOUNDARY); + assertFalse(future.isDone()); + assertTrue(loadStarted.await(5, TimeUnit.SECONDS)); + + NormalMatching newNormal = normalMatching(TENANT_ID, TOPIC, 1, "receiverB", "delivererB", 2); + NavigableMap> additions = new TreeMap<>(Comparators.RouteMatcherComparator); + additions.put(newNormal.matcher, Set.of(newNormal)); + cache.refresh(AddRoutesTask.of(additions)); + + assertFalse(future.isDone()); + allowLoad.countDown(); + + Set initial = future.join(); + assertTrue(initial.contains(existing)); + + await().atMost(Duration.ofSeconds(5)) + .until(() -> cache.getMatch(TOPIC, FULL_BOUNDARY).join().contains(newNormal)); + } - @Override - public int maxGroupFanout() { - return groupFanout; - } + @Test + public void shouldDestroyMeters() { + TenantRouteCache cache = new TenantRouteCache(TENANT_ID, matcher, settingProvider, EXPIRY, FANOUT_CHECK, + directExecutor()); + caches.add(cache); + assertGauge(TENANT_ID, TenantMetric.MqttRouteCacheSize); - @Override - public int persistentFanout() { - return persistentFanout; - } + cache.destroy(); + assertNoGauge(TENANT_ID, TenantMetric.MqttRouteCacheSize); + } - @Override - public int groupFanout() { - return groupFanout; - } + private TenantRouteCache newCache(Executor executor) { + TenantRouteCache cache = new TenantRouteCache(TENANT_ID, matcher, settingProvider, + EXPIRY, FANOUT_CHECK, ticker, executor); + caches.add(cache); + return cache; + } - @Override - public Set routes() { - return matchings; + private IMatchedRoutes mockRoutesWithBackingSet(int maxPersistentFanout, int maxGroupFanout, Set seed) { + MatchedRoutes routes = new MatchedRoutes(TENANT_ID, TOPIC, eventCollector, maxPersistentFanout, maxGroupFanout); + for (Matching m : seed) { + if (m instanceof NormalMatching nm) { + routes.addNormalMatching(nm); + } else if (m instanceof GroupMatching gm) { + routes.putGroupMatching(gm); } - }; + } + return routes; } private static final class ManualTicker implements Ticker { - private long nanos = 0L; + private final AtomicLong nanos = new AtomicLong(); @Override public long read() { - return nanos; + return nanos.get(); } - void advance(Duration d) { - nanos += d.toNanos(); + void advance(Duration duration) { + nanos.addAndGet(duration.toNanos()); } } -} \ No newline at end of file +} diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcherTest.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcherTest.java new file mode 100644 index 000000000..e7bd53eab --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcherTest.java @@ -0,0 +1,507 @@ +/* + * 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.bifromq.dist.worker.cache; + +import static org.apache.bifromq.basekv.utils.BoundaryUtil.FULL_BOUNDARY; +import static org.apache.bifromq.basekv.utils.BoundaryUtil.toBoundary; +import static org.apache.bifromq.basekv.utils.BoundaryUtil.upperBound; +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toGroupRouteKey; +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toNormalRouteKey; +import static org.apache.bifromq.dist.worker.schema.Matchings.normalMatching; +import static org.apache.bifromq.dist.worker.schema.Matchings.receiverUrl; +import static org.apache.bifromq.dist.worker.schema.Matchings.unorderedGroupMatching; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + +import com.google.protobuf.ByteString; +import io.micrometer.core.instrument.Timer; +import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import java.util.TreeMap; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Supplier; +import org.apache.bifromq.basekv.proto.Boundary; +import org.apache.bifromq.basekv.store.api.IKVIterator; +import org.apache.bifromq.basekv.store.api.IKVReader; +import org.apache.bifromq.basekv.utils.BoundaryUtil; +import org.apache.bifromq.dist.rpc.proto.RouteGroup; +import org.apache.bifromq.dist.worker.schema.GroupMatching; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; +import org.apache.bifromq.plugin.eventcollector.Event; +import org.apache.bifromq.plugin.eventcollector.IEventCollector; +import org.apache.bifromq.plugin.eventcollector.distservice.GroupFanoutThrottled; +import org.apache.bifromq.plugin.eventcollector.distservice.PersistentFanoutThrottled; +import org.apache.bifromq.util.BSUtil; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class TenantRouteMatcherTest { + private static final String TENANT_ID = "tenantA"; + private static final String OTHER_TENANT = "tenantB"; + + private SimpleMeterRegistry meterRegistry; + private RecordingEventCollector eventCollector; + private Timer matchTimer; + + private static NavigableMap newTreeMap() { + return new TreeMap<>(ByteString.unsignedLexicographicalComparator()); + } + + @BeforeMethod + public void setUp() { + meterRegistry = new SimpleMeterRegistry(); + eventCollector = new RecordingEventCollector(); + matchTimer = meterRegistry.timer("tenantRouteMatch"); + } + + @AfterMethod + public void tearDown() { + meterRegistry.close(); + } + + @Test + public void matchAllReturnsEmptyWhenNoTenantDataPresent() { + NavigableMap kvData = newTreeMap(); + NormalMatching otherTenantRoute = normalMatching(OTHER_TENANT, "sensors/+/temp", 1, "receiverX", "delivererX", + 1); + kvData.put(toNormalRouteKey(OTHER_TENANT, otherTenantRoute.matcher, otherTenantRoute.receiverUrl()), + BSUtil.toByteString(otherTenantRoute.incarnation())); + + Set topics = Set.of("sensors/device1/temp", "sensors/device1/humidity"); + TenantRouteMatcher matcher = + new TenantRouteMatcher(TENANT_ID, () -> new TreeMapKVReader(kvData), eventCollector, matchTimer); + + Map matchedRoutes = matcher.matchAll(topics, 10, 10); + + assertEquals(matchedRoutes.keySet(), topics); + matchedRoutes.values().forEach(routes -> { + assertTrue(routes.routes().isEmpty()); + assertEquals(routes.persistentFanout(), 0); + assertEquals(routes.groupFanout(), 0); + }); + assertTrue(eventCollector.events().isEmpty()); + } + + @Test + public void matchAllAcrossMultipleTopics() { + NavigableMap kvData = newTreeMap(); + + NormalMatching tempRoute = normalMatching(TENANT_ID, "sensors/+/temp", 1, "receiverA", "delivererA", 1); + kvData.put(toNormalRouteKey(TENANT_ID, tempRoute.matcher, tempRoute.receiverUrl()), + BSUtil.toByteString(tempRoute.incarnation())); + + NormalMatching humidityRoute = + normalMatching(TENANT_ID, "sensors/+/humidity", 1, "receiverB", "delivererB", 2); + kvData.put( + toNormalRouteKey(TENANT_ID, humidityRoute.matcher, humidityRoute.receiverUrl()), + BSUtil.toByteString(humidityRoute.incarnation())); + + TenantRouteMatcher matcher = + new TenantRouteMatcher(TENANT_ID, () -> new TreeMapKVReader(kvData), eventCollector, matchTimer); + + Set topics = Set.of( + "sensors/device1/temp", + "sensors/device1/humidity", + "sensors/device2/temp"); + + Map matchedRoutes = matcher.matchAll(topics, 10, 10); + + assertEquals(matchedRoutes.keySet(), topics); + assertTrue(matchedRoutes.get("sensors/device1/temp").routes().contains(tempRoute)); + assertTrue(matchedRoutes.get("sensors/device2/temp").routes().contains(tempRoute)); + assertTrue(matchedRoutes.get("sensors/device1/humidity").routes().contains(humidityRoute)); + + assertEquals(matchedRoutes.get("sensors/device1/temp").persistentFanout(), 1); + assertEquals(matchedRoutes.get("sensors/device2/temp").persistentFanout(), 1); + assertEquals(matchedRoutes.get("sensors/device1/humidity").persistentFanout(), 1); + matchedRoutes.values().forEach(routes -> assertEquals(routes.groupFanout(), 0)); + assertTrue(eventCollector.events().isEmpty()); + } + + @Test + public void reuseCachedFilterMatchesForRepeatedSubscriptions() { + NavigableMap kvData = newTreeMap(); + + NormalMatching firstRoute = normalMatching(TENANT_ID, "devices/+/status", 1, "receiverA", "delivererA", 1); + NormalMatching secondRoute = normalMatching(TENANT_ID, "devices/+/status", 2, "receiverB", "delivererB", 1); + + kvData.put(toNormalRouteKey(TENANT_ID, firstRoute.matcher, firstRoute.receiverUrl()), + BSUtil.toByteString(firstRoute.incarnation())); + kvData.put(toNormalRouteKey(TENANT_ID, secondRoute.matcher, secondRoute.receiverUrl()), + BSUtil.toByteString(secondRoute.incarnation())); + + TenantRouteMatcher matcher = + new TenantRouteMatcher(TENANT_ID, () -> new TreeMapKVReader(kvData), eventCollector, matchTimer); + + Set topics = Set.of("devices/a/status", "devices/b/status"); + + Map matchedRoutes = matcher.matchAll(topics, 5, 5); + + topics.forEach(topic -> { + IMatchedRoutes routes = matchedRoutes.get(topic); + assertTrue(routes.routes().contains(firstRoute)); + assertTrue(routes.routes().contains(secondRoute)); + assertEquals(routes.persistentFanout(), 1); // only first route counts as persistent (subBrokerId == 1) + assertEquals(routes.groupFanout(), 0); + }); + assertTrue(eventCollector.events().isEmpty()); + } + + @Test + public void matchAllWithSharedSubscription() { + NavigableMap kvData = newTreeMap(); + + GroupMatching groupMatching = unorderedGroupMatching( + TENANT_ID, + "alerts/+/+/temperature", + "groupAlpha", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 10L, + receiverUrl(2, "receiverB", "delivererB"), 11L)); + + kvData.put(toGroupRouteKey(TENANT_ID, groupMatching.matcher), + RouteGroup.newBuilder().putAllMembers(groupMatching.receivers()).build().toByteString()); + + TenantRouteMatcher matcher = + new TenantRouteMatcher(TENANT_ID, () -> new TreeMapKVReader(kvData), eventCollector, matchTimer); + + Set topics = Set.of("alerts/site1/device1/temperature", "alerts/site1/device2/temperature"); + + Map matchedRoutes = matcher.matchAll(topics, 10, 10); + + topics.forEach(topic -> { + IMatchedRoutes routes = matchedRoutes.get(topic); + assertTrue(routes.routes().contains(groupMatching)); + assertEquals(routes.persistentFanout(), 0); + assertEquals(routes.groupFanout(), 1); + }); + assertTrue(eventCollector.events().isEmpty()); + } + + @Test + public void skipNonMatchingRoutesAndFallbackToSeek() { + NavigableMap kvData = newTreeMap(); + for (int i = 0; i < 21; i++) { + NormalMatching noise = normalMatching(TENANT_ID, "invalid/" + i, 1, "noise" + i, "deliverer" + i, i); + kvData.put(toNormalRouteKey(TENANT_ID, noise.matcher, noise.receiverUrl()), + BSUtil.toByteString(noise.incarnation())); + } + + NormalMatching valid = normalMatching(TENANT_ID, "metrics/+/cpu", 1, "receiverA", "delivererA", 1); + kvData.put(toNormalRouteKey(TENANT_ID, valid.matcher, valid.receiverUrl()), + BSUtil.toByteString(valid.incarnation())); + + TreeMapKVReader kvReader = new TreeMapKVReader(kvData); + TenantRouteMatcher matcher = new TenantRouteMatcher(TENANT_ID, () -> kvReader, eventCollector, matchTimer); + + Set topics = Set.of("metrics/server1/cpu"); + + Map matchedRoutes = matcher.matchAll(topics, 10, 10); + + IMatchedRoutes result = matchedRoutes.get("metrics/server1/cpu"); + assertTrue(result.routes().contains(valid)); + assertEquals(result.persistentFanout(), 1); + assertEquals(result.groupFanout(), 0); + + TreeMapKVIterator iterator = kvReader.lastIterator(); + assertNotNull(iterator); + assertTrue(iterator.getSeekCount() >= 2); // initial seek + fallback seek + assertTrue(iterator.getNextCount() >= 21); // probed through noise entries + assertTrue(eventCollector.events().isEmpty()); + } + + @Test + public void isolateRoutesByTenant() { + NavigableMap kvData = newTreeMap(); + + NormalMatching tenantRoute = normalMatching(TENANT_ID, "devices/+/signal", 1, "receiverA", "delivererA", 1); + kvData.put(toNormalRouteKey(TENANT_ID, tenantRoute.matcher, tenantRoute.receiverUrl()), + BSUtil.toByteString(tenantRoute.incarnation())); + + NormalMatching otherTenantRoute = + normalMatching(OTHER_TENANT, "devices/+/signal", 1, "receiverB", "delivererB", 1); + kvData.put(toNormalRouteKey(OTHER_TENANT, otherTenantRoute.matcher, otherTenantRoute.receiverUrl()), + BSUtil.toByteString(otherTenantRoute.incarnation())); + + Supplier readerSupplier = () -> new TreeMapKVReader(kvData); + + TenantRouteMatcher matcherTenant = + new TenantRouteMatcher(TENANT_ID, readerSupplier, eventCollector, matchTimer); + TenantRouteMatcher matcherOther = + new TenantRouteMatcher(OTHER_TENANT, readerSupplier, eventCollector, matchTimer); + + Set topics = Set.of("devices/a/signal"); + + Map tenantMatches = matcherTenant.matchAll(topics, 10, 10); + assertTrue(tenantMatches.get("devices/a/signal").routes().contains(tenantRoute)); + assertFalse(tenantMatches.get("devices/a/signal").routes().contains(otherTenantRoute)); + + Map otherMatches = matcherOther.matchAll(topics, 10, 10); + assertTrue(otherMatches.get("devices/a/signal").routes().contains(otherTenantRoute)); + assertFalse(otherMatches.get("devices/a/signal").routes().contains(tenantRoute)); + + assertTrue(eventCollector.events().isEmpty()); + } + + @Test + public void triggerPersistentFanoutThrottling() { + NavigableMap kvData = newTreeMap(); + + NormalMatching first = normalMatching(TENANT_ID, "alarms/+/critical", 1, "receiverA", "delivererA", 1); + NormalMatching second = normalMatching(TENANT_ID, "alarms/+/critical", 1, "receiverB", "delivererB", 2); + + kvData.put(toNormalRouteKey(TENANT_ID, first.matcher, first.receiverUrl()), + BSUtil.toByteString(first.incarnation())); + kvData.put(toNormalRouteKey(TENANT_ID, second.matcher, second.receiverUrl()), + BSUtil.toByteString(second.incarnation())); + + TenantRouteMatcher matcher = + new TenantRouteMatcher(TENANT_ID, () -> new TreeMapKVReader(kvData), eventCollector, matchTimer); + + Map matchedRoutes = + matcher.matchAll(Set.of("alarms/device1/critical"), 1, 10); + + IMatchedRoutes routes = matchedRoutes.get("alarms/device1/critical"); + assertEquals(routes.persistentFanout(), 1); + assertEquals(routes.groupFanout(), 0); + assertEquals(routes.routes().size(), 1); + + List events = eventCollector.eventsOfType(PersistentFanoutThrottled.class); + assertEquals(events.size(), 1); + PersistentFanoutThrottled event = events.get(0); + assertEquals(event.tenantId(), TENANT_ID); + assertEquals(event.topic(), "alarms/device1/critical"); + assertEquals(event.mqttTopicFilter(), second.mqttTopicFilter()); + assertEquals(event.maxCount(), 1); + } + + @Test + public void triggerGroupFanoutThrottling() { + NavigableMap kvData = newTreeMap(); + + GroupMatching first = unorderedGroupMatching( + TENANT_ID, + "jobs/+/progress", + "groupA", + Map.of(receiverUrl(1, "receiverA", "delivererA"), 1L)); + GroupMatching second = unorderedGroupMatching( + TENANT_ID, + "jobs/+/progress", + "groupB", + Map.of(receiverUrl(1, "receiverB", "delivererB"), 1L)); + + kvData.put(toGroupRouteKey(TENANT_ID, first.matcher), + RouteGroup.newBuilder().putAllMembers(first.receivers()).build().toByteString()); + kvData.put(toGroupRouteKey(TENANT_ID, second.matcher), + RouteGroup.newBuilder().putAllMembers(second.receivers()).build().toByteString()); + + TenantRouteMatcher matcher = + new TenantRouteMatcher(TENANT_ID, () -> new TreeMapKVReader(kvData), eventCollector, matchTimer); + + Map matchedRoutes = + matcher.matchAll(Set.of("jobs/job1/progress"), 10, 1); + + IMatchedRoutes routes = matchedRoutes.get("jobs/job1/progress"); + assertEquals(routes.groupFanout(), 1); + assertEquals(routes.routes().stream().filter(m -> m.type() == Matching.Type.Group).count(), 1); + + List events = eventCollector.eventsOfType(GroupFanoutThrottled.class); + assertEquals(events.size(), 1); + GroupFanoutThrottled event = events.get(0); + assertEquals(event.tenantId(), TENANT_ID); + assertEquals(event.topic(), "jobs/job1/progress"); + // second comes before first in lexicographical order by bucketing key + assertEquals(event.mqttTopicFilter(), first.mqttTopicFilter()); + assertEquals(event.maxCount(), 1); + } + + private static final class TreeMapKVReader implements IKVReader { + private final NavigableMap data; + private TreeMapKVIterator lastIterator; + + private TreeMapKVReader(NavigableMap data) { + this.data = data; + } + + @Override + public Boundary boundary() { + if (data.isEmpty()) { + return FULL_BOUNDARY; + } + ByteString start = data.firstKey(); + ByteString end = upperBound(data.lastKey()); + return toBoundary(start, end); + } + + @Override + public long size(Boundary boundary) { + if (data.isEmpty()) { + return 0; + } + ByteString start = BoundaryUtil.startKey(boundary); + ByteString end = BoundaryUtil.endKey(boundary); + NavigableMap sub = data; + if (start != null) { + sub = sub.tailMap(start, true); + } + if (end != null) { + sub = sub.headMap(end, false); + } + return sub.size(); + } + + @Override + public boolean exist(ByteString key) { + return data.containsKey(key); + } + + @Override + public java.util.Optional get(ByteString key) { + return java.util.Optional.ofNullable(data.get(key)); + } + + @Override + public IKVIterator iterator() { + lastIterator = new TreeMapKVIterator(data); + return lastIterator; + } + + @Override + public void refresh() { + // no-op for in-memory stub + } + + TreeMapKVIterator lastIterator() { + return lastIterator; + } + } + + private static final class TreeMapKVIterator implements IKVIterator { + private final NavigableMap data; + private Map.Entry current; + private int seekCount; + private int nextCount; + + private TreeMapKVIterator(NavigableMap data) { + this.data = data; + } + + @Override + public ByteString key() { + if (current == null) { + throw new IllegalStateException("Iterator is not valid"); + } + return current.getKey(); + } + + @Override + public ByteString value() { + if (current == null) { + throw new IllegalStateException("Iterator is not valid"); + } + return current.getValue(); + } + + @Override + public boolean isValid() { + return current != null; + } + + @Override + public void next() { + if (current == null) { + throw new IllegalStateException("Iterator is not valid"); + } + current = data.higherEntry(current.getKey()); + nextCount++; + } + + @Override + public void prev() { + if (current == null) { + throw new IllegalStateException("Iterator is not valid"); + } + current = data.lowerEntry(current.getKey()); + nextCount++; + } + + @Override + public void seekToFirst() { + current = data.firstEntry(); + seekCount++; + } + + @Override + public void seekToLast() { + current = data.lastEntry(); + seekCount++; + } + + @Override + public void seek(ByteString key) { + current = key == null ? data.firstEntry() : data.ceilingEntry(key); + seekCount++; + } + + @Override + public void seekForPrev(ByteString key) { + current = key == null ? data.lastEntry() : data.floorEntry(key); + seekCount++; + } + + int getSeekCount() { + return seekCount; + } + + int getNextCount() { + return nextCount; + } + } + + private static final class RecordingEventCollector implements IEventCollector { + private final List> events = new CopyOnWriteArrayList<>(); + + @Override + public void report(Event event) { + events.add((Event) event.clone()); + } + + List> events() { + return new ArrayList<>(events); + } + + > List eventsOfType(Class type) { + List filtered = new ArrayList<>(); + for (Event event : events) { + if (type.isInstance(event)) { + filtered.add(type.cast(event)); + } + } + return filtered; + } + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/schema/Matchings.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/schema/Matchings.java new file mode 100644 index 000000000..cfdf08902 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/schema/Matchings.java @@ -0,0 +1,88 @@ +/* + * 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.bifromq.dist.worker.schema; + +import static org.apache.bifromq.util.TopicConst.DELIMITER; +import static org.apache.bifromq.util.TopicConst.ORDERED_SHARE; +import static org.apache.bifromq.util.TopicConst.UNORDERED_SHARE; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.bifromq.type.RouteMatcher; + +/** + * Test Helper for creating NormalMatching & GroupMatching directly. + */ +public final class Matchings { + private Matchings() { + } + + public static NormalMatching normalMatching(String tenantId, + String mqttTopicFilter, + int subBrokerId, + String receiverId, + String delivererKey, + long incarnation) { + RouteMatcher matcher = RouteMatcher.newBuilder() + .setType(RouteMatcher.Type.Normal) + .addAllFilterLevel(split(mqttTopicFilter)) + .setMqttTopicFilter(mqttTopicFilter) + .build(); + return new NormalMatching(tenantId, + matcher, + KVSchemaUtil.toReceiverUrl(subBrokerId, receiverId, delivererKey), + incarnation); + } + + public static GroupMatching unorderedGroupMatching(String tenantId, + String sharedTopicFilter, + String group, + Map members) { + RouteMatcher matcher = RouteMatcher.newBuilder() + .setType(RouteMatcher.Type.UnorderedShare) + .addAllFilterLevel(split(sharedTopicFilter)) + .setGroup(group) + .setMqttTopicFilter(UNORDERED_SHARE + DELIMITER + group + DELIMITER + sharedTopicFilter) + .build(); + return new GroupMatching(tenantId, matcher, members); + } + + public static GroupMatching orderedGroupMatching(String tenantId, + String sharedTopicFilter, + String group, + Map members) { + RouteMatcher matcher = RouteMatcher.newBuilder() + .setType(RouteMatcher.Type.OrderedShare) + .addAllFilterLevel(split(sharedTopicFilter)) + .setGroup(group) + .setMqttTopicFilter(ORDERED_SHARE + DELIMITER + group + DELIMITER + sharedTopicFilter) + .build(); + return new GroupMatching(tenantId, matcher, members); + } + + public static String receiverUrl(int subBrokerId, String receiverId, String delivererKey) { + return KVSchemaUtil.toReceiverUrl(subBrokerId, receiverId, delivererKey); + } + + private static List split(String topicFilter) { + return Arrays.asList(topicFilter.split("/", -1)); + } +} diff --git a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/ITenantMeter.java b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/ITenantMeter.java index ea90c41d5..345de9a4e 100644 --- a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/ITenantMeter.java +++ b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/ITenantMeter.java @@ -14,13 +14,14 @@ * "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. + * under the License. */ package org.apache.bifromq.metrics; import io.micrometer.core.instrument.Timer; import java.util.function.Supplier; +import java.util.function.ToDoubleFunction; public interface ITenantMeter { String TAG_TENANT_ID = "tenantId"; @@ -37,6 +38,18 @@ static void stopGauging(String tenantId, TenantMetric gaugeMetric, String... tag TenantGauges.stopGauging(tenantId, gaugeMetric, tagValuePair); } + static void counting(String tenantId, + TenantMetric counterMetric, + C ctr, + ToDoubleFunction supplier, + String... tagValuePair) { + TenantFunctionCounters.counting(tenantId, counterMetric, ctr, supplier, tagValuePair); + } + + static void stopCounting(String tenantId, TenantMetric counterMetric, String... tagValuePair) { + TenantFunctionCounters.stopCounting(tenantId, counterMetric, tagValuePair); + } + void recordCount(TenantMetric metric); void recordCount(TenantMetric metric, double inc); diff --git a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantFunctionCounters.java b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantFunctionCounters.java new file mode 100644 index 000000000..591000380 --- /dev/null +++ b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantFunctionCounters.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.bifromq.metrics; + +import static org.apache.bifromq.metrics.ITenantMeter.TAG_TENANT_ID; + +import io.micrometer.core.instrument.FunctionCounter; +import io.micrometer.core.instrument.Meter; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Tags; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.ToDoubleFunction; + +class TenantFunctionCounters { + private static final ConcurrentMap> TENANT_FUNCTION_COUNTERS = new ConcurrentHashMap<>(); + + static void counting(String tenantId, TenantMetric counterMetric, C ctr, ToDoubleFunction supplier, + String... tagValuePair) { + assert counterMetric.meterType == Meter.Type.COUNTER && counterMetric.isFunction; + TENANT_FUNCTION_COUNTERS.compute(tenantId, (k, v) -> { + if (v == null) { + v = new HashMap<>(); + } + Tags tags = Tags.of(tagValuePair); + v.computeIfAbsent(new TenantMetricKey(counterMetric, tags), + tenantMetricKey -> FunctionCounter.builder(counterMetric.metricName, ctr, supplier) + .tags(tags.and(TAG_TENANT_ID, tenantId)) + .register(Metrics.globalRegistry)); + return v; + }); + } + + static void stopCounting(String tenantId, TenantMetric counterMetric, String... tagValuePair) { + assert counterMetric.meterType == Meter.Type.COUNTER && counterMetric.isFunction; + TENANT_FUNCTION_COUNTERS.computeIfPresent(tenantId, (k, ctrMap) -> { + Tags tags = Tags.of(tagValuePair); + FunctionCounter functionCounter = ctrMap.remove(new TenantMetricKey(counterMetric, tags)); + if (functionCounter != null) { + Metrics.globalRegistry.remove(functionCounter); + } + if (ctrMap.isEmpty()) { + return null; + } + return ctrMap; + }); + } + + private record TenantMetricKey(TenantMetric counterMetrics, Tags tags) { + } +} diff --git a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMeter.java b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMeter.java index aec230201..fde19d77f 100644 --- a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMeter.java +++ b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMeter.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.metrics; @@ -33,38 +33,6 @@ @Slf4j class TenantMeter implements ITenantMeter { private static final Cleaner CLEANER = Cleaner.create(); - - private static class State implements Runnable { - final Map meters = new HashMap<>(); - - State(Tags tags) { - for (TenantMetric metric : TenantMetric.values()) { - switch (metric.meterType) { - case COUNTER: - meters.put(metric, Metrics.counter(metric.metricName, tags)); - break; - case TIMER: - meters.put(metric, Metrics.timer(metric.metricName, tags)); - break; - case DISTRIBUTION_SUMMARY: - meters.put(metric, Metrics.summary(metric.metricName, tags)); - break; - case GAUGE: - // ignore gauge - break; - default: - throw new UnsupportedOperationException("Unsupported traffic meter type"); - } - } - } - - @Override - public void run() { - meters.values().forEach(Metrics.globalRegistry::remove); - meters.clear(); - } - } - private final State state; private final Tags tags; private final Cleaner.Cleanable cleanable; @@ -98,8 +66,41 @@ public void recordSummary(TenantMetric metric, double value) { ((DistributionSummary) state.meters.get(metric)).record(value); } - public void destroy() { cleanable.clean(); } + + private static class State implements Runnable { + final Map meters = new HashMap<>(); + + State(Tags tags) { + for (TenantMetric metric : TenantMetric.values()) { + switch (metric.meterType) { + case COUNTER: { + if (!metric.isFunction) { + meters.put(metric, Metrics.counter(metric.metricName, tags)); + } + } + break; + case TIMER: + meters.put(metric, Metrics.timer(metric.metricName, tags)); + break; + case DISTRIBUTION_SUMMARY: + meters.put(metric, Metrics.summary(metric.metricName, tags)); + break; + case GAUGE: + // ignore gauge + break; + default: + throw new UnsupportedOperationException("Unsupported traffic meter type"); + } + } + } + + @Override + public void run() { + meters.values().forEach(Metrics.globalRegistry::remove); + meters.clear(); + } + } } diff --git a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMetric.java b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMetric.java index f80dab53e..8f516e1c5 100644 --- a/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMetric.java +++ b/bifromq-metrics/src/main/java/org/apache/bifromq/metrics/TenantMetric.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.metrics; @@ -72,7 +72,9 @@ public enum TenantMetric { MqttPersistentSubCountGauge("mqtt.psub.num.gauge", Meter.Type.GAUGE), MqttRouteCacheSize("mqtt.route.cache.size.gauge", Meter.Type.GAUGE), - MqttRouteCacheMissCount("mqtt.route.cache.miss.count", Meter.Type.COUNTER), + MqttRouteCacheHitCount("mqtt.route.cache.hit.count", Meter.Type.COUNTER, true), + MqttRouteCacheEvictCount("mqtt.route.cache.evict.count", Meter.Type.COUNTER, true), + MqttRouteCacheMissCount("mqtt.route.cache.miss.count", Meter.Type.COUNTER, true), // retain related MqttIngressRetainBytes("mqtt.ingress.retain.bytes", Meter.Type.DISTRIBUTION_SUMMARY), MqttRetainedBytes("mqtt.retained.bytes", Meter.Type.DISTRIBUTION_SUMMARY), @@ -83,9 +85,16 @@ public enum TenantMetric { public final String metricName; public final Meter.Type meterType; + public final boolean isFunction; TenantMetric(String metricName, Meter.Type meterType) { + this(metricName, meterType, false); + } + + TenantMetric(String metricName, Meter.Type meterType, boolean isFunction) { + assert !isFunction || meterType == Meter.Type.COUNTER; this.metricName = metricName; this.meterType = meterType; + this.isFunction = isFunction; } } diff --git a/bifromq-mqtt/bifromq-mqtt-server/src/test/java/org/apache/bifromq/mqtt/integration/v3/MQTTLastWillTest.java b/bifromq-mqtt/bifromq-mqtt-server/src/test/java/org/apache/bifromq/mqtt/integration/v3/MQTTLastWillTest.java index 8b8c6f455..ab9955640 100644 --- a/bifromq-mqtt/bifromq-mqtt-server/src/test/java/org/apache/bifromq/mqtt/integration/v3/MQTTLastWillTest.java +++ b/bifromq-mqtt/bifromq-mqtt-server/src/test/java/org/apache/bifromq/mqtt/integration/v3/MQTTLastWillTest.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.mqtt.integration.v3; @@ -27,6 +27,11 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; +import com.google.protobuf.ByteString; +import io.reactivex.rxjava3.core.Observable; +import io.reactivex.rxjava3.observers.TestObserver; +import java.util.concurrent.CompletableFuture; +import lombok.extern.slf4j.Slf4j; import org.apache.bifromq.mqtt.integration.MQTTTest; import org.apache.bifromq.mqtt.integration.v3.client.MqttMsg; import org.apache.bifromq.mqtt.integration.v3.client.MqttTestClient; @@ -36,11 +41,6 @@ import org.apache.bifromq.plugin.authprovider.type.MQTT3AuthResult; import org.apache.bifromq.plugin.authprovider.type.Ok; import org.apache.bifromq.plugin.eventcollector.Event; -import com.google.protobuf.ByteString; -import io.reactivex.rxjava3.core.Observable; -import io.reactivex.rxjava3.observers.TestObserver; -import java.util.concurrent.CompletableFuture; -import lombok.extern.slf4j.Slf4j; import org.eclipse.paho.client.mqttv3.MqttConnectOptions; import org.testng.annotations.Test; @@ -122,6 +122,7 @@ public void lastWillQoS1Retained() { doAnswer(invocationOnMock -> null).when(eventCollector).report(any(Event.class)); + // Publisher with retained will MqttConnectOptions lwtPubConnOpts = new MqttConnectOptions(); lwtPubConnOpts.setCleanSession(true); lwtPubConnOpts.setWill(willTopic, willPayload.toByteArray(), 1, true); @@ -129,17 +130,33 @@ public void lastWillQoS1Retained() { MqttTestClient lwtPubClient = new MqttTestClient(BROKER_URI, "lwtPubclient"); lwtPubClient.connect(lwtPubConnOpts); + // Warm route: pre-subscribe and ensure path is active + MqttConnectOptions warmSubConnOpts = new MqttConnectOptions(); + warmSubConnOpts.setCleanSession(true); + warmSubConnOpts.setUserName(userName); + MqttTestClient warmSubClient = new MqttTestClient(BROKER_URI, "lwtWarmSubClient"); + warmSubClient.connect(warmSubConnOpts); + TestObserver warmSub = warmSubClient.subscribe(willTopic, 1).test(); + await().until(() -> { + lwtPubClient.publish(willTopic, 1, ByteString.copyFromUtf8("test"), false); + return !warmSub.values().isEmpty(); + }); + + // Trigger will publication and retained storage kill(deviceKey, "lwtPubclient").join(); + // Fresh subscriber to receive retained snapshot (retain flag true) MqttConnectOptions lwtSubConnOpts = new MqttConnectOptions(); lwtSubConnOpts.setCleanSession(true); lwtSubConnOpts.setUserName(userName); - MqttTestClient lwtSubClient = new MqttTestClient(BROKER_URI, "lwtSubClient"); lwtSubClient.connect(lwtSubConnOpts); Observable topicSub = lwtSubClient.subscribe(willTopic, 1); - MqttMsg msg = topicSub.blockingFirst(); + MqttMsg msg = topicSub + .firstElement() + .timeout(30, java.util.concurrent.TimeUnit.SECONDS) + .blockingGet(); assertEquals(msg.topic, willTopic); assertEquals(msg.qos, 1); assertEquals(msg.payload, willPayload); @@ -148,6 +165,7 @@ public void lastWillQoS1Retained() { // clear the retained will message lwtSubClient.publish(willTopic, 1, ByteString.EMPTY, true); lwtSubClient.disconnect(); + warmSubClient.disconnect(); } @Test(groups = "integration", dependsOnMethods = "lastWillQoS1Retained") @@ -223,23 +241,41 @@ public void lastWillQoS2Retained() { .setGranted(Granted.getDefaultInstance()) .build())); + // Publisher with retained will MqttConnectOptions lwtPubConnOpts = new MqttConnectOptions(); lwtPubConnOpts.setCleanSession(true); lwtPubConnOpts.setWill(willTopic, willPayload.toByteArray(), 2, true); lwtPubConnOpts.setUserName(userName); MqttTestClient lwtPubClient = new MqttTestClient(BROKER_URI, "lwtPubclient"); lwtPubClient.connect(lwtPubConnOpts); + + // Warm route: pre-subscribe and ensure path is active + MqttConnectOptions warmSubConnOpts = new MqttConnectOptions(); + warmSubConnOpts.setCleanSession(true); + warmSubConnOpts.setUserName(userName); + MqttTestClient warmSubClient = new MqttTestClient(BROKER_URI, "lwtWarmSubClientQoS2"); + warmSubClient.connect(warmSubConnOpts); + io.reactivex.rxjava3.observers.TestObserver warmSub = warmSubClient.subscribe(willTopic, 2).test(); + await().until(() -> { + lwtPubClient.publish(willTopic, 2, ByteString.copyFromUtf8("test"), false); + return !warmSub.values().isEmpty(); + }); + + // Trigger will publication and retained storage kill(deviceKey, "lwtPubclient").join(); + // Fresh subscriber to receive retained snapshot (retain flag true) MqttConnectOptions lwtSubConnOpts = new MqttConnectOptions(); lwtSubConnOpts.setCleanSession(true); lwtSubConnOpts.setUserName(userName); - MqttTestClient lwtSubClient = new MqttTestClient(BROKER_URI, "lwtSubClient"); lwtSubClient.connect(lwtSubConnOpts); Observable topicSub = lwtSubClient.subscribe(willTopic, 2); - MqttMsg msg = topicSub.blockingFirst(); + MqttMsg msg = topicSub + .firstElement() + .timeout(30, java.util.concurrent.TimeUnit.SECONDS) + .blockingGet(); assertEquals(msg.topic, willTopic); assertEquals(msg.qos, 2); assertEquals(msg.payload, willPayload); @@ -248,5 +284,6 @@ public void lastWillQoS2Retained() { // clear the retained will message lwtSubClient.publish(willTopic, 2, ByteString.EMPTY, true); lwtSubClient.disconnect(); + warmSubClient.disconnect(); } } diff --git a/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/GroupFanoutThrottled.java b/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/GroupFanoutThrottled.java index f4b2b8e19..ad5342381 100644 --- a/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/GroupFanoutThrottled.java +++ b/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/GroupFanoutThrottled.java @@ -33,6 +33,7 @@ public final class GroupFanoutThrottled extends Event { private String tenantId; private String topic; + private String mqttTopicFilter; private int maxCount; @Override diff --git a/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/PersistentFanoutThrottled.java b/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/PersistentFanoutThrottled.java index bf5b95ce6..8626d7db0 100644 --- a/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/PersistentFanoutThrottled.java +++ b/bifromq-plugin/bifromq-plugin-event-collector/src/main/java/org/apache/bifromq/plugin/eventcollector/distservice/PersistentFanoutThrottled.java @@ -33,6 +33,7 @@ public final class PersistentFanoutThrottled extends Event { private String tenantId; private String topic; + private String mqttTopicFilter; private int maxCount; @Override diff --git a/bifromq-util/src/main/java/org/apache/bifromq/util/index/TopicLevelTrie.java b/bifromq-util/src/main/java/org/apache/bifromq/util/index/TopicLevelTrie.java index bab422292..a7e31e243 100644 --- a/bifromq-util/src/main/java/org/apache/bifromq/util/index/TopicLevelTrie.java +++ b/bifromq-util/src/main/java/org/apache/bifromq/util/index/TopicLevelTrie.java @@ -19,9 +19,7 @@ package org.apache.bifromq.util.index; -import static java.util.Collections.emptySet; - -import com.google.common.collect.Sets; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -200,21 +198,22 @@ private LookupResult lookup(INode i, Map, BranchSelector.Action> branches = branchSelector.selectBranch(cn.branches, topicLevels, currentLevel); - Set values = emptySet(); + // Flatten union: accumulate into a mutable set to avoid deep SetView chains + Set values = new HashSet<>(); for (Map.Entry, BranchSelector.Action> entry : branches.entrySet()) { Branch branch = entry.getKey(); BranchSelector.Action action = entry.getValue(); switch (action) { case MATCH_AND_CONTINUE, CONTINUE -> { if (action == BranchSelector.Action.MATCH_AND_CONTINUE) { - values = Sets.union(values, branch.values()); + values.addAll(branch.values()); } // Continue if (branch.iNode != null) { LookupResult result = lookup(branch.iNode, i, topicLevels, currentLevel + 1, branchSelector); if (result.successOrRetry) { - values = Sets.union(values, result.values); + values.addAll(result.values); } else { return result; } @@ -222,7 +221,7 @@ private LookupResult lookup(INode i, } case MATCH_AND_STOP, STOP -> { if (action == BranchSelector.Action.MATCH_AND_STOP) { - values = Sets.union(values, branch.values()); + values.addAll(branch.values()); } } default -> throw new IllegalStateException("Unknown action: " + action); @@ -305,4 +304,4 @@ private boolean couldTrim(Branch br) { MainNode main = br.iNode.main(); return main.tNode != null; } -} \ No newline at end of file +} From 1afae4e221aa93e6a8913f358a228068baadf8cf Mon Sep 17 00:00:00 2001 From: Yonny Hao Date: Thu, 18 Sep 2025 16:44:04 +0800 Subject: [PATCH 2/2] Reduce heap usage during matching and routing --- .../bifromq-dist-coproc-proto/pom.xml | 11 +- .../dist/trie/ITopicFilterIterator.java | 16 +- .../dist/trie/MTopicFilterTrieNode.java | 104 +++++++-- .../dist/trie/NTopicFilterTrieNode.java | 115 ++++++++-- .../dist/trie/STopicFilterTrieNode.java | 111 ++++++++-- .../trie/ThreadLocalTopicFilterIterator.java | 33 +++ .../dist/trie/TopicFilterIterator.java | 55 +++-- .../dist/trie/TopicFilterTrieNode.java | 20 +- .../dist/trie/TopicFilterIteratorTest.java | 47 ++-- .../dist/trie/TopicFilterPoolTest.java | 204 ++++++++++++++++++ .../server/scheduler/BatchDistServerCall.java | 58 ++--- .../dist/worker/schema/GroupMatching.java | 2 +- .../dist/worker/schema/KVSchemaConstants.java | 34 +++ .../dist/worker/schema/KVSchemaUtil.java | 110 ++-------- .../dist/worker/schema/NormalMatching.java | 4 +- .../bifromq/dist/worker/schema/Receiver.java | 24 +++ .../schema/cache/GroupMatchingCache.java | 47 ++++ .../schema/cache/NormalMatchingCache.java | 46 ++++ .../worker/schema/cache/ReceiverCache.java | 41 ++++ .../worker/schema/cache/RouteDetailCache.java | 116 ++++++++++ .../worker/schema/cache/RouteGroupCache.java | 44 ++++ .../schema/cache/GroupMatchingCacheTest.java | 166 ++++++++++++++ .../schema/cache/NormalMatchingCacheTest.java | 139 ++++++++++++ .../schema/cache/ReceiverCacheTest.java | 90 ++++++++ .../schema/cache/RouteDetailCacheTest.java | 174 +++++++++++++++ .../schema/cache/RouteGroupCacheTest.java | 121 +++++++++++ .../dist/worker/cache/TenantRouteMatcher.java | 118 +++++----- .../bifromq/dist/worker/KeyLayoutTest.java | 15 +- 28 files changed, 1790 insertions(+), 275 deletions(-) create mode 100644 bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ThreadLocalTopicFilterIterator.java create mode 100644 bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterPoolTest.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaConstants.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Receiver.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCache.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCache.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCache.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCache.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCache.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCacheTest.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCacheTest.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCacheTest.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCacheTest.java create mode 100644 bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCacheTest.java diff --git a/bifromq-dist/bifromq-dist-coproc-proto/pom.xml b/bifromq-dist/bifromq-dist-coproc-proto/pom.xml index 6c0ecc7b5..ec34f3778 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/pom.xml +++ b/bifromq-dist/bifromq-dist-coproc-proto/pom.xml @@ -34,6 +34,11 @@ org.apache.bifromq bifromq-common-type + + com.github.ben-manes.caffeine + caffeine + ${caffeine.version} + org.apache.bifromq bifromq-dist-worker-schema @@ -47,6 +52,10 @@ org.openjdk.jmh jmh-generator-annprocess + + org.awaitility + awaitility + org.apache.logging.log4j log4j-api @@ -71,4 +80,4 @@ - \ No newline at end of file + diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ITopicFilterIterator.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ITopicFilterIterator.java index a79402bc9..1286a83af 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ITopicFilterIterator.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ITopicFilterIterator.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.trie; @@ -29,7 +29,19 @@ * * @param the value type for topic associated value */ -public interface ITopicFilterIterator { +public interface ITopicFilterIterator extends AutoCloseable { + /** + * Init the iterator with the given root node. + * + * @param root the root node of the topic trie + */ + void init(TopicTrieNode root); + + /** + * Reset the iterator after using. + */ + void close(); + /** * Seek to the given topic filter levels, so that the next topic filter is greater or equals to the given topic * filter levels. diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/MTopicFilterTrieNode.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/MTopicFilterTrieNode.java index 104c4eac1..ce97b6f4d 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/MTopicFilterTrieNode.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/MTopicFilterTrieNode.java @@ -14,18 +14,23 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.trie; import static org.apache.bifromq.util.TopicConst.MULTI_WILDCARD; -import static java.util.Collections.emptySet; -import static java.util.Collections.singleton; -import com.google.common.collect.Sets; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalCause; +import com.github.benmanes.caffeine.cache.Scheduler; +import com.github.benmanes.caffeine.cache.Ticker; +import java.util.HashSet; import java.util.NoSuchElementException; import java.util.Set; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.atomic.AtomicLong; /** * Multi-level topic filter trie node. @@ -33,15 +38,81 @@ * @param value type */ final class MTopicFilterTrieNode extends TopicFilterTrieNode { - private final Set> backingTopics; + private static final ConcurrentLinkedDeque KEYS = new ConcurrentLinkedDeque<>(); + private static final AtomicLong SEQ = new AtomicLong(); + private static volatile Ticker TICKER = Ticker.systemTicker(); + private static final Cache> POOL = Caffeine.newBuilder() + .expireAfterAccess(EXPIRE_AFTER) + .recordStats() + .scheduler(Scheduler.systemScheduler()) + .ticker(() -> TICKER.read()) + .removalListener((Long key, MTopicFilterTrieNode value, RemovalCause cause) -> { + KEYS.remove(key); + if (cause == RemovalCause.EXPIRED || cause == RemovalCause.SIZE) { + value.recycle(); + } + }) + .build(); + + private final Set> backingTopics = new HashSet<>(); + + MTopicFilterTrieNode() { + } + + static MTopicFilterTrieNode borrow(TopicFilterTrieNode parent, + Set> siblingTopicTrieNodes) { + while (true) { + Long key = KEYS.pollFirst(); + if (key == null) { + break; + } + @SuppressWarnings("unchecked") + MTopicFilterTrieNode pooled = (MTopicFilterTrieNode) POOL.asMap().remove(key); + if (pooled != null) { + return pooled.init(parent, siblingTopicTrieNodes); + } + } + MTopicFilterTrieNode node = new MTopicFilterTrieNode<>(); + return node.init(parent, siblingTopicTrieNodes); + } + + static void release(MTopicFilterTrieNode node) { + node.recycle(); + long key = SEQ.incrementAndGet(); + KEYS.offerLast(key); + POOL.put(key, node); + } + + // test hooks (package-private) + static void poolClear() { + POOL.invalidateAll(); + POOL.cleanUp(); + KEYS.clear(); + } + + static void poolCleanUp() { + POOL.cleanUp(); + } + + static int poolApproxSize() { + return KEYS.size(); + } + + static void setTicker(Ticker ticker) { + TICKER = ticker != null ? ticker : Ticker.systemTicker(); + } - MTopicFilterTrieNode(TopicFilterTrieNode parent, Set> siblingTopicTrieNodes) { - super(parent); - Set> topics = parent != null ? parent.backingTopics() : emptySet(); + MTopicFilterTrieNode init(TopicFilterTrieNode parent, Set> siblingTopicTrieNodes) { + assert siblingTopicTrieNodes != null; + this.parent = parent; + backingTopics.clear(); + if (parent != null) { + backingTopics.addAll(parent.backingTopics()); + } for (TopicTrieNode sibling : siblingTopicTrieNodes) { - topics = collectTopics(sibling, topics); + collectTopics(sibling); } - backingTopics = topics; + return this; } @Override @@ -54,17 +125,15 @@ Set> backingTopics() { return backingTopics; } - private Set> collectTopics(TopicTrieNode node, Set> topics) { + private void collectTopics(TopicTrieNode node) { if (node.isUserTopic()) { - topics = Sets.union(topics, singleton(node)); + backingTopics.add(node); } for (TopicTrieNode child : node.children().values()) { - topics = collectTopics(child, topics); + collectTopics(child); } - return topics; } - @Override void seekChild(String childLevelName) { @@ -104,4 +173,9 @@ void prevChild() { TopicFilterTrieNode childNode() { throw new NoSuchElementException(); } + + private void recycle() { + parent = null; + backingTopics.clear(); + } } diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/NTopicFilterTrieNode.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/NTopicFilterTrieNode.java index 9149e6238..72b40ec88 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/NTopicFilterTrieNode.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/NTopicFilterTrieNode.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.trie; @@ -22,6 +22,11 @@ import static org.apache.bifromq.util.TopicConst.MULTI_WILDCARD; import static org.apache.bifromq.util.TopicConst.SINGLE_WILDCARD; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalCause; +import com.github.benmanes.caffeine.cache.Scheduler; +import com.github.benmanes.caffeine.cache.Ticker; import java.util.HashSet; import java.util.Map; import java.util.NavigableMap; @@ -30,6 +35,8 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.atomic.AtomicLong; /** * Normal level topic filter trie node. @@ -37,24 +44,89 @@ * @param value type */ final class NTopicFilterTrieNode extends TopicFilterTrieNode { - private final String levelName; - private final NavigableSet subLevelNames; - private final NavigableMap>> subTopicTrieNodes; - private final Set> subWildcardMatchableTopicTrieNodes; - private final Set> backingTopics; - + private static final ConcurrentLinkedDeque KEYS = new ConcurrentLinkedDeque<>(); + private static final AtomicLong SEQ = new AtomicLong(); + private static volatile Ticker TICKER = Ticker.systemTicker(); + private static final Cache> POOL = Caffeine.newBuilder() + .expireAfterAccess(EXPIRE_AFTER) + .recordStats() + .scheduler(Scheduler.systemScheduler()) + .ticker(() -> TICKER.read()) + .removalListener((Long key, NTopicFilterTrieNode value, RemovalCause cause) -> { + KEYS.remove(key); + if (cause == RemovalCause.EXPIRED || cause == RemovalCause.SIZE) { + value.recycle(); + } + }) + .build(); + + private final NavigableSet subLevelNames = new TreeSet<>(); + private final NavigableMap>> subTopicTrieNodes = new TreeMap<>(); + private final Set> subWildcardMatchableTopicTrieNodes = new HashSet<>(); + private final Set> backingTopics = new HashSet<>(); + private String levelName; // point to the sub node during iteration private String subLevelName; - NTopicFilterTrieNode(TopicFilterTrieNode parent, String levelName, Set> siblingTopicTrieNodes) { - super(parent); + NTopicFilterTrieNode() { + } + + static NTopicFilterTrieNode borrow(TopicFilterTrieNode parent, + String levelName, + Set> siblingTopicTrieNodes) { + while (true) { + Long key = KEYS.pollFirst(); + if (key == null) { + break; + } + @SuppressWarnings("unchecked") + NTopicFilterTrieNode pooled = (NTopicFilterTrieNode) POOL.asMap().remove(key); + if (pooled != null) { + return pooled.init(parent, levelName, siblingTopicTrieNodes); + } + } + NTopicFilterTrieNode node = new NTopicFilterTrieNode<>(); + return node.init(parent, levelName, siblingTopicTrieNodes); + } + + static void release(NTopicFilterTrieNode node) { + node.recycle(); + long key = SEQ.incrementAndGet(); + KEYS.offerLast(key); + POOL.put(key, node); + } + + // test hooks (package-private) + static void poolClear() { + POOL.invalidateAll(); + POOL.cleanUp(); + KEYS.clear(); + } + + static void poolCleanUp() { + POOL.cleanUp(); + } + + static int poolApproxSize() { + return KEYS.size(); + } + + static void setTicker(Ticker ticker) { + TICKER = ticker != null ? ticker : Ticker.systemTicker(); + } + + NTopicFilterTrieNode init(TopicFilterTrieNode parent, String levelName, + Set> siblingTopicTrieNodes) { assert levelName != null; + assert siblingTopicTrieNodes != null; assert siblingTopicTrieNodes.stream().allMatch(node -> node.levelName().equals(levelName)); - this.subTopicTrieNodes = new TreeMap<>(); - this.subLevelNames = new TreeSet<>(); - this.subWildcardMatchableTopicTrieNodes = new HashSet<>(); + this.parent = parent; this.levelName = levelName; - this.backingTopics = new HashSet<>(); + subLevelName = null; + subLevelNames.clear(); + subTopicTrieNodes.clear(); + subWildcardMatchableTopicTrieNodes.clear(); + backingTopics.clear(); for (TopicTrieNode sibling : siblingTopicTrieNodes) { if (sibling.isUserTopic()) { backingTopics.add(sibling); @@ -77,6 +149,7 @@ final class NTopicFilterTrieNode extends TopicFilterTrieNode { subLevelNames.add(SINGLE_WILDCARD); } seekChild(""); + return this; } @Override @@ -142,9 +215,19 @@ TopicFilterTrieNode childNode() { throw new NoSuchElementException(); } return switch (subLevelName) { - case MULTI_WILDCARD -> new MTopicFilterTrieNode<>(this, subWildcardMatchableTopicTrieNodes); - case SINGLE_WILDCARD -> new STopicFilterTrieNode<>(this, subWildcardMatchableTopicTrieNodes); - default -> new NTopicFilterTrieNode<>(this, subLevelName, subTopicTrieNodes.get(subLevelName)); + case MULTI_WILDCARD -> MTopicFilterTrieNode.borrow(this, subWildcardMatchableTopicTrieNodes); + case SINGLE_WILDCARD -> STopicFilterTrieNode.borrow(this, subWildcardMatchableTopicTrieNodes); + default -> NTopicFilterTrieNode.borrow(this, subLevelName, subTopicTrieNodes.get(subLevelName)); }; } + + private void recycle() { + parent = null; + levelName = null; + subLevelName = null; + subLevelNames.clear(); + subTopicTrieNodes.clear(); + subWildcardMatchableTopicTrieNodes.clear(); + backingTopics.clear(); + } } diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/STopicFilterTrieNode.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/STopicFilterTrieNode.java index 0ac8285dd..a6da05494 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/STopicFilterTrieNode.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/STopicFilterTrieNode.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.trie; @@ -22,6 +22,11 @@ import static org.apache.bifromq.util.TopicConst.MULTI_WILDCARD; import static org.apache.bifromq.util.TopicConst.SINGLE_WILDCARD; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalCause; +import com.github.benmanes.caffeine.cache.Scheduler; +import com.github.benmanes.caffeine.cache.Ticker; import java.util.HashSet; import java.util.Map; import java.util.NavigableMap; @@ -30,6 +35,8 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.atomic.AtomicLong; /** * Single level topic filter trie node. @@ -37,20 +44,84 @@ * @param value type */ final class STopicFilterTrieNode extends TopicFilterTrieNode { - private final NavigableSet subLevelNames; - private final NavigableMap>> subTopicTrieNodes; - private final Set> subWildcardMatchableTopicTrieNodes; - private final Set> backingTopics; + private static final ConcurrentLinkedDeque KEYS = new ConcurrentLinkedDeque<>(); + private static final AtomicLong SEQ = new AtomicLong(); + private static volatile Ticker TICKER = Ticker.systemTicker(); + private static final Cache> POOL = Caffeine.newBuilder() + .expireAfterAccess(EXPIRE_AFTER) + .recordStats() + .scheduler(Scheduler.systemScheduler()) + .ticker(() -> TICKER.read()) + .removalListener((Long key, STopicFilterTrieNode value, RemovalCause cause) -> { + KEYS.remove(key); + if (cause == RemovalCause.EXPIRED || cause == RemovalCause.SIZE) { + value.recycle(); + } + }) + .build(); + + private final NavigableSet subLevelNames = new TreeSet<>(); + private final NavigableMap>> subTopicTrieNodes = new TreeMap<>(); + private final Set> subWildcardMatchableTopicTrieNodes = new HashSet<>(); + private final Set> backingTopics = new HashSet<>(); // point to the sub node during iteration private String subLevelName; - STopicFilterTrieNode(TopicFilterTrieNode parent, Set> siblingTopicTrieNodes) { - super(parent); - this.subLevelNames = new TreeSet<>(); - this.subTopicTrieNodes = new TreeMap<>(); - this.subWildcardMatchableTopicTrieNodes = new HashSet<>(); - this.backingTopics = new HashSet<>(); + STopicFilterTrieNode() { + } + + static STopicFilterTrieNode borrow(TopicFilterTrieNode parent, + Set> siblingTopicTrieNodes) { + while (true) { + Long key = KEYS.pollFirst(); + if (key == null) { + break; + } + @SuppressWarnings("unchecked") + STopicFilterTrieNode pooled = (STopicFilterTrieNode) POOL.asMap().remove(key); + if (pooled != null) { + return pooled.init(parent, siblingTopicTrieNodes); + } + } + STopicFilterTrieNode node = new STopicFilterTrieNode<>(); + return node.init(parent, siblingTopicTrieNodes); + } + + static void release(STopicFilterTrieNode node) { + node.recycle(); + long key = SEQ.incrementAndGet(); + KEYS.offerLast(key); + POOL.put(key, node); + } + + // test hooks (package-private) + static void poolClear() { + POOL.invalidateAll(); + POOL.cleanUp(); + KEYS.clear(); + } + + static void poolCleanUp() { + POOL.cleanUp(); + } + + static int poolApproxSize() { + return KEYS.size(); + } + + static void setTicker(Ticker ticker) { + TICKER = ticker != null ? ticker : Ticker.systemTicker(); + } + + STopicFilterTrieNode init(TopicFilterTrieNode parent, Set> siblingTopicTrieNodes) { + assert siblingTopicTrieNodes != null; + this.parent = parent; + subLevelName = null; + subLevelNames.clear(); + subTopicTrieNodes.clear(); + subWildcardMatchableTopicTrieNodes.clear(); + backingTopics.clear(); for (TopicTrieNode sibling : siblingTopicTrieNodes) { if (sibling.isUserTopic()) { backingTopics.add(sibling); @@ -63,7 +134,6 @@ final class STopicFilterTrieNode extends TopicFilterTrieNode { subTopicTrieNodes.computeIfAbsent(subNode.levelName(), k -> new HashSet<>()).add(subNode); subLevelNames.add(subNode.levelName()); } - } // # match parent if (!backingTopics.isEmpty()) { @@ -73,8 +143,8 @@ final class STopicFilterTrieNode extends TopicFilterTrieNode { subLevelNames.add(MULTI_WILDCARD); subLevelNames.add(SINGLE_WILDCARD); } - // point to first child after init seekChild(""); + return this; } @Override @@ -140,9 +210,18 @@ TopicFilterTrieNode childNode() { throw new NoSuchElementException(); } return switch (subLevelName) { - case MULTI_WILDCARD -> new MTopicFilterTrieNode<>(this, subWildcardMatchableTopicTrieNodes); - case SINGLE_WILDCARD -> new STopicFilterTrieNode<>(this, subWildcardMatchableTopicTrieNodes); - default -> new NTopicFilterTrieNode<>(this, subLevelName, subTopicTrieNodes.get(subLevelName)); + case MULTI_WILDCARD -> MTopicFilterTrieNode.borrow(this, subWildcardMatchableTopicTrieNodes); + case SINGLE_WILDCARD -> STopicFilterTrieNode.borrow(this, subWildcardMatchableTopicTrieNodes); + default -> NTopicFilterTrieNode.borrow(this, subLevelName, subTopicTrieNodes.get(subLevelName)); }; } + + private void recycle() { + parent = null; + subLevelName = null; + subLevelNames.clear(); + subTopicTrieNodes.clear(); + subWildcardMatchableTopicTrieNodes.clear(); + backingTopics.clear(); + } } diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ThreadLocalTopicFilterIterator.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ThreadLocalTopicFilterIterator.java new file mode 100644 index 000000000..f02c6ee65 --- /dev/null +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/ThreadLocalTopicFilterIterator.java @@ -0,0 +1,33 @@ +/* + * 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.bifromq.dist.trie; + +import static java.lang.ThreadLocal.withInitial; + +public class ThreadLocalTopicFilterIterator { + private static final ThreadLocal> INSTANCE = withInitial(TopicFilterIterator::new); + + public static ITopicFilterIterator get(TopicTrieNode root) { + @SuppressWarnings("unchecked") + ITopicFilterIterator itr = ((ITopicFilterIterator) INSTANCE.get()); + itr.init(root); + return itr; + } +} diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterIterator.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterIterator.java index 4e9fac883..1f42aa419 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterIterator.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterIterator.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.trie; @@ -36,21 +36,32 @@ * @param the value type for topic associated value */ public class TopicFilterIterator implements ITopicFilterIterator { - private final TopicTrieNode topicTrieRoot; // the invariant of the stack: // empty: no valid topic filter to iterator from expansion set // non-empty: always point to a valid topic filter in expansion set when there is no operation private final Stack> traverseStack = new Stack<>(); - public TopicFilterIterator(TopicTrieNode root) { + private TopicTrieNode topicTrieRoot; + + public TopicFilterIterator() { + } + + @Override + public void init(TopicTrieNode root) { this.topicTrieRoot = root; seek(Collections.emptyList()); } + @Override + public void close() { + clearTraverseStack(); + topicTrieRoot = null; + } + @Override public void seek(List filterLevels) { - traverseStack.clear(); - traverseStack.add(TopicFilterTrieNode.from(topicTrieRoot)); + clearTraverseStack(); + traverseStack.push(TopicFilterTrieNode.from(topicTrieRoot)); int i = -1; out: while (!traverseStack.isEmpty() && i < filterLevels.size()) { @@ -73,7 +84,7 @@ public void seek(List filterLevels) { traverseStack.push(node.childNode()); } else { // backtrace - traverseStack.pop(); + popAndRelease(); if (traverseStack.isEmpty()) { break; } @@ -85,7 +96,7 @@ public void seek(List filterLevels) { traverseStack.push(parent.childNode()); break out; } else { - traverseStack.pop(); + popAndRelease(); } } } @@ -94,7 +105,7 @@ public void seek(List filterLevels) { // no least next topicfilter exists in expansion set for the given topic filter // drain the stack while (!traverseStack.isEmpty()) { - traverseStack.pop(); + popAndRelease(); } } } @@ -112,8 +123,8 @@ public void seek(List filterLevels) { @Override public void seekPrev(List filterLevels) { - traverseStack.clear(); - traverseStack.add(TopicFilterTrieNode.from(topicTrieRoot)); + clearTraverseStack(); + traverseStack.push(TopicFilterTrieNode.from(topicTrieRoot)); int i = -1; out: while (!traverseStack.isEmpty() && i < filterLevels.size()) { @@ -129,7 +140,7 @@ public void seekPrev(List filterLevels) { // levelNameToSeek == levelName if (i == filterLevels.size()) { // backtrace to find strictly greatest prev topic filter - traverseStack.pop(); + popAndRelease(); if (traverseStack.isEmpty()) { break; } @@ -142,7 +153,7 @@ public void seekPrev(List filterLevels) { break out; } else if (parent.backingTopics().isEmpty()) { // if current stack do not represent a topicfilter in expansion set, backtrace one level up - traverseStack.pop(); + popAndRelease(); } else { // current stack represents the greatest previous topic filter // make sure it points to valid child position @@ -164,7 +175,7 @@ public void seekPrev(List filterLevels) { return; } // backtrace - traverseStack.pop(); + popAndRelease(); if (traverseStack.isEmpty()) { break; } @@ -178,7 +189,7 @@ public void seekPrev(List filterLevels) { break out; } else if (parent.backingTopics().isEmpty()) { // if current stack not represent a valid topic, backtrace one level up - traverseStack.pop(); + popAndRelease(); } else { // current stack represents a topicfilter in expansion set // make sure it points to valid child position @@ -193,7 +204,7 @@ public void seekPrev(List filterLevels) { // no greatest prev topicfilter exists in expansion set for the given topic filter // drain the stack while (!traverseStack.isEmpty()) { - traverseStack.pop(); + popAndRelease(); } } } @@ -220,7 +231,7 @@ public boolean isValid() { public void prev() { assert traverseStack.isEmpty() || !traverseStack.peek().backingTopics().isEmpty(); while (!traverseStack.isEmpty()) { - traverseStack.pop(); + popAndRelease(); if (traverseStack.isEmpty()) { return; } @@ -257,7 +268,7 @@ public void next() { break; } } else { - traverseStack.pop(); + popAndRelease(); if (!traverseStack.isEmpty()) { traverseStack.peek().nextChild(); } @@ -287,4 +298,14 @@ public Map, Set> value() { } return value; } + + private void clearTraverseStack() { + while (!traverseStack.isEmpty()) { + popAndRelease(); + } + } + + private void popAndRelease() { + TopicFilterTrieNode.release(traverseStack.pop()); + } } diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterTrieNode.java b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterTrieNode.java index 32cdb5654..8b46240bd 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterTrieNode.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/main/java/org/apache/bifromq/dist/trie/TopicFilterTrieNode.java @@ -22,6 +22,7 @@ import static org.apache.bifromq.util.TopicConst.NUL; import com.google.common.collect.Lists; +import java.time.Duration; import java.util.List; import java.util.Set; @@ -31,14 +32,25 @@ * @param value type */ abstract class TopicFilterTrieNode { - protected final TopicFilterTrieNode parent; + static final Duration EXPIRE_AFTER = Duration.ofMinutes(1); - protected TopicFilterTrieNode(TopicFilterTrieNode parent) { - this.parent = parent; + protected TopicFilterTrieNode parent; + + protected TopicFilterTrieNode() { } static TopicFilterTrieNode from(TopicTrieNode root) { - return new NTopicFilterTrieNode<>(null, root.levelName(), Set.of(root)); + return NTopicFilterTrieNode.borrow(null, root.levelName(), Set.of(root)); + } + + static void release(TopicFilterTrieNode node) { + if (node instanceof MTopicFilterTrieNode mNode) { + MTopicFilterTrieNode.release(mNode); + } else if (node instanceof STopicFilterTrieNode sNode) { + STopicFilterTrieNode.release(sNode); + } else if (node instanceof NTopicFilterTrieNode nNode) { + NTopicFilterTrieNode.release(nNode); + } } /** diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterIteratorTest.java b/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterIteratorTest.java index 530a9be6d..fc1ea4df0 100644 --- a/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterIteratorTest.java +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterIteratorTest.java @@ -14,7 +14,7 @@ * "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. + * under the License. */ package org.apache.bifromq.dist.trie; @@ -28,9 +28,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; -import org.apache.bifromq.dist.TestUtil; -import org.apache.bifromq.util.TopicConst; -import org.apache.bifromq.util.TopicUtil; import java.util.ArrayList; import java.util.HashSet; import java.util.List; @@ -38,6 +35,9 @@ import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import lombok.extern.slf4j.Slf4j; +import org.apache.bifromq.dist.TestUtil; +import org.apache.bifromq.util.TopicConst; +import org.apache.bifromq.util.TopicUtil; import org.testng.annotations.Test; @Slf4j @@ -47,7 +47,8 @@ public void expandRandomLocalTopic() { String topic = randomTopic(); TopicTrieNode topicTrie = TopicTrieNode.builder(false).addTopic(parse(topic, false), "v1").build(); - TopicFilterIterator itr = new TopicFilterIterator<>(topicTrie); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(topicTrie); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -72,7 +73,8 @@ public void seekExistAndIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -94,7 +96,8 @@ public void randomSeekAndIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -117,7 +120,8 @@ public void seekExistAndBackwardIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -142,7 +146,8 @@ public void randomSeekAndBackwardIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -171,7 +176,8 @@ public void seekPrevExistAndIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -198,7 +204,8 @@ public void randomSeekPrevAndIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -227,7 +234,8 @@ public void seekPrevExistAndBackwardIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -252,7 +260,8 @@ public void randomSeekPrevAndBackwardIteration() { builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); builder.addTopic(parse(randomTopic(), false), "_"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin(NUL, itr.key())); @@ -276,7 +285,8 @@ public void associatedValues() { builder.addTopic(parse("a", false), "v1"); builder.addTopic(parse("a/b", false), "v2"); builder.addTopic(parse("c", false), "v3"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); itr.seek(List.of("#")); assertEquals(itr.value().size(), 3); assertEquals(itr.value().get(List.of("a")), Set.of("v1")); @@ -309,7 +319,8 @@ public void localSysTopicMatch() { builder.addTopic(parse("$sys/a", false), "v1"); builder.addTopic(parse("a/b", false), "v2"); builder.addTopic(parse("c", false), "v3"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); itr.seek(List.of("#")); assertEquals(itr.value().size(), 2); assertEquals(itr.value().get(List.of("a", "b")), Set.of("v2")); @@ -322,7 +333,8 @@ public void globalSysTopicMatch() { builder.addTopic(parse("tenant/$sys/a", false), "v1"); builder.addTopic(parse("tenant/a/b", false), "v2"); builder.addTopic(parse("tenant/c", false), "v3"); - TopicFilterIterator itr = new TopicFilterIterator<>(builder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); itr.seek(List.of("tenant", "#")); assertEquals(itr.value().size(), 2); assertEquals(itr.value().get(List.of("tenant", "a", "b")), Set.of("v2")); @@ -337,7 +349,8 @@ private void expandTopics(Map> topicToFilters, boolean isGl allTopicFilters.addAll(topicToFilters.get(topic)); } List sortedTopicFilters = allTopicFilters.stream().map(TopicUtil::escape).sorted().toList(); - TopicFilterIterator iterator = new TopicFilterIterator<>(topicTrieBuilder.build()); + TopicFilterIterator iterator = new TopicFilterIterator<>(); + iterator.init(topicTrieBuilder.build()); List generated = new ArrayList<>(); for (; iterator.isValid(); iterator.next()) { generated.add(fastJoin(NUL, iterator.key())); diff --git a/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterPoolTest.java b/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterPoolTest.java new file mode 100644 index 000000000..7be9ee385 --- /dev/null +++ b/bifromq-dist/bifromq-dist-coproc-proto/src/test/java/org/apache/bifromq/dist/trie/TopicFilterPoolTest.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.bifromq.dist.trie; + +import static org.awaitility.Awaitility.await; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertTrue; + +import com.github.benmanes.caffeine.cache.Ticker; +import java.time.Duration; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +public class TopicFilterPoolTest { + + @BeforeMethod + public void resetPools() { + NTopicFilterTrieNode.poolClear(); + STopicFilterTrieNode.poolClear(); + MTopicFilterTrieNode.poolClear(); + // reset ticker back to system + NTopicFilterTrieNode.setTicker(null); + STopicFilterTrieNode.setTicker(null); + MTopicFilterTrieNode.setTicker(null); + } + + @AfterMethod + public void cleanup() { + NTopicFilterTrieNode.poolClear(); + STopicFilterTrieNode.poolClear(); + MTopicFilterTrieNode.poolClear(); + } + + @Test + public void reusePoolEntryForNTopicNode() { + TopicTrieNode root = buildLocalTrie(); + TopicTrieNode aNode = root.child("a"); + + NTopicFilterTrieNode n1 = NTopicFilterTrieNode.borrow(null, "a", Set.of(aNode)); + int id1 = System.identityHashCode(n1); + // basic behavior sanity + assertEquals(n1.levelName(), "a"); + assertTrue(n1.backingTopics().contains(aNode)); + NTopicFilterTrieNode.release(n1); + + NTopicFilterTrieNode n2 = NTopicFilterTrieNode.borrow(null, "a", Set.of(aNode)); + int id2 = System.identityHashCode(n2); + assertSame(n1, n2); + assertEquals(id1, id2); + // re-init takes effect + assertEquals(n2.levelName(), "a"); + NTopicFilterTrieNode.release(n2); + } + + @Test + public void reusePoolEntryForSTopicNode() { + TopicTrieNode root = buildLocalTrie(); + Set> wildcardMatchables = new HashSet<>(); + for (TopicTrieNode child : root.children().values()) { + if (child.wildcardMatchable()) { + wildcardMatchables.add(child); + } + } + + STopicFilterTrieNode s1 = STopicFilterTrieNode.borrow(null, wildcardMatchables); + STopicFilterTrieNode.release(s1); + STopicFilterTrieNode s2 = STopicFilterTrieNode.borrow(null, wildcardMatchables); + assertSame(s1, s2); + STopicFilterTrieNode.release(s2); + } + + @Test + public void reusePoolEntryForMTopicNode() { + TopicTrieNode root = buildLocalTrie(); + Set> wildcardMatchables = new HashSet<>(); + for (TopicTrieNode child : root.children().values()) { + if (child.wildcardMatchable()) { + wildcardMatchables.add(child); + } + } + MTopicFilterTrieNode m1 = MTopicFilterTrieNode.borrow(null, wildcardMatchables); + assertTrue(m1.backingTopics().stream().anyMatch(TopicTrieNode::isUserTopic)); + MTopicFilterTrieNode.release(m1); + MTopicFilterTrieNode m2 = MTopicFilterTrieNode.borrow(null, wildcardMatchables); + assertSame(m1, m2); + MTopicFilterTrieNode.release(m2); + } + + @Test + public void ttlExpiryEvictsEntries() { + FakeTicker ticker = new FakeTicker(); + NTopicFilterTrieNode.setTicker(ticker); + STopicFilterTrieNode.setTicker(ticker); + MTopicFilterTrieNode.setTicker(ticker); + + TopicTrieNode root = buildLocalTrie(); + TopicTrieNode aNode = root.child("a"); + Set> wildcardMatchables = new HashSet<>(); + for (TopicTrieNode child : root.children().values()) { + if (child.wildcardMatchable()) { + wildcardMatchables.add(child); + } + } + + // put one entry in each pool + NTopicFilterTrieNode.release(NTopicFilterTrieNode.borrow(null, "a", Set.of(aNode))); + STopicFilterTrieNode.release(STopicFilterTrieNode.borrow(null, wildcardMatchables)); + MTopicFilterTrieNode.release(MTopicFilterTrieNode.borrow(null, wildcardMatchables)); + + await().until(() -> NTopicFilterTrieNode.poolApproxSize() == 1 + && STopicFilterTrieNode.poolApproxSize() == 1 + && MTopicFilterTrieNode.poolApproxSize() == 1); + + // advance beyond TTL and force cleanup + ticker.advance(Duration.ofMinutes(2)); + NTopicFilterTrieNode.poolCleanUp(); + STopicFilterTrieNode.poolCleanUp(); + MTopicFilterTrieNode.poolCleanUp(); + + await().until(() -> NTopicFilterTrieNode.poolApproxSize() == 0 + && STopicFilterTrieNode.poolApproxSize() == 0 + && MTopicFilterTrieNode.poolApproxSize() == 0); + } + + @Test + public void iteratorBacktraceReleasesNodesNoLeak() { + TopicTrieNode.Builder builder = TopicTrieNode.builder(false); + for (int i = 0; i < 10; i++) { + builder.addTopic(List.of("a" + i), "v" + i); + for (int j = 0; j < 3; j++) { + builder.addTopic(List.of("a" + i, "b" + j), "v" + i + j); + } + } + builder.addTopic(List.of("$sys", "x"), "sys"); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(builder.build()); + + for (int round = 0; round < 6; round++) { + if ((round & 1) == 0) { + // forward full scan + itr.seek(List.of()); + for (; itr.isValid(); itr.next()) { + // no-op + } + } else { + // backward full scan + itr.seekPrev(List.of()); + for (; itr.isValid(); itr.prev()) { + // no-op + } + } + // after each round, pools should not grow unbounded + assertTrue(NTopicFilterTrieNode.poolApproxSize() < 256); + assertTrue(STopicFilterTrieNode.poolApproxSize() < 256); + assertTrue(MTopicFilterTrieNode.poolApproxSize() < 256); + } + } + + private TopicTrieNode buildLocalTrie() { + return TopicTrieNode.builder(false) + .addTopic(List.of("a"), "v1") + .addTopic(List.of("a", "b"), "v2") + .addTopic(List.of("c"), "v3") + .addTopic(List.of("$sys", "x"), "sys") + .build(); + } + + static class FakeTicker implements Ticker { + private final AtomicLong nanos = new AtomicLong(System.nanoTime()); + + @Override + public long read() { + return nanos.get(); + } + + void advance(Duration d) { + nanos.addAndGet(d.toNanos()); + } + } +} + diff --git a/bifromq-dist/bifromq-dist-server/src/main/java/org/apache/bifromq/dist/server/scheduler/BatchDistServerCall.java b/bifromq-dist/bifromq-dist-server/src/main/java/org/apache/bifromq/dist/server/scheduler/BatchDistServerCall.java index 6cb199b09..292870448 100644 --- a/bifromq-dist/bifromq-dist-server/src/main/java/org/apache/bifromq/dist/server/scheduler/BatchDistServerCall.java +++ b/bifromq-dist/bifromq-dist-server/src/main/java/org/apache/bifromq/dist/server/scheduler/BatchDistServerCall.java @@ -59,7 +59,8 @@ import org.apache.bifromq.dist.rpc.proto.DistPack; import org.apache.bifromq.dist.rpc.proto.DistServiceROCoProcInput; import org.apache.bifromq.dist.rpc.proto.Fact; -import org.apache.bifromq.dist.trie.TopicFilterIterator; +import org.apache.bifromq.dist.trie.ITopicFilterIterator; +import org.apache.bifromq.dist.trie.ThreadLocalTopicFilterIterator; import org.apache.bifromq.dist.trie.TopicTrieNode; import org.apache.bifromq.type.ClientInfo; import org.apache.bifromq.type.Message; @@ -231,36 +232,39 @@ private Collection rangeLookup() { batch.keySet() .forEach(topic -> topicTrieBuilder.addTopic(TopicUtil.parse(batcherKey.tenantId(), topic, false), topic)); - TopicFilterIterator topicFilterIterator = new TopicFilterIterator<>(topicTrieBuilder.build()); - List finalCandidates = new LinkedList<>(); - for (KVRangeSetting candidate : allCandidates) { - Optional factOpt = candidate.getFact(Fact.class); - if (factOpt.isEmpty()) { - finalCandidates.add(candidate); - continue; - } - Fact fact = factOpt.get(); - if (!fact.hasFirstGlobalFilterLevels() || !fact.hasLastGlobalFilterLevels()) { - // range is empty - continue; - } - List firstFilterLevels = fact.getFirstGlobalFilterLevels().getFilterLevelList(); - List lastFilterLevels = fact.getLastGlobalFilterLevels().getFilterLevelList(); - topicFilterIterator.seek(firstFilterLevels); - if (topicFilterIterator.isValid()) { - // firstTopicFilter <= nextTopicFilter - if (topicFilterIterator.key().equals(firstFilterLevels) || - fastJoin(NUL, topicFilterIterator.key()).compareTo(fastJoin(NUL, lastFilterLevels)) <= 0) { - // if firstTopicFilter == nextTopicFilter || nextFilterLevels <= lastFilterLevels - // add to finalCandidates + try (ITopicFilterIterator topicFilterIterator = + ThreadLocalTopicFilterIterator.get(topicTrieBuilder.build())) { + topicFilterIterator.init(topicTrieBuilder.build()); + List finalCandidates = new LinkedList<>(); + for (KVRangeSetting candidate : allCandidates) { + Optional factOpt = candidate.getFact(Fact.class); + if (factOpt.isEmpty()) { finalCandidates.add(candidate); + continue; + } + Fact fact = factOpt.get(); + if (!fact.hasFirstGlobalFilterLevels() || !fact.hasLastGlobalFilterLevels()) { + // range is empty + continue; + } + List firstFilterLevels = fact.getFirstGlobalFilterLevels().getFilterLevelList(); + List lastFilterLevels = fact.getLastGlobalFilterLevels().getFilterLevelList(); + topicFilterIterator.seek(firstFilterLevels); + if (topicFilterIterator.isValid()) { + // firstTopicFilter <= nextTopicFilter + if (topicFilterIterator.key().equals(firstFilterLevels) + || fastJoin(NUL, topicFilterIterator.key()).compareTo(fastJoin(NUL, lastFilterLevels)) <= 0) { + // if firstTopicFilter == nextTopicFilter || nextFilterLevels <= lastFilterLevels + // add to finalCandidates + finalCandidates.add(candidate); + } + } else { + // endTopicFilter < firstTopicFilter, stop + break; } - } else { - // endTopicFilter < firstTopicFilter, stop - break; } + return finalCandidates; } - return finalCandidates; } private boolean hasSingleItem(Collection collection) { diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java index 58d4622db..ee9351bd4 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/GroupMatching.java @@ -38,7 +38,7 @@ public final class GroupMatching extends Matching { public final List receiverList; private final Map receivers; - GroupMatching(String tenantId, RouteMatcher matcher, Map members) { + public GroupMatching(String tenantId, RouteMatcher matcher, Map members) { super(tenantId, matcher); assert matcher.getType() != RouteMatcher.Type.Normal; this.receivers = members; diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaConstants.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaConstants.java new file mode 100644 index 000000000..7f5db3851 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaConstants.java @@ -0,0 +1,34 @@ +/* + * 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.bifromq.dist.worker.schema; + +import com.google.protobuf.ByteString; + +public class KVSchemaConstants { + public static final ByteString SCHEMA_VER = ByteString.copyFrom(new byte[] {0x00}); + public static final int MAX_RECEIVER_BUCKETS = 0xFF; // one byte + public static final byte FLAG_NORMAL = 0x01; + public static final byte FLAG_UNORDERED = 0x02; + public static final byte FLAG_ORDERED = 0x03; + public static final ByteString SEPARATOR_BYTE = ByteString.copyFrom(new byte[] {0x00}); + public static final ByteString FLAG_NORMAL_VAL = ByteString.copyFrom(new byte[] {FLAG_NORMAL}); + public static final ByteString FLAG_UNORDERED_VAL = ByteString.copyFrom(new byte[] {FLAG_UNORDERED}); + public static final ByteString FLAG_ORDERED_VAL = ByteString.copyFrom(new byte[] {FLAG_ORDERED}); +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java index a46090e38..630dbdcc2 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/KVSchemaUtil.java @@ -21,20 +21,24 @@ import static com.google.protobuf.ByteString.copyFromUtf8; import static com.google.protobuf.UnsafeByteOperations.unsafeWrap; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.FLAG_NORMAL_VAL; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.FLAG_ORDERED_VAL; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.FLAG_UNORDERED_VAL; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.MAX_RECEIVER_BUCKETS; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.SCHEMA_VER; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.SEPARATOR_BYTE; +import static org.apache.bifromq.dist.worker.schema.cache.RouteGroupCache.get; import static org.apache.bifromq.util.BSUtil.toByteString; -import static org.apache.bifromq.util.BSUtil.toShort; -import static org.apache.bifromq.util.TopicConst.DELIMITER; import static org.apache.bifromq.util.TopicConst.NUL; -import static org.apache.bifromq.util.TopicConst.ORDERED_SHARE; -import static org.apache.bifromq.util.TopicConst.UNORDERED_SHARE; -import static org.apache.bifromq.util.TopicUtil.parse; -import static org.apache.bifromq.util.TopicUtil.unescape; -import com.github.benmanes.caffeine.cache.Interner; import com.google.protobuf.ByteString; import java.util.List; import org.apache.bifromq.dist.rpc.proto.MatchRoute; import org.apache.bifromq.dist.rpc.proto.RouteGroup; +import org.apache.bifromq.dist.worker.schema.cache.GroupMatchingCache; +import org.apache.bifromq.dist.worker.schema.cache.NormalMatchingCache; +import org.apache.bifromq.dist.worker.schema.cache.ReceiverCache; +import org.apache.bifromq.dist.worker.schema.cache.RouteDetailCache; import org.apache.bifromq.type.RouteMatcher; import org.apache.bifromq.util.BSUtil; @@ -42,18 +46,6 @@ * Utility for working with the data stored in dist worker. */ public class KVSchemaUtil { - public static final ByteString SCHEMA_VER = ByteString.copyFrom(new byte[] {0x00}); - private static final int MAX_RECEIVER_BUCKETS = 0xFF; // one byte - private static final byte FLAG_NORMAL = 0x01; - private static final byte FLAG_UNORDERED = 0x02; - private static final byte FLAG_ORDERED = 0x03; - private static final ByteString SEPARATOR_BYTE = ByteString.copyFrom(new byte[] {0x00}); - private static final ByteString FLAG_NORMAL_VAL = ByteString.copyFrom(new byte[] {FLAG_NORMAL}); - private static final ByteString FLAG_UNORDERED_VAL = ByteString.copyFrom(new byte[] {FLAG_UNORDERED}); - private static final ByteString FLAG_ORDERED_VAL = ByteString.copyFrom(new byte[] {FLAG_ORDERED}); - private static final Interner MATCHING_INTERNER = Interner.newWeakInterner(); - private static final Interner RECEIVER_INTERNER = Interner.newWeakInterner(); - public static String toReceiverUrl(MatchRoute route) { return toReceiverUrl(route.getBrokerId(), route.getReceiverId(), route.getDelivererKey()); } @@ -63,34 +55,25 @@ public static String toReceiverUrl(int subBrokerId, String receiverId, String de } public static Receiver parseReceiver(String receiverUrl) { - String[] parts = receiverUrl.split(NUL); - return RECEIVER_INTERNER.intern(new Receiver(Integer.parseInt(parts[0]), parts[1], parts[2])); + return ReceiverCache.get(receiverUrl); } public static Matching buildMatchRoute(ByteString routeKey, ByteString routeValue) { RouteDetail routeDetail = parseRouteDetail(routeKey); - try { - if (routeDetail.matcher().getType() == RouteMatcher.Type.Normal) { - return buildNormalMatchRoute(routeDetail, BSUtil.toLong(routeValue)); - } - return buildGroupMatchRoute(routeDetail, RouteGroup.parseFrom(routeValue)); - } catch (Exception e) { - throw new IllegalStateException("Unable to parse matching record", e); + if (routeDetail.matcher().getType() == RouteMatcher.Type.Normal) { + return buildNormalMatchRoute(routeDetail, BSUtil.toLong(routeValue)); } + return buildGroupMatchRoute(routeDetail, get(routeValue)); } public static Matching buildNormalMatchRoute(RouteDetail routeDetail, long incarnation) { assert routeDetail.matcher().getType() == RouteMatcher.Type.Normal; - return MATCHING_INTERNER.intern(new NormalMatching(routeDetail.tenantId(), - routeDetail.matcher(), - routeDetail.receiverUrl(), - incarnation)); + return NormalMatchingCache.get(routeDetail, incarnation); } public static Matching buildGroupMatchRoute(RouteDetail routeDetail, RouteGroup group) { assert routeDetail.matcher().getType() != RouteMatcher.Type.Normal; - return MATCHING_INTERNER.intern(new GroupMatching(routeDetail.tenantId(), routeDetail.matcher(), - group.getMembersMap())); + return GroupMatchingCache.get(routeDetail, group); } public static ByteString tenantBeginKey(String tenantId) { @@ -125,60 +108,7 @@ public static ByteString toGroupRouteKey(String tenantId, RouteMatcher routeMatc } public static RouteDetail parseRouteDetail(ByteString routeKey) { - // - short tenantIdLen = tenantIdLen(routeKey); - int tenantIdStartIdx = SCHEMA_VER.size() + Short.BYTES; - int escapedTopicFilterStartIdx = tenantIdStartIdx + tenantIdLen; - int receiverBytesLen = receiverBytesLen(routeKey); - int receiverBytesStartIdx = routeKey.size() - Short.BYTES - receiverBytesLen; - int receiverBytesEndIdx = routeKey.size() - Short.BYTES; - int flagByteIdx = receiverBytesStartIdx - 1; - int separatorBytesIdx = flagByteIdx - 1 - 2; // 2 bytes separator - String receiverInfo = routeKey.substring(receiverBytesStartIdx, receiverBytesEndIdx).toStringUtf8(); - byte flag = routeKey.byteAt(flagByteIdx); - - String tenantId = routeKey.substring(tenantIdStartIdx, escapedTopicFilterStartIdx).toStringUtf8(); - String escapedTopicFilter = - routeKey.substring(escapedTopicFilterStartIdx, separatorBytesIdx).toStringUtf8(); - switch (flag) { - case FLAG_NORMAL -> { - RouteMatcher matcher = RouteMatcher.newBuilder() - .setType(RouteMatcher.Type.Normal) - .addAllFilterLevel(parse(escapedTopicFilter, true)) - .setMqttTopicFilter(unescape(escapedTopicFilter)) - .build(); - return new RouteDetail(tenantId, matcher, receiverInfo); - } - case FLAG_UNORDERED -> { - RouteMatcher matcher = RouteMatcher.newBuilder() - .setType(RouteMatcher.Type.UnorderedShare) - .addAllFilterLevel(parse(escapedTopicFilter, true)) - .setGroup(receiverInfo) - .setMqttTopicFilter( - UNORDERED_SHARE + DELIMITER + receiverInfo + DELIMITER + unescape(escapedTopicFilter)) - .build(); - return new RouteDetail(tenantId, matcher, null); - } - case FLAG_ORDERED -> { - RouteMatcher matcher = RouteMatcher.newBuilder() - .setType(RouteMatcher.Type.OrderedShare) - .addAllFilterLevel(parse(escapedTopicFilter, true)) - .setGroup(receiverInfo) - .setMqttTopicFilter( - ORDERED_SHARE + DELIMITER + receiverInfo + DELIMITER + unescape(escapedTopicFilter)) - .build(); - return new RouteDetail(tenantId, matcher, null); - } - default -> throw new UnsupportedOperationException("Unknown route type: " + flag); - } - } - - private static short tenantIdLen(ByteString routeKey) { - return toShort(routeKey.substring(SCHEMA_VER.size(), SCHEMA_VER.size() + Short.BYTES)); - } - - private static short receiverBytesLen(ByteString routeKey) { - return toShort(routeKey.substring(routeKey.size() - Short.BYTES)); + return RouteDetailCache.get(routeKey); } private static ByteString toReceiverBytes(String receiver) { @@ -190,8 +120,4 @@ private static byte bucket(String receiver) { int hash = receiver.hashCode(); return (byte) ((hash ^ (hash >>> 16)) & MAX_RECEIVER_BUCKETS); } - - public record Receiver(int subBrokerId, String receiverId, String delivererKey) { - - } } diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java index f7650af30..3090ba954 100644 --- a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/NormalMatching.java @@ -38,9 +38,9 @@ public class NormalMatching extends Matching { private final MatchInfo matchInfo; @EqualsAndHashCode.Exclude - private final KVSchemaUtil.Receiver receiver; + private final Receiver receiver; - NormalMatching(String tenantId, RouteMatcher matcher, String receiverUrl, long incarnation) { + public NormalMatching(String tenantId, RouteMatcher matcher, String receiverUrl, long incarnation) { super(tenantId, matcher); this.receiverUrl = receiverUrl; this.receiver = KVSchemaUtil.parseReceiver(receiverUrl); diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Receiver.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Receiver.java new file mode 100644 index 000000000..8c7a61b80 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/Receiver.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.bifromq.dist.worker.schema; + +public record Receiver(int subBrokerId, String receiverId, String delivererKey) { + +} \ No newline at end of file diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCache.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCache.java new file mode 100644 index 000000000..17924f0a1 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCache.java @@ -0,0 +1,47 @@ +/* + * 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.bifromq.dist.worker.schema.cache; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Interner; +import org.apache.bifromq.dist.rpc.proto.RouteGroup; +import org.apache.bifromq.dist.worker.schema.GroupMatching; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.RouteDetail; +import org.apache.bifromq.type.RouteMatcher; + +public class GroupMatchingCache { + private static final Interner CACHE_KEY_INTERNER = Interner.newWeakInterner(); + private static final Cache GROUP_MATCHING_CACHE = Caffeine.newBuilder().weakKeys() + .build(); + private static final Interner MATCHING_INTERNER = Interner.newWeakInterner(); + + public static Matching get(RouteDetail routeDetail, RouteGroup group) { + assert routeDetail.matcher().getType() != RouteMatcher.Type.Normal; + CacheKey key = CACHE_KEY_INTERNER.intern(new CacheKey(routeDetail, group)); + return GROUP_MATCHING_CACHE.get(key, k -> + MATCHING_INTERNER.intern(new GroupMatching(k.routeDetail().tenantId(), k.routeDetail.matcher(), + k.routeGroup().getMembersMap()))); + } + + private record CacheKey(RouteDetail routeDetail, RouteGroup routeGroup) { + } +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCache.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCache.java new file mode 100644 index 000000000..712364b4b --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCache.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.bifromq.dist.worker.schema.cache; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Interner; +import org.apache.bifromq.dist.worker.schema.Matching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; +import org.apache.bifromq.dist.worker.schema.RouteDetail; +import org.apache.bifromq.type.RouteMatcher; + +public class NormalMatchingCache { + private static final Interner CACHE_KEY_INTERNER = Interner.newWeakInterner(); + private static final Cache NORMAL_MATCHING_CACHE = Caffeine.newBuilder().weakKeys() + .build(); + private static final Interner MATCHING_INTERNER = Interner.newWeakInterner(); + + public static Matching get(RouteDetail routeDetail, long incarnation) { + assert routeDetail.matcher().getType() == RouteMatcher.Type.Normal; + CacheKey key = CACHE_KEY_INTERNER.intern(new CacheKey(routeDetail, incarnation)); + return NORMAL_MATCHING_CACHE.get(key, k -> MATCHING_INTERNER.intern(new NormalMatching(k.routeDetail.tenantId(), + k.routeDetail.matcher(), + k.routeDetail.receiverUrl(), k.incarnation))); + } + + private record CacheKey(RouteDetail routeDetail, long incarnation) { + } +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCache.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCache.java new file mode 100644 index 000000000..de3a84151 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCache.java @@ -0,0 +1,41 @@ +/* + * 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.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.util.TopicConst.NUL; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Interner; +import org.apache.bifromq.dist.worker.schema.Receiver; + +public class ReceiverCache { + private static final Interner RECEIVER_URL_INTERNER = Interner.newWeakInterner(); + private static final Interner RECEIVER_INTERNER = Interner.newWeakInterner(); + private static final Cache RECEIVER_CACHE = Caffeine.newBuilder().weakKeys().build(); + + public static Receiver get(String receiverUrl) { + receiverUrl = RECEIVER_URL_INTERNER.intern(receiverUrl); + return RECEIVER_CACHE.get(receiverUrl, k -> { + String[] parts = k.split(NUL); + return RECEIVER_INTERNER.intern(new Receiver(Integer.parseInt(parts[0]), parts[1], parts[2])); + }); + } +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCache.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCache.java new file mode 100644 index 000000000..19f67a47f --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCache.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.FLAG_NORMAL; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.FLAG_ORDERED; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.FLAG_UNORDERED; +import static org.apache.bifromq.dist.worker.schema.KVSchemaConstants.SCHEMA_VER; +import static org.apache.bifromq.util.BSUtil.toShort; +import static org.apache.bifromq.util.TopicConst.DELIMITER; +import static org.apache.bifromq.util.TopicConst.ORDERED_SHARE; +import static org.apache.bifromq.util.TopicConst.UNORDERED_SHARE; +import static org.apache.bifromq.util.TopicUtil.parse; +import static org.apache.bifromq.util.TopicUtil.unescape; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Interner; +import com.google.protobuf.ByteString; +import org.apache.bifromq.dist.worker.schema.RouteDetail; +import org.apache.bifromq.type.RouteMatcher; + +public class RouteDetailCache { + private static final Interner TENANTID_INTERNER = Interner.newWeakInterner(); + private static final Interner MQTT_TOPIC_FILTER_INTERNER = Interner.newWeakInterner(); + private static final Interner ROUTE_KEY_INTERNER = Interner.newWeakInterner(); + private static final Interner ROUTE_MATCHER_INTERNER = Interner.newWeakInterner(); + private static final Cache ROUTE_MATCHER_CACHE = Caffeine.newBuilder().weakKeys().build(); + private static final Cache ROUTE_DETAIL_CACHE = Caffeine.newBuilder().weakKeys().build(); + + public static RouteDetail get(ByteString routeKey) { + // + routeKey = ROUTE_KEY_INTERNER.intern(routeKey); + return ROUTE_DETAIL_CACHE.get(routeKey, k -> { + short tenantIdLen = tenantIdLen(k); + int tenantIdStartIdx = SCHEMA_VER.size() + Short.BYTES; + int escapedTopicFilterStartIdx = tenantIdStartIdx + tenantIdLen; + int receiverBytesLen = receiverBytesLen(k); + int receiverBytesStartIdx = k.size() - Short.BYTES - receiverBytesLen; + int receiverBytesEndIdx = k.size() - Short.BYTES; + int flagByteIdx = receiverBytesStartIdx - 1; + int separatorBytesIdx = flagByteIdx - 1 - 2; // 2 bytes separator + String receiverInfo = k.substring(receiverBytesStartIdx, receiverBytesEndIdx).toStringUtf8(); + byte flag = k.byteAt(flagByteIdx); + + String tenantId = TENANTID_INTERNER.intern( + k.substring(tenantIdStartIdx, escapedTopicFilterStartIdx).toStringUtf8()); + String escapedTopicFilter = k.substring(escapedTopicFilterStartIdx, separatorBytesIdx) + .toStringUtf8(); + switch (flag) { + case FLAG_NORMAL -> { + String mqttTopicFilter = MQTT_TOPIC_FILTER_INTERNER.intern(unescape(escapedTopicFilter)); + RouteMatcher matcher = ROUTE_MATCHER_CACHE.get(mqttTopicFilter, + t -> ROUTE_MATCHER_INTERNER.intern(RouteMatcher.newBuilder() + .setType(RouteMatcher.Type.Normal) + .addAllFilterLevel(parse(escapedTopicFilter, true)) + .setMqttTopicFilter(t) + .build())); + return new RouteDetail(tenantId, matcher, receiverInfo); // receiverInfo is the receiverUrl + } + case FLAG_UNORDERED -> { + String mqttTopicFilter = MQTT_TOPIC_FILTER_INTERNER.intern( + UNORDERED_SHARE + DELIMITER + receiverInfo + DELIMITER + unescape(escapedTopicFilter)); + RouteMatcher matcher = ROUTE_MATCHER_CACHE.get(mqttTopicFilter, + t -> ROUTE_MATCHER_INTERNER.intern(RouteMatcher.newBuilder() + .setType(RouteMatcher.Type.UnorderedShare) + .addAllFilterLevel(parse(escapedTopicFilter, true)) + .setGroup(receiverInfo) // receiverInfo is the group name + .setMqttTopicFilter(t) + .build())); + return new RouteDetail(tenantId, matcher, null); + } + case FLAG_ORDERED -> { + String mqttTopicFilter = MQTT_TOPIC_FILTER_INTERNER.intern( + ORDERED_SHARE + DELIMITER + receiverInfo + DELIMITER + unescape(escapedTopicFilter)); + RouteMatcher matcher = ROUTE_MATCHER_CACHE.get(mqttTopicFilter, t -> { + RouteMatcher m = RouteMatcher.newBuilder() + .setType(RouteMatcher.Type.OrderedShare) + .addAllFilterLevel(parse(escapedTopicFilter, true)) + .setGroup(receiverInfo) // group name + .setMqttTopicFilter(t) + .build(); + return ROUTE_MATCHER_INTERNER.intern(m); + }); + return new RouteDetail(tenantId, matcher, null); + } + default -> throw new UnsupportedOperationException("Unknown route type: " + flag); + } + }); + } + + private static short tenantIdLen(ByteString routeKey) { + return toShort(routeKey.substring(SCHEMA_VER.size(), SCHEMA_VER.size() + Short.BYTES)); + } + + private static short receiverBytesLen(ByteString routeKey) { + return toShort(routeKey.substring(routeKey.size() - Short.BYTES)); + } +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCache.java b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCache.java new file mode 100644 index 000000000..8e2de8a19 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/main/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCache.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.bifromq.dist.worker.schema.cache; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Interner; +import com.google.protobuf.ByteString; +import com.google.protobuf.InvalidProtocolBufferException; +import org.apache.bifromq.dist.rpc.proto.RouteGroup; + +public class RouteGroupCache { + private static final Interner ROUTE_GROUP_BYTES_INTERNER = Interner.newWeakInterner(); + private static final Interner ROUTE_GROUP_INTERNER = Interner.newWeakInterner(); + private static final Cache ROUTE_GROUP_CACHE = Caffeine.newBuilder().weakKeys().build(); + + public static RouteGroup get(ByteString routeGroupBytes) { + routeGroupBytes = ROUTE_GROUP_BYTES_INTERNER.intern(routeGroupBytes); + return ROUTE_GROUP_CACHE.get(routeGroupBytes, k -> { + try { + return ROUTE_GROUP_INTERNER.intern(RouteGroup.parseFrom(k)); + } catch (InvalidProtocolBufferException e) { + throw new IllegalStateException("Unable to parse matching record", e); + } + }); + } +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCacheTest.java b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCacheTest.java new file mode 100644 index 000000000..100530e40 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/GroupMatchingCacheTest.java @@ -0,0 +1,166 @@ +/* + * 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.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotSame; +import static org.testng.Assert.assertSame; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import org.apache.bifromq.dist.rpc.proto.RouteGroup; +import org.apache.bifromq.dist.worker.schema.GroupMatching; +import org.apache.bifromq.dist.worker.schema.NormalMatching; +import org.apache.bifromq.dist.worker.schema.RouteDetail; +import org.apache.bifromq.type.RouteMatcher; +import org.apache.bifromq.util.TopicUtil; +import org.testng.annotations.Test; + +public class GroupMatchingCacheTest { + + private static RouteDetail detail(String tenantId, RouteMatcher matcher) { + return new RouteDetail(tenantId, matcher, null); + } + + private static RouteGroup groupOf(Map members) { + RouteGroup.Builder b = RouteGroup.newBuilder(); + for (Map.Entry e : members.entrySet()) { + b.putMembers(e.getKey(), e.getValue()); + } + return b.build(); + } + + @Test + public void sameKeyIntern() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topic = "$share/group-" + UUID.randomUUID() + "/home/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topic); + + Map members = new HashMap<>(); + members.put(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L); + members.put(toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"), 2L); + + RouteDetail d = detail(tenantId, matcher); + RouteGroup g = groupOf(members); + + GroupMatching m1 = (GroupMatching) GroupMatchingCache.get(d, g); + GroupMatching m2 = (GroupMatching) GroupMatchingCache.get(d, g); + assertSame(m1, m2); + } + + @Test + public void membershipChangeCreatesNewInstance() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topic = "$share/group-" + UUID.randomUUID() + "/home/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topic); + + String url1 = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"); + String url2 = toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"); + + RouteDetail d = detail(tenantId, matcher); + RouteGroup g1 = RouteGroup.newBuilder().putMembers(url1, 1L).putMembers(url2, 2L).build(); + RouteGroup g2 = RouteGroup.newBuilder().putMembers(url1, 1L).build(); // remove one member + + GroupMatching m1 = (GroupMatching) GroupMatchingCache.get(d, g1); + GroupMatching m2 = (GroupMatching) GroupMatchingCache.get(d, g2); + assertNotSame(m1, m2); + } + + @Test + public void orderedFlagPropagates() { + String tenantId = "tenant-" + UUID.randomUUID(); + RouteMatcher unordered = TopicUtil.from("$share/g" + UUID.randomUUID() + "/a/b"); + RouteMatcher ordered = TopicUtil.from("$oshare/g" + UUID.randomUUID() + "/a/b"); + RouteGroup g = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L) + .build(); + + GroupMatching mu = (GroupMatching) GroupMatchingCache.get(detail(tenantId, unordered), g); + GroupMatching mo = (GroupMatching) GroupMatchingCache.get(detail(tenantId, ordered), g); + + // unordered share => ordered=false; ordered share => ordered=true + assertEquals(mu.ordered, false); + assertEquals(mo.ordered, true); + } + + @Test + public void receiverListConsistency() { + String tenantId = "tenant-" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from("$share/g" + UUID.randomUUID() + "/a/b"); + + String url1 = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"); + String url2 = toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"); + long inc1 = 1L, inc2 = 2L; + + RouteGroup g = RouteGroup.newBuilder().putMembers(url1, inc1).putMembers(url2, inc2).build(); + GroupMatching m = (GroupMatching) GroupMatchingCache.get(detail(tenantId, matcher), g); + + Map expected = g.getMembersMap(); + assertEquals(m.receiverList.size(), expected.size()); + + Map actual = new HashMap<>(); + for (NormalMatching n : m.receiverList) { + actual.put(n.receiverUrl(), n.incarnation()); + // Each sub matching preserves matcher mqttTopicFilter + assertEquals(n.matchInfo().getMatcher().getMqttTopicFilter(), matcher.getMqttTopicFilter()); + } + assertEquals(actual, expected); + } + + @Test + public void concurrentGetSameKey() throws Exception { + String tenantId = "tenant-" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from("$share/g" + UUID.randomUUID() + "/a/b"); + RouteGroup g = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L) + .putMembers(toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"), 2L) + .build(); + RouteDetail d = detail(tenantId, matcher); + + int threads = 16; + ExecutorService pool = Executors.newFixedThreadPool(threads); + try { + CountDownLatch start = new CountDownLatch(1); + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { + futures.add(pool.submit(() -> { + start.await(); + return (GroupMatching) GroupMatchingCache.get(d, g); + })); + } + start.countDown(); + GroupMatching first = futures.get(0).get(); + for (Future f : futures) { + assertSame(first, f.get(), "Concurrent gets should return same GroupMatching instance"); + } + } finally { + pool.shutdownNow(); + } + } +} + diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCacheTest.java b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCacheTest.java new file mode 100644 index 000000000..60750a7cf --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/NormalMatchingCacheTest.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.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotSame; +import static org.testng.Assert.assertSame; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import org.apache.bifromq.dist.worker.schema.NormalMatching; +import org.apache.bifromq.dist.worker.schema.RouteDetail; +import org.apache.bifromq.type.MatchInfo; +import org.apache.bifromq.type.RouteMatcher; +import org.apache.bifromq.util.TopicUtil; +import org.testng.annotations.Test; + +public class NormalMatchingCacheTest { + + private static RouteDetail detail(String tenantId, String topicFilter, String receiverUrl) { + RouteMatcher matcher = TopicUtil.from(topicFilter); + return new RouteDetail(tenantId, matcher, receiverUrl); + } + + @Test + public void sameKeySameIncarnationIntern() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/s/" + UUID.randomUUID(); + String receiverUrl = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + RouteDetail d = detail(tenantId, topicFilter, receiverUrl); + + NormalMatching m1 = (NormalMatching) NormalMatchingCache.get(d, 1L); + NormalMatching m2 = (NormalMatching) NormalMatchingCache.get(d, 1L); + assertSame(m1, m2); + } + + @Test + public void differentIncarnationNotSame() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/s/" + UUID.randomUUID(); + String receiverUrl = toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + RouteDetail d = detail(tenantId, topicFilter, receiverUrl); + + NormalMatching m1 = (NormalMatching) NormalMatchingCache.get(d, 1L); + NormalMatching m2 = (NormalMatching) NormalMatchingCache.get(d, 2L); + assertNotSame(m1, m2); + } + + @Test + public void differentReceiverNotSame() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/s/" + UUID.randomUUID(); + String receiverUrl1 = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"); + String receiverUrl2 = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"); + NormalMatching m1 = (NormalMatching) NormalMatchingCache.get(detail(tenantId, topicFilter, receiverUrl1), 1L); + NormalMatching m2 = (NormalMatching) NormalMatchingCache.get(detail(tenantId, topicFilter, receiverUrl2), 1L); + assertNotSame(m1, m2); + } + + @Test + public void differentMatcherNotSame() { + String tenantId = "tenant-" + UUID.randomUUID(); + String receiverUrl = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"); + NormalMatching m1 = (NormalMatching) NormalMatchingCache.get(detail(tenantId, "/a/#", receiverUrl), 1L); + NormalMatching m2 = (NormalMatching) NormalMatchingCache.get(detail(tenantId, "/a/+", receiverUrl), 1L); + assertNotSame(m1, m2); + } + + @Test + public void fieldCorrectness() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/s/" + UUID.randomUUID(); + int subBrokerId = 7; + String receiverId = "inbox-" + UUID.randomUUID(); + String deliverer = "d-" + UUID.randomUUID(); + String receiverUrl = toReceiverUrl(subBrokerId, receiverId, deliverer); + NormalMatching m = (NormalMatching) NormalMatchingCache.get(detail(tenantId, topicFilter, receiverUrl), 42L); + + assertEquals(m.receiverUrl(), receiverUrl); + assertEquals(m.subBrokerId(), subBrokerId); + assertEquals(m.delivererKey(), deliverer); + MatchInfo info = m.matchInfo(); + assertEquals(info.getReceiverId(), receiverId); + assertEquals(info.getIncarnation(), 42L); + assertEquals(info.getMatcher().getMqttTopicFilter(), topicFilter); + } + + @Test + public void concurrentGetSameKey() throws Exception { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/s/" + UUID.randomUUID(); + String receiverUrl = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + RouteDetail d = detail(tenantId, topicFilter, receiverUrl); + + int threads = 16; + ExecutorService pool = Executors.newFixedThreadPool(threads); + try { + CountDownLatch start = new CountDownLatch(1); + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { + futures.add(pool.submit(() -> { + start.await(); + return (NormalMatching) NormalMatchingCache.get(d, 1L); + })); + } + start.countDown(); + NormalMatching first = futures.get(0).get(); + for (Future f : futures) { + assertSame(first, f.get()); + } + } finally { + pool.shutdownNow(); + } + } +} + diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCacheTest.java b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCacheTest.java new file mode 100644 index 000000000..c045a58a9 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/ReceiverCacheTest.java @@ -0,0 +1,90 @@ +/* + * 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.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotSame; +import static org.testng.Assert.assertSame; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import org.apache.bifromq.dist.worker.schema.Receiver; +import org.testng.annotations.Test; + +public class ReceiverCacheTest { + + @Test + public void parseAndCache() { + int brokerId = (int) (System.nanoTime() & 0x7fffffff); + String receiverId = "inbox-" + UUID.randomUUID(); + String delivererKey = "deliverer-" + UUID.randomUUID(); + String url = toReceiverUrl(brokerId, receiverId, delivererKey); + + Receiver r1 = ReceiverCache.get(url); + Receiver r2 = ReceiverCache.get(url); + + assertSame(r1, r2); + assertEquals(r1.subBrokerId(), brokerId); + assertEquals(r1.receiverId(), receiverId); + assertEquals(r1.delivererKey(), delivererKey); + } + + @Test + public void differentReceiverNotSame() { + String url1 = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"); + String url2 = toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"); + + Receiver r1 = ReceiverCache.get(url1); + Receiver r2 = ReceiverCache.get(url2); + + assertNotSame(r1, r2); + } + + @Test + public void concurrentGetSameKey() throws Exception { + int threads = 16; + ExecutorService pool = Executors.newFixedThreadPool(threads); + try { + String url = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + CountDownLatch start = new CountDownLatch(1); + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { + futures.add(pool.submit(() -> { + start.await(); + return ReceiverCache.get(url); + })); + } + start.countDown(); + Receiver first = futures.get(0).get(); + for (Future f : futures) { + assertSame(first, f.get()); + } + } finally { + pool.shutdownNow(); + } + } +} + diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCacheTest.java b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCacheTest.java new file mode 100644 index 000000000..f233ffa57 --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteDetailCacheTest.java @@ -0,0 +1,174 @@ +/* + * 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.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toGroupRouteKey; +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toNormalRouteKey; +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotSame; +import static org.testng.Assert.assertSame; + +import com.google.protobuf.ByteString; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import org.apache.bifromq.dist.worker.schema.RouteDetail; +import org.apache.bifromq.type.RouteMatcher; +import org.apache.bifromq.util.TopicUtil; +import org.testng.annotations.Test; + +public class RouteDetailCacheTest { + + @Test + public void parseNormalRouteDetailAndIntern() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/home/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topicFilter); + String receiverUrl = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + + ByteString routeKey = toNormalRouteKey(tenantId, matcher, receiverUrl); + RouteDetail d1 = RouteDetailCache.get(routeKey); + RouteDetail d2 = RouteDetailCache.get(routeKey); + + assertSame(d1, d2); + assertEquals(d1.tenantId(), tenantId); + assertEquals(d1.receiverUrl(), receiverUrl); + assertEquals(d1.matcher().getType(), RouteMatcher.Type.Normal); + assertEquals(d1.matcher().getMqttTopicFilter(), topicFilter); + } + + @Test + public void parseUnorderedShareRouteDetail() { + String tenantId = "tenant-" + UUID.randomUUID(); + String origTopicFilter = "$share/group-" + UUID.randomUUID() + "/s/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(origTopicFilter); + + ByteString routeKey = toGroupRouteKey(tenantId, matcher); + RouteDetail d = RouteDetailCache.get(routeKey); + + assertEquals(d.tenantId(), tenantId); + assertEquals(d.receiverUrl(), null); + assertEquals(d.matcher().getType(), RouteMatcher.Type.UnorderedShare); + assertEquals(d.matcher().getGroup(), matcher.getGroup()); + assertEquals(d.matcher().getMqttTopicFilter(), origTopicFilter); + } + + @Test + public void parseOrderedShareRouteDetail() { + String tenantId = "tenant-" + UUID.randomUUID(); + String origTopicFilter = "$oshare/group-" + UUID.randomUUID() + "/s/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(origTopicFilter); + + ByteString routeKey = toGroupRouteKey(tenantId, matcher); + RouteDetail d = RouteDetailCache.get(routeKey); + + assertEquals(d.tenantId(), tenantId); + assertEquals(d.receiverUrl(), null); + assertEquals(d.matcher().getType(), RouteMatcher.Type.OrderedShare); + assertEquals(d.matcher().getGroup(), matcher.getGroup()); + assertEquals(d.matcher().getMqttTopicFilter(), origTopicFilter); + } + + @Test + public void equalContentDifferentByteStringInstancesInterned() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/room/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topicFilter); + String receiverUrl = toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + + ByteString k1 = toNormalRouteKey(tenantId, matcher, receiverUrl); + ByteString k2 = ByteString.copyFrom(k1.toByteArray()); + + RouteDetail d1 = RouteDetailCache.get(k1); + RouteDetail d2 = RouteDetailCache.get(k2); + assertSame(d1, d2); + } + + @Test + public void differentKeysProduceDifferentDetails() { + String tenantId1 = "tenant-" + UUID.randomUUID(); + String tenantId2 = "tenant-" + UUID.randomUUID(); + String topicFilter = "/a/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topicFilter); + String receiverUrl = toReceiverUrl(3, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + + ByteString k1 = toNormalRouteKey(tenantId1, matcher, receiverUrl); + ByteString k2 = toNormalRouteKey(tenantId2, matcher, receiverUrl); + + RouteDetail d1 = RouteDetailCache.get(k1); + RouteDetail d2 = RouteDetailCache.get(k2); + assertNotSame(d1, d2); + } + + @Test(expectedExceptions = UnsupportedOperationException.class) + public void unsupportedFlagThrows() { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/x/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topicFilter); + String receiverUrl = toReceiverUrl(9, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + + ByteString normalKey = toNormalRouteKey(tenantId, matcher, receiverUrl); + // Rewrite flag byte to an unsupported value (e.g., 0x7F) + short receiverBytesLen = normalKey.substring(normalKey.size() - Short.BYTES).asReadOnlyByteBuffer().getShort(); + int receiverBytesStartIdx = normalKey.size() - Short.BYTES - receiverBytesLen; + int flagByteIdx = receiverBytesStartIdx - 1; + ByteString prefix = normalKey.substring(0, flagByteIdx); + ByteString invalidFlag = ByteString.copyFrom(new byte[] {(byte) 0x7F}); + ByteString suffix = normalKey.substring(flagByteIdx + 1); + ByteString invalidKey = prefix.concat(invalidFlag).concat(suffix); + + // Should throw UnsupportedOperationException + RouteDetailCache.get(invalidKey); + } + + @Test + public void concurrentGetSameKey() throws Exception { + String tenantId = "tenant-" + UUID.randomUUID(); + String topicFilter = "/home/" + UUID.randomUUID(); + RouteMatcher matcher = TopicUtil.from(topicFilter); + String receiverUrl = toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d-" + UUID.randomUUID()); + ByteString k = toNormalRouteKey(tenantId, matcher, receiverUrl); + + int threads = 16; + ExecutorService pool = Executors.newFixedThreadPool(threads); + try { + CountDownLatch start = new CountDownLatch(1); + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { + futures.add(pool.submit(() -> { + start.await(); + return RouteDetailCache.get(k); + })); + } + start.countDown(); + RouteDetail first = futures.get(0).get(); + for (Future f : futures) { + assertSame(first, f.get()); + } + } finally { + pool.shutdownNow(); + } + } +} diff --git a/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCacheTest.java b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCacheTest.java new file mode 100644 index 000000000..644453b6e --- /dev/null +++ b/bifromq-dist/bifromq-dist-worker-schema/src/test/java/org/apache/bifromq/dist/worker/schema/cache/RouteGroupCacheTest.java @@ -0,0 +1,121 @@ +/* + * 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.bifromq.dist.worker.schema.cache; + +import static org.apache.bifromq.dist.worker.schema.KVSchemaUtil.toReceiverUrl; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotEquals; +import static org.testng.Assert.assertSame; +import static org.testng.Assert.assertThrows; + +import com.google.protobuf.ByteString; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import org.apache.bifromq.dist.rpc.proto.RouteGroup; +import org.testng.annotations.Test; + +public class RouteGroupCacheTest { + + @Test + public void sameByteStringInstanceIsInterned() { + RouteGroup group = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L) + .putMembers(toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"), 2L) + .build(); + ByteString bytes = group.toByteString(); + + RouteGroup g1 = RouteGroupCache.get(bytes); + RouteGroup g2 = RouteGroupCache.get(bytes); + + assertSame(g1, g2); + } + + @Test + public void equalContentDifferentByteString() { + RouteGroup group = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L) + .build(); + ByteString bs1 = group.toByteString(); + ByteString bs2 = ByteString.copyFrom(bs1.toByteArray()); + + RouteGroup g1 = RouteGroupCache.get(bs1); + RouteGroup g2 = RouteGroupCache.get(bs2); + + // Content equality must hold + assertEquals(g1, g2); + // Implementation may or may not intern across different ByteString instances; avoid over-constraining + assertSame(g1, g2); + } + + @Test + public void differentContent() { + ByteString bs1 = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L) + .build().toByteString(); + ByteString bs2 = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 2L) // different incarnation + .build().toByteString(); + + RouteGroup g1 = RouteGroupCache.get(bs1); + RouteGroup g2 = RouteGroupCache.get(bs2); + + assertNotEquals(g1, g2); + } + + @Test + public void invalidBytes() { + ByteString invalid = ByteString.copyFromUtf8("not-a-route-group"); + assertThrows(RuntimeException.class, () -> RouteGroupCache.get(invalid)); + } + + @Test + public void concurrentGetSameKey() throws Exception { + RouteGroup group = RouteGroup.newBuilder() + .putMembers(toReceiverUrl(1, "inbox-" + UUID.randomUUID(), "d1"), 1L) + .putMembers(toReceiverUrl(2, "inbox-" + UUID.randomUUID(), "d2"), 2L) + .build(); + ByteString bytes = group.toByteString(); + + int threads = 16; + ExecutorService pool = Executors.newFixedThreadPool(threads); + try { + CountDownLatch start = new CountDownLatch(1); + List> futures = new ArrayList<>(); + for (int i = 0; i < threads; i++) { + futures.add(pool.submit(() -> { + start.await(); + return RouteGroupCache.get(bytes); + })); + } + start.countDown(); + RouteGroup first = futures.get(0).get(); + for (Future f : futures) { + assertSame(first, f.get()); + } + } finally { + pool.shutdownNow(); + } + } +} diff --git a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java index 2fb3ae79d..9e263c1d7 100644 --- a/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java +++ b/bifromq-dist/bifromq-dist-worker/src/main/java/org/apache/bifromq/dist/worker/cache/TenantRouteMatcher.java @@ -39,7 +39,8 @@ import org.apache.bifromq.basekv.proto.Boundary; import org.apache.bifromq.basekv.store.api.IKVIterator; import org.apache.bifromq.basekv.store.api.IKVReader; -import org.apache.bifromq.dist.trie.TopicFilterIterator; +import org.apache.bifromq.dist.trie.ITopicFilterIterator; +import org.apache.bifromq.dist.trie.ThreadLocalTopicFilterIterator; import org.apache.bifromq.dist.trie.TopicTrieNode; import org.apache.bifromq.dist.worker.schema.GroupMatching; import org.apache.bifromq.dist.worker.schema.Matching; @@ -85,75 +86,76 @@ public Map matchAll(Set topics, if (isNULLRange(tenantBoundary)) { return matchedRoutes; } - - TopicFilterIterator expansionSetItr = new TopicFilterIterator<>(topicTrieBuilder.build()); - Map, Set> matchedTopicFilters = new HashMap<>(); - IKVIterator itr = rangeReader.iterator(); - // track seek - itr.seek(tenantBoundary.getStartKey()); - int probe = 0; - - while (itr.isValid() && compare(itr.key(), tenantBoundary.getEndKey()) < 0) { - // track itr.key() - Matching matching = buildMatchRoute(itr.key(), itr.value()); - // key: topic - Set matchedTopics = matchedTopicFilters.get(matching.matcher.getFilterLevelList()); - if (matchedTopics == null) { - List seekTopicFilter = matching.matcher.getFilterLevelList(); - expansionSetItr.seek(seekTopicFilter); - if (expansionSetItr.isValid()) { - List topicFilterToMatch = expansionSetItr.key(); - if (topicFilterToMatch.equals(seekTopicFilter)) { - Set backingTopics = new HashSet<>(); - for (Set topicSet : expansionSetItr.value().values()) { - for (String topic : topicSet) { - MatchedRoutes matchResult = (MatchedRoutes) matchedRoutes.computeIfAbsent(topic, - k -> new MatchedRoutes(tenantId, k, eventCollector, maxPersistentFanoutCount, - maxGroupFanoutCount)); - switch (matching.type()) { - case Normal -> matchResult.addNormalMatching((NormalMatching) matching); - case Group -> matchResult.putGroupMatching((GroupMatching) matching); - default -> { - // never happen + try (ITopicFilterIterator expansionSetItr = + ThreadLocalTopicFilterIterator.get(topicTrieBuilder.build())) { + expansionSetItr.init(topicTrieBuilder.build()); + Map, Set> matchedTopicFilters = new HashMap<>(); + IKVIterator itr = rangeReader.iterator(); + // track seek + itr.seek(tenantBoundary.getStartKey()); + int probe = 0; + while (itr.isValid() && compare(itr.key(), tenantBoundary.getEndKey()) < 0) { + // track itr.key() + Matching matching = buildMatchRoute(itr.key(), itr.value()); + // key: topic + Set matchedTopics = matchedTopicFilters.get(matching.matcher.getFilterLevelList()); + if (matchedTopics == null) { + List seekTopicFilter = matching.matcher.getFilterLevelList(); + expansionSetItr.seek(seekTopicFilter); + if (expansionSetItr.isValid()) { + List topicFilterToMatch = expansionSetItr.key(); + if (topicFilterToMatch.equals(seekTopicFilter)) { + Set backingTopics = new HashSet<>(); + for (Set topicSet : expansionSetItr.value().values()) { + for (String topic : topicSet) { + MatchedRoutes matchResult = (MatchedRoutes) matchedRoutes.computeIfAbsent(topic, + k -> new MatchedRoutes(tenantId, k, eventCollector, maxPersistentFanoutCount, + maxGroupFanoutCount)); + switch (matching.type()) { + case Normal -> matchResult.addNormalMatching((NormalMatching) matching); + case Group -> matchResult.putGroupMatching((GroupMatching) matching); + default -> { + // never happen + } } + backingTopics.add(topic); } - backingTopics.add(topic); } - } - matchedTopicFilters.put(matching.matcher.getFilterLevelList(), backingTopics); - itr.next(); - probe = 0; - } else { - // next() is much cheaper than seek(), we probe following 20 entries - if (probe++ < 20) { - // probe next + matchedTopicFilters.put(matching.matcher.getFilterLevelList(), backingTopics); itr.next(); + probe = 0; } else { - // seek to match next topic filter - ByteString nextMatch = tenantRouteStartKey(tenantId, topicFilterToMatch); - itr.seek(nextMatch); + // next() is much cheaper than seek(), we probe following 20 entries + if (probe++ < 20) { + // probe next + itr.next(); + } else { + // seek to match next topic filter + ByteString nextMatch = tenantRouteStartKey(tenantId, topicFilterToMatch); + itr.seek(nextMatch); + } } + } else { + break; // no more topic filter to match, stop here } } else { - break; // no more topic filter to match, stop here - } - } else { - itr.next(); - for (String topic : matchedTopics) { - MatchedRoutes matchResult = (MatchedRoutes) matchedRoutes.computeIfAbsent(topic, - k -> new MatchedRoutes(tenantId, k, eventCollector, maxPersistentFanoutCount, - maxGroupFanoutCount)); - switch (matching.type()) { - case Normal -> matchResult.addNormalMatching((NormalMatching) matching); - case Group -> matchResult.putGroupMatching((GroupMatching) matching); - default -> { - // never happen + itr.next(); + for (String topic : matchedTopics) { + MatchedRoutes matchResult = (MatchedRoutes) matchedRoutes.computeIfAbsent(topic, + k -> new MatchedRoutes(tenantId, k, eventCollector, maxPersistentFanoutCount, + maxGroupFanoutCount)); + switch (matching.type()) { + case Normal -> matchResult.addNormalMatching((NormalMatching) matching); + case Group -> matchResult.putGroupMatching((GroupMatching) matching); + default -> { + // never happen + } } } } } + sample.stop(timer); + return matchedRoutes; } - sample.stop(timer); - return matchedRoutes; } } diff --git a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/KeyLayoutTest.java b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/KeyLayoutTest.java index c63d44d3a..fd6ae0e42 100644 --- a/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/KeyLayoutTest.java +++ b/bifromq-dist/bifromq-dist-worker/src/test/java/org/apache/bifromq/dist/worker/KeyLayoutTest.java @@ -23,12 +23,6 @@ import static org.apache.bifromq.util.TopicUtil.parse; import static org.testng.Assert.assertEquals; -import org.apache.bifromq.basekv.utils.BoundaryUtil; -import org.apache.bifromq.dist.rpc.proto.MatchRoute; -import org.apache.bifromq.dist.trie.TopicFilterIterator; -import org.apache.bifromq.dist.trie.TopicTrieNode; -import org.apache.bifromq.dist.worker.schema.KVSchemaUtil; -import org.apache.bifromq.util.TopicUtil; import com.google.protobuf.ByteString; import java.util.ArrayList; import java.util.LinkedHashMap; @@ -39,6 +33,12 @@ import java.util.TreeSet; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import org.apache.bifromq.basekv.utils.BoundaryUtil; +import org.apache.bifromq.dist.rpc.proto.MatchRoute; +import org.apache.bifromq.dist.trie.TopicFilterIterator; +import org.apache.bifromq.dist.trie.TopicTrieNode; +import org.apache.bifromq.dist.worker.schema.KVSchemaUtil; +import org.apache.bifromq.util.TopicUtil; import org.testng.annotations.Test; public class KeyLayoutTest { @@ -47,7 +47,8 @@ public void layout() { List topics = List.of("$", "b", "a/b", "b/c", "a/b/c", "b/c/d"); TopicTrieNode.Builder trieNodeBuilder = TopicTrieNode.builder(false); topics.forEach(t -> trieNodeBuilder.addTopic(parse(t, false), t)); - TopicFilterIterator itr = new TopicFilterIterator<>(trieNodeBuilder.build()); + TopicFilterIterator itr = new TopicFilterIterator<>(); + itr.init(trieNodeBuilder.build()); List generated = new ArrayList<>(); for (; itr.isValid(); itr.next()) { generated.add(fastJoin("/", itr.key()));