From 14ffbb06848ee97e5f386f10eea6ba88ac7ea46a Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 5 Jun 2024 23:47:57 +0800 Subject: [PATCH 01/30] the first commit --- .../pulsar/broker/service/TopicGCTest.java | 284 +++++++++++++++++- .../pulsar/client/impl/LookupServiceTest.java | 57 +++- .../pulsar/client/impl/LookupService.java | 1 + .../client/impl/MultiTopicsConsumerImpl.java | 173 ++++++----- .../impl/PatternMultiTopicsConsumerImpl.java | 201 +++++++++---- .../PatternMultiTopicsConsumerImplTest.java | 104 +++---- .../pulsar/common/lookup/GetTopicsResult.java | 24 ++ .../pulsar/common/naming/TopicName.java | 4 + 8 files changed, 647 insertions(+), 201 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 8fdf0723ea8d1..8cd3f82c53916 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -18,24 +18,34 @@ */ package org.apache.pulsar.broker.service; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; import java.util.Optional; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import lombok.EqualsAndHashCode; import lombok.extern.slf4j.Slf4j; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerConsumerBase; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.TopicMessageId; +import org.apache.pulsar.client.impl.ConsumerImpl; +import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.apache.pulsar.common.policies.data.InactiveTopicDeleteMode; import org.awaitility.Awaitility; +import org.awaitility.reflect.WhiteboxImpl; import org.testng.annotations.AfterClass; import org.testng.annotations.BeforeClass; +import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @Slf4j @@ -63,9 +73,32 @@ protected void doInitConf() throws Exception { this.conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); } - @Test - public void testCreateConsumerAfterOnePartDeleted() throws Exception { + private enum SubscribeTopicType { + MULTI_PARTITIONED_TOPIC, + REGEX_TOPIC; + } + + @DataProvider(name = "subscribeTopicTypes") + public Object[][] subTopicTypes() { + return new Object[][]{ + {SubscribeTopicType.MULTI_PARTITIONED_TOPIC}, + {SubscribeTopicType.REGEX_TOPIC} + }; + } + + private void setSubscribeTopic(ConsumerBuilder consumerBuilder, SubscribeTopicType subscribeTopicType, + String topicName, String topicPattern) { + if (subscribeTopicType.equals(SubscribeTopicType.MULTI_PARTITIONED_TOPIC)) { + consumerBuilder.topic(topicName); + } else { + consumerBuilder.topicsPattern(Pattern.compile(topicPattern)); + } + } + + @Test(dataProvider = "subscribeTopicTypes", timeOut = 300 * 1000) + public void testRecreateConsumerAfterOnePartGc(SubscribeTopicType subscribeTopicType) throws Exception { final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; final String partition0 = topic + "-partition-0"; final String partition1 = topic + "-partition-1"; final String subscription = "s1"; @@ -77,8 +110,12 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { .enableBatching(false).create(); Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) .enableBatching(false).create(); - org.apache.pulsar.client.api.Consumer consumer1 = pulsarClient.newConsumer(Schema.STRING).topic(topic) - .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); + ConsumerBuilder consumerBuilder1 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder1, subscribeTopicType, topic, topicPattern); + Consumer consumer1 = consumerBuilder1.subscribe(); // Make consume all messages for one topic, do not consume any messages for another one. producer0.send("1"); @@ -97,18 +134,247 @@ public void testCreateConsumerAfterOnePartDeleted() throws Exception { }); // Verify that the consumer subscribed with partitioned topic can be created successful. - Consumer consumerAllPartition = pulsarClient.newConsumer(Schema.STRING).topic(topic) - .subscriptionName(subscription).isAckReceiptEnabled(true).subscribe(); - Message msg = consumerAllPartition.receive(2, TimeUnit.SECONDS); + ConsumerBuilder consumerBuilder2 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder2, subscribeTopicType, topic, topicPattern); + Consumer consumer2 = consumerBuilder2.subscribe(); + Message msg = consumer2.receive(2, TimeUnit.SECONDS); + String receivedMsgValue = msg.getValue(); + log.info("received msg: {}", receivedMsgValue); + consumer2.acknowledge(msg); + + // cleanup. + consumer2.close(); + producer0.close(); + producer1.close(); + admin.topics().deletePartitionedTopic(topic); + } + + @Test(dataProvider = "subscribeTopicTypes", timeOut = 300 * 1000) + public void testAppendCreateConsumerAfterOnePartGc(SubscribeTopicType subscribeTopicType) throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + admin.topics().createSubscription(topic, subscription, MessageId.earliest); + + // create consumers and producers. + Producer producer0 = pulsarClient.newProducer(Schema.STRING).topic(partition0) + .enableBatching(false).create(); + Producer producer1 = pulsarClient.newProducer(Schema.STRING).topic(partition1) + .enableBatching(false).create(); + ConsumerBuilder consumerBuilder1 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder1, subscribeTopicType, topic, topicPattern); + Consumer consumer1 = consumerBuilder1.subscribe(); + + // Make consume all messages for one topic, do not consume any messages for another one. + producer0.send("partition-0-1"); + producer1.send("partition-1-1"); + producer1.send("partition-1-2"); + producer1.send("partition-1-4"); + admin.topics().skipAllMessages(partition0, subscription); + + // Wait for topic GC. + // Partition 0 will be deleted about 20s later, left 2min to avoid flaky. + producer0.close(); + Awaitility.await().atMost(2, TimeUnit.MINUTES).untilAsserted(() -> { + CompletableFuture> tp1 = pulsar.getBrokerService().getTopic(partition0, false); + CompletableFuture> tp2 = pulsar.getBrokerService().getTopic(partition1, false); + assertTrue(tp1 == null || !tp1.get().isPresent()); + assertTrue(tp2 != null && tp2.get().isPresent()); + }); + + // Verify that the messages under "partition-1" still can be ack. + for (int i = 0; i < 2; i++) { + Message msg = consumer1.receive(2, TimeUnit.SECONDS); + assertNotNull(msg, "Expected at least received 2 messages."); + log.info("===> received msg[{}]: {}", i, msg.getValue()); + TopicMessageId messageId = (TopicMessageId) msg.getMessageId(); + if (messageId.getOwnerTopic().equals(partition1)) { + consumer1.acknowledgeAsync(msg); + } + } + consumer1.close(); + + // Verify that the consumer subscribed with partitioned topic can be created successful. + ConsumerBuilder consumerBuilder2 = pulsarClient.newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared); + setSubscribeTopic(consumerBuilder2, subscribeTopicType, topic, topicPattern); + Consumer consumer2 = consumerBuilder2.subscribe(); + producer1.send("partition-1-5"); + Message msg = consumer2.receive(2, TimeUnit.SECONDS); assertNotNull(msg); String receivedMsgValue = msg.getValue(); log.info("received msg: {}", receivedMsgValue); - consumerAllPartition.acknowledge(msg); + consumer2.acknowledge(msg); // cleanup. - consumerAllPartition.close(); + consumer2.close(); producer0.close(); producer1.close(); admin.topics().deletePartitionedTopic(topic); } + + @Test(timeOut = 180 * 1000) + public void testAfterAllPartDeleted() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String partition2 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 3); + // Create consumer. + PatternMultiTopicsConsumerImpl c1 = (PatternMultiTopicsConsumerImpl) pulsarClient + .newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared) + .topicsPattern(Pattern.compile(topicPattern)).subscribe(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(consumers.size(), 3); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the first time. + admin.topics().delete(partition0, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(consumers.size(), 2); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the second time. + admin.topics().delete(partition1, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(consumers.size(), 1); + assertTrue(consumers.containsKey(partition2)); + }); + // Delete partitions the third time. + admin.topics().delete(partition1, true); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 0); + assertEquals(consumers.size(), 0); + }); + + // cleanup. + c1.close(); + admin.topics().deletePartitionedTopic(topic); + } + + @Test(timeOut = 180 * 1000) + public void testExpandPartitions() throws Exception { + final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); + final String topicPattern = "persistent://public/default/tp.*"; + final String partition0 = topic + "-partition-0"; + final String partition1 = topic + "-partition-1"; + final String subscription = "s1"; + admin.topics().createPartitionedTopic(topic, 2); + // Delete partitions. + admin.topics().delete(partition0, true); + admin.topics().delete(partition1, true); + // Create consumer. + PatternMultiTopicsConsumerImpl c1 = (PatternMultiTopicsConsumerImpl) pulsarClient + .newConsumer(Schema.STRING) + .subscriptionName(subscription) + .isAckReceiptEnabled(true) + .subscriptionType(SubscriptionType.Shared) + .topicsPattern(Pattern.compile(topicPattern)).subscribe(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 0); + assertEquals(consumers.size(), 0); + }); + // Trigger partitions creation. + pulsarClient.newConsumer(Schema.STRING).subscriptionName(subscription) + .subscriptionType(SubscriptionType.Shared).topic(topic).subscribe().close(); + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(consumers.size(), 2); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + }); + // Expand partitions the first time. + admin.topics().updatePartitionedTopic(topic, 3); + final String partition2 = topic + "-partition-2"; + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(consumers.size(), 3); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + }); + // Expand partitions the second time. + admin.topics().updatePartitionedTopic(topic, 4); + final String partition3 = topic + "-partition-3"; + // Check subscriptions. + Awaitility.await().untilAsserted(() -> { + ConcurrentHashMap> consumers + = WhiteboxImpl.getInternalState(c1, "consumers"); + ConcurrentHashMap partitionedTopics + = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); + assertEquals(partitionedTopics.size(), 1); + assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(consumers.size(), 4); + assertTrue(consumers.containsKey(partition0)); + assertTrue(consumers.containsKey(partition1)); + assertTrue(consumers.containsKey(partition2)); + assertTrue(consumers.containsKey(partition3)); + }); + + // cleanup. + c1.close(); + admin.topics().deletePartitionedTopic(topic); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java index 59cb7ae03d0e3..dec5b89963f65 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java @@ -108,8 +108,8 @@ public void testGetTopicsOfGetTopicsResult(boolean isUsingHttpLookup) throws Exc // Verify the new method "GetTopicsResult.nonPartitionedOrPartitionTopics" works as expected. Collection nonPartitionedOrPartitionTopics = lookupService.getTopicsUnderNamespace(NamespaceName.get("public/default"), - Mode.PERSISTENT, "public/default/.*", null).join() - .getNonPartitionedOrPartitionTopics(); + Mode.PERSISTENT, "public/default/.*", null).join() + .getNonPartitionedOrPartitionTopics(); assertTrue(nonPartitionedOrPartitionTopics.contains(nonPartitionedTopic)); assertFalse(nonPartitionedOrPartitionTopics.contains(partitionedTopic)); assertFalse(nonPartitionedOrPartitionTopics.contains(nonPersistentTopic)); @@ -125,4 +125,57 @@ public void testGetTopicsOfGetTopicsResult(boolean isUsingHttpLookup) throws Exc admin.topics().delete(nonPartitionedTopic, false); } +// @Test(dataProvider = "isUsingHttpLookup") +// public void testGetExistsPartitions(boolean isUsingHttpLookup) throws Exception { +// LookupService lookupService = getLookupService(isUsingHttpLookup); +// String nonPartitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); +// admin.topics().createNonPartitionedTopic(nonPartitionedTopic); +// String partitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); +// admin.topics().createPartitionedTopic(partitionedTopic, 3); +// String nonPersistentTopic = BrokerTestUtil.newUniqueName("non-persistent://public/default/tp"); +// +// assertEquals(lookupService.getExistsPartitions(nonPartitionedTopic).join(), Collections.emptyList()); +// assertEquals(lookupService.getExistsPartitions(partitionedTopic).join(), Arrays.asList(0, 1, 2)); +// try { +// lookupService.getExistsPartitions(nonPersistentTopic).join(); +// fail("Expected an error"); +// } catch (Exception ex) { +// assertTrue(ex.getMessage().contains("not support")); +// } +// +// // Cleanup. +// admin.topics().deletePartitionedTopic(partitionedTopic, false); +// admin.topics().delete(nonPartitionedTopic, false); +// } +// +// @Test(dataProvider = "isUsingHttpLookup") +// public void testGetExistsPartitionsIfDisabledBrokerFilter(boolean isUsingHttpLookup) throws Exception { +// cleanup(); +// enableBrokerSideSubscriptionPatternEvaluation = false; +// setup(); +// +// LookupService lookupService = getLookupService(isUsingHttpLookup); +// String nonPartitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); +// admin.topics().createNonPartitionedTopic(nonPartitionedTopic); +// String partitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); +// admin.topics().createPartitionedTopic(partitionedTopic, 3); +// String nonPersistentTopic = BrokerTestUtil.newUniqueName("non-persistent://public/default/tp"); +// +// assertEquals(lookupService.getExistsPartitions(nonPartitionedTopic).join(), Collections.emptyList()); +// assertEquals(lookupService.getExistsPartitions(partitionedTopic).join(), Arrays.asList(0, 1, 2)); +// try { +// lookupService.getExistsPartitions(nonPersistentTopic).join(); +// fail("Expected an error"); +// } catch (Exception ex) { +// assertTrue(ex.getMessage().contains("not support")); +// } +// +// // Cleanup. +// admin.topics().deletePartitionedTopic(partitionedTopic, false); +// admin.topics().delete(nonPartitionedTopic, false); +// // Reset broker config. +// cleanup(); +// enableBrokerSideSubscriptionPatternEvaluation = true; +// setup(); +// } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index ccd1f6b23f2f3..d5ef544a41844 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -124,4 +124,5 @@ CompletableFuture getPartitionedTopicMetadata(TopicNam */ CompletableFuture getTopicsUnderNamespace(NamespaceName namespace, Mode mode, String topicPattern, String topicsHash); + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 8047e05351ac1..f2c00ad2781bf 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -47,6 +47,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import java.util.function.Predicate; +import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; import javax.annotation.Nullable; @@ -68,6 +70,7 @@ import org.apache.pulsar.client.util.ConsumerName; import org.apache.pulsar.client.util.ExecutorProvider; import org.apache.pulsar.common.api.proto.CommandAck.AckType; +import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.util.CompletableFutureCancellationHandler; @@ -81,14 +84,14 @@ public class MultiTopicsConsumerImpl extends ConsumerBase { public static final String DUMMY_TOPIC_NAME_PREFIX = "MultiTopicsConsumer-"; // Map , when get do ACK, consumer will by find by topic name - private final ConcurrentHashMap> consumers; + protected final ConcurrentHashMap> consumers; // Map , store partition number for each topic protected final ConcurrentHashMap partitionedTopics; // Queue of partition consumers on which we have stopped calling receiveAsync() because the // shared incoming queue was full - private final ConcurrentLinkedQueue> pausedConsumers; + protected final ConcurrentLinkedQueue> pausedConsumers; // sum of topicPartitions, simple topic has 1, partitioned topic equals to partition number. AtomicInteger allTopicPartitionsNumber; @@ -935,6 +938,7 @@ private void removeTopic(String topic) { * @param topicName topic name without the partition suffix. */ public CompletableFuture subscribeAsync(String topicName, boolean createTopicIfDoesNotExist) { + log.info("===> topicName: {}", topicName); TopicName topicNameInstance = getTopicName(topicName); if (topicNameInstance == null) { return FutureUtil.failedFuture( @@ -1043,11 +1047,12 @@ private void doSubscribeTopicPartitions(Schema schema, String topicName, int numPartitions, boolean createIfDoesNotExist) { + log.info("===> doSubscribeTopicPartitions {}: {}", numPartitions, topicName); if (log.isDebugEnabled()) { log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); } - List>> futureList; + CompletableFuture subscribeAllPartitionsFuture; if (numPartitions != PartitionedTopicMetadata.NON_PARTITIONED) { // Below condition is true if subscribeAsync() has been invoked second time with same // topicName before the first invocation had reached this point. @@ -1067,30 +1072,45 @@ private void doSubscribeTopicPartitions(Schema schema, ConsumerConfigurationData configurationData = getInternalConsumerConfig(); configurationData.setReceiverQueueSize(receiverQueueSize); - futureList = IntStream - .range(0, numPartitions) - .mapToObj( - partitionIndex -> { - String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); - CompletableFuture> subFuture = new CompletableFuture<>(); - configurationData.setStartPaused(paused); - ConsumerImpl newConsumer = createInternalConsumer(configurationData, partitionName, - partitionIndex, subFuture, createIfDoesNotExist, schema); - synchronized (pauseMutex) { - if (paused) { - newConsumer.pause(); - } else { - newConsumer.resume(); - } - consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + CompletableFuture> partitionsFuture; + if (!createIfDoesNotExist) { + partitionsFuture = getExistsPartitions(topic); + } else { + partitionsFuture = CompletableFuture.completedFuture(IntStream.range(0, numPartitions) + .mapToObj(i -> Integer.valueOf(i)) + .collect(Collectors.toList())); + } + subscribeAllPartitionsFuture = partitionsFuture.thenCompose(partitions -> { + log.info("===> partitions: {}", partitions); + if (partitions.isEmpty()) { + partitionedTopics.remove(topicName, numPartitions); + return CompletableFuture.completedFuture(null); + } + List>> subscribeList = new ArrayList<>(); + for (int partitionIndex : partitions) { + String partitionName = TopicName.get(topicName).getPartition(partitionIndex).toString(); + CompletableFuture> subFuture = new CompletableFuture<>(); + configurationData.setStartPaused(paused); + ConsumerImpl newConsumer = createInternalConsumer(configurationData, partitionName, + partitionIndex, subFuture, createIfDoesNotExist, schema); + synchronized (pauseMutex) { + if (paused) { + newConsumer.pause(); + } else { + newConsumer.resume(); } - return subFuture; - }) - .collect(Collectors.toList()); + consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + } + log.info("===> consumers: {}", consumers.keySet()); + subscribeList.add(subFuture); + } + return FutureUtil.waitForAll(subscribeList); + }); } else { allTopicPartitionsNumber.incrementAndGet(); - CompletableFuture> subFuture = new CompletableFuture<>(); + CompletableFuture> subscribeFuture = new CompletableFuture<>(); + subscribeAllPartitionsFuture = subscribeFuture.thenAccept(__ -> {}); synchronized (pauseMutex) { consumers.compute(topicName, (key, existingValue) -> { @@ -1104,7 +1124,7 @@ private void doSubscribeTopicPartitions(Schema schema, } else { internalConfig.setStartPaused(paused); ConsumerImpl newConsumer = createInternalConsumer(internalConfig, topicName, - -1, subFuture, createIfDoesNotExist, schema); + -1, subscribeFuture, createIfDoesNotExist, schema); if (paused) { newConsumer.pause(); } else { @@ -1114,11 +1134,10 @@ private void doSubscribeTopicPartitions(Schema schema, } }); } - futureList = Collections.singletonList(subFuture); + } - FutureUtil.waitForAll(futureList) - .thenAccept(finalFuture -> { + subscribeAllPartitionsFuture.thenAccept(finalFuture -> { if (allTopicPartitionsNumber.get() > getCurrentReceiverQueueSize()) { setCurrentReceiverQueueSize(allTopicPartitionsNumber.get()); } @@ -1255,59 +1274,6 @@ public CompletableFuture unsubscribeAsync(String topicName) { return unsubscribeFuture; } - /*** - * Remove a consumer for a topic. - * @param topicName topic name contains the partition suffix. - */ - public CompletableFuture removeConsumerAsync(String topicName) { - checkArgument(TopicName.isValid(topicName), "Invalid topic name:" + topicName); - - if (getState() == State.Closing || getState() == State.Closed) { - return FutureUtil.failedFuture( - new PulsarClientException.AlreadyClosedException("Topics Consumer was already closed")); - } - - CompletableFuture unsubscribeFuture = new CompletableFuture<>(); - String topicPartName = TopicName.get(topicName).getPartitionedTopicName(); - - - List> consumersToClose = consumers.values().stream() - .filter(consumer -> { - String consumerTopicName = consumer.getTopic(); - return TopicName.get(consumerTopicName).getPartitionedTopicName().equals(topicPartName); - }).collect(Collectors.toList()); - - List> futureList = consumersToClose.stream() - .map(ConsumerImpl::closeAsync).collect(Collectors.toList()); - - FutureUtil.waitForAll(futureList) - .whenComplete((r, ex) -> { - if (ex == null) { - consumersToClose.forEach(consumer1 -> { - consumers.remove(consumer1.getTopic()); - pausedConsumers.remove(consumer1); - allTopicPartitionsNumber.decrementAndGet(); - }); - - removeTopic(topicName); - if (unAckedMessageTracker instanceof UnAckedTopicMessageTracker) { - ((UnAckedTopicMessageTracker) unAckedMessageTracker).removeTopicMessages(topicName); - } - - unsubscribeFuture.complete(null); - log.info("[{}] [{}] [{}] Removed Topics Consumer, allTopicPartitionsNumber: {}", - topicName, subscription, consumerName, allTopicPartitionsNumber); - } else { - unsubscribeFuture.completeExceptionally(ex); - setState(State.Failed); - log.error("[{}] [{}] [{}] Could not remove Topics Consumer", - topicName, subscription, consumerName, ex.getCause()); - } - }); - - return unsubscribeFuture; - } - // get topics name public List getPartitionedTopics() { @@ -1573,4 +1539,51 @@ protected void setCurrentReceiverQueueSize(int newSize) { CURRENT_RECEIVER_QUEUE_SIZE_UPDATER.set(this, newSize); resumeReceivingFromPausedConsumersIfNeeded(); } + + /** + * Get the exists partitions of a partitioned topic, the result does not contain the partitions which has not been + * created yet(in other words, the partitions that do not exist in the response of "pulsar-admin topics list"). + * @return sorted partitions list if it is a partitioned topic; @return an empty list if it is a non-partitioned + * topic. + */ + private CompletableFuture> getExistsPartitions(String topic) { + TopicName topicName = TopicName.get(topic); + if (!topicName.isPersistent()) { + return FutureUtil.failedFuture(new IllegalArgumentException("The API LookupService.getExistsPartitions does" + + " not support non-persistent topic yet.")); + } + return client.getLookup().getTopicsUnderNamespace(topicName.getNamespaceObject(), + topicName.isPersistent() ? CommandGetTopicsOfNamespace.Mode.PERSISTENT : CommandGetTopicsOfNamespace.Mode.NON_PERSISTENT, + "^" + topicName.getPartitionedTopicName() + "$", + null).thenApply(getTopicsResult -> { + if (getTopicsResult.getNonPartitionedOrPartitionTopics() == null + || getTopicsResult.getNonPartitionedOrPartitionTopics().isEmpty()) { + return Collections.emptyList(); + } + // If broker version is less than "2.11.x", it does not support broker-side pattern check, so append + // a client-side pattern check. + // If lookup service is typed HttpLookupService, the HTTP API does not support broker-side pattern + // check yet, so append a client-side pattern check. + Predicate clientSideFilter; + if (getTopicsResult.isFiltered()) { + clientSideFilter = __ -> true; + } else { + clientSideFilter = + tp -> Pattern.compile(TopicName.getPartitionPattern(topic)).matcher(tp).matches(); + } + ArrayList list = new ArrayList<>(getTopicsResult.getNonPartitionedOrPartitionTopics().size()); + for (String partition : getTopicsResult.getNonPartitionedOrPartitionTopics()) { + int partitionIndex = TopicName.get(partition).getPartitionIndex(); + if (partitionIndex < 0) { + // It is not a partition. + continue; + } + if (clientSideFilter.test(partition)) { + list.add(partitionIndex); + } + } + Collections.sort(list); + return list; + }); + } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index ffca79dfa4342..3dc479b491f28 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -28,12 +28,12 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.conf.ConsumerConfigurationData; @@ -69,6 +69,31 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl= 2.11. + * 3. A scheduled task {@link #recheckPatternTimeout} will call {@link #recheckTopicsChange()}, this scenario only + * exists in the version < 2.11. + * 4. The topics change events will trigger a {@link #recheckTopicsChange()}. + * + * So when you are using a release >= 2.11, there are three scenarios: [1, 2, 4]. + * - The events related scenario-4 will be executed at the same thread, because it will be triggered by + * {@link ClientCnx}; + * - The events related scenario-2 will be executed after switching {@link ClientCnx}, so this event will happen + * after the events related to scenario-4, this guarantees the variable {@link #updateSubscriptionFuture} will be + * updated before the new {@link ClientCnx} was set. + * - If the update subscription task related scenario-2 fails, it will schedule a retry task. Since it checks all + * topics, so all things will be fine if it is later than any events related scenario-4. + * + * When you are using a release < 2.11, there are three scenarios: [1, 3, 4]. Since it checks all topics, + * so all things will be fine. + */ + private volatile CompletableFuture updateSubscriptionFuture; + /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ @@ -83,6 +108,7 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, ConsumerInterceptors interceptors) { super(client, conf, executorProvider, subscribeFuture, schema, interceptors, false /* createTopicIfDoesNotExist */); + updateSubscriptionFuture = subscribeFuture.thenAccept(__ -> {}); this.topicsPattern = topicsPattern; this.topicsHash = topicsHash; this.subscriptionMode = subscriptionMode; @@ -164,55 +190,55 @@ private CompletableFuture recheckTopicsChange() { final int epoch = recheckPatternEpoch.incrementAndGet(); return client.getLookup().getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, topicsHash) .thenCompose(getTopicsResult -> { - // If "recheckTopicsChange" has been called more than one times, only make the last one take affects. - // Use "synchronized (recheckPatternTaskBackoff)" instead of + // If "recheckTopicsChange" has been called more than one times, only make the last one take + // affects. Use "synchronized (recheckPatternTaskBackoff)" instead of // `synchronized(PatternMultiTopicsConsumerImpl.this)` to avoid locking in a wider range. synchronized (recheckPatternTaskBackoff) { if (recheckPatternEpoch.get() > epoch) { return CompletableFuture.completedFuture(null); } if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", - namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), + log.debug( + "Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", + namespaceName, getTopicsResult.getTopics().size(), + getTopicsResult.getTopicsHash(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> - log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); + log.debug("Get topics under namespace {}, topic: {}", namespaceName, + topicName)); } - final List oldTopics = new ArrayList<>(getPartitionedTopics()); - for (String partition : getPartitions()) { - TopicName topicName = TopicName.get(partition); - if (!topicName.isPartitioned() || !oldTopics.contains(topicName.getPartitionedTopicName())) { - oldTopics.add(partition); - } - } + final List oldTopics = new ArrayList<>(getPartitions()); return updateSubscriptions(topicsPattern, this::setTopicsHash, getTopicsResult, topicsChangeListener, oldTopics); } }); } - static CompletableFuture updateSubscriptions(Pattern topicsPattern, + private CompletableFuture updateSubscriptions(Pattern topicsPattern, java.util.function.Consumer topicsHashSetter, GetTopicsResult getTopicsResult, TopicsChangedListener topicsChangedListener, List oldTopics) { - topicsHashSetter.accept(getTopicsResult.getTopicsHash()); - if (!getTopicsResult.isChanged()) { - return CompletableFuture.completedFuture(null); - } + updateSubscriptionFuture = updateSubscriptionFuture.whenComplete((__, ex) -> {}).thenCompose(ignore -> { + topicsHashSetter.accept(getTopicsResult.getTopicsHash()); + if (!getTopicsResult.isChanged()) { + return CompletableFuture.completedFuture(null); + } - List newTopics; - if (getTopicsResult.isFiltered()) { - newTopics = getTopicsResult.getTopics(); - } else { - newTopics = TopicList.filterTopics(getTopicsResult.getTopics(), topicsPattern); - } + List newTopics; + if (getTopicsResult.isFiltered()) { + newTopics = getTopicsResult.getNonPartitionedOrPartitionTopics(); + } else { + newTopics = getTopicsResult.filterTopics(topicsPattern).getNonPartitionedOrPartitionTopics(); + } - final List> listenersCallback = new ArrayList<>(2); - listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); - listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); - return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); + final List> listenersCallback = new ArrayList<>(2); + listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); + listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); + return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); + }); + return updateSubscriptionFuture; } public Pattern getPattern() { @@ -247,23 +273,50 @@ private class PatternTopicsChangedListener implements TopicsChangedListener { */ @Override public CompletableFuture onTopicsRemoved(Collection removedTopics) { - CompletableFuture removeFuture = new CompletableFuture<>(); - + log.info("===> removedTopics: {}", removedTopics); if (removedTopics.isEmpty()) { - removeFuture.complete(null); - return removeFuture; + return CompletableFuture.completedFuture(null); + } + + // Unsubscribe and remove consumers in memory. + List> unsubscribeList = new ArrayList<>(removedTopics.size()); + Set partialRemoved = new HashSet<>(removedTopics.size()); + for (String tp : removedTopics) { + ConsumerImpl consumer = consumers.get(tp); + if (consumer != null) { + CompletableFuture unsubscribeFuture = new CompletableFuture<>(); + consumer.closeAsync().whenComplete((__, ex) -> { + if (ex != null) { + log.error("[{}] Failed to unsubscribe from topics: {}", tp, ex); + unsubscribeFuture.completeExceptionally(ex); + } else { + consumers.remove(tp, consumer); + unsubscribeFuture.complete(null); + } + }); + unsubscribeList.add(unsubscribeFuture); + partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); + } } - List> futures = Lists.newArrayListWithExpectedSize(partitionedTopics.size()); - removedTopics.stream().forEach(topic -> futures.add(removeConsumerAsync(topic))); - FutureUtil.waitForAll(futures) - .thenAccept(finalFuture -> removeFuture.complete(null)) - .exceptionally(ex -> { - log.warn("[{}] Failed to unsubscribe from topics: {}", topic, ex.getMessage()); - removeFuture.completeExceptionally(ex); - return null; + // Remove partitioned topics in memory. + return FutureUtil.waitForAll(unsubscribeList).whenComplete((__, ex) -> { + for (String groupedTopicRemoved : partialRemoved) { + Integer partitions = partitionedTopics.get(groupedTopicRemoved); + if (partitions != null) { + boolean allPartitionsHasBeenRemoved = true; + for (int i = 0; i < partitions; i++) { + if (consumers.containsKey(TopicName.get(groupedTopicRemoved).getPartition(i))) { + allPartitionsHasBeenRemoved = false; + break; + } + } + if (allPartitionsHasBeenRemoved) { + partitionedTopics.remove(groupedTopicRemoved); + } + } + } }); - return removeFuture; } /** @@ -271,29 +324,61 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) */ @Override public CompletableFuture onTopicsAdded(Collection addedTopics) { - CompletableFuture addFuture = new CompletableFuture<>(); - + log.info("===> addedTopics: {}", addedTopics); if (addedTopics.isEmpty()) { - addFuture.complete(null); - return addFuture; + return CompletableFuture.completedFuture(null); } + updateSubscriptionFuture.join(); + log.info("===> addedTopics 2: {}", addedTopics); - Set addTopicPartitionedName = addedTopics.stream() - .map(addTopicName -> TopicName.get(addTopicName).getPartitionedTopicName()) - .collect(Collectors.toSet()); + List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); - List> futures = Lists.newArrayListWithExpectedSize(partitionedTopics.size()); - addTopicPartitionedName.forEach(partitionedTopic -> futures.add( - subscribeAsync(partitionedTopic, - false /* createTopicIfDoesNotExist */))); - FutureUtil.waitForAll(futures) - .thenAccept(finalFuture -> addFuture.complete(null)) - .exceptionally(ex -> { - log.warn("[{}] Failed to subscribe to topics: {}", topic, ex.getMessage()); - addFuture.completeExceptionally(ex); + /** + * Three cases: + * 1. Expand partitions. + * 2. Non-partitioned topic, but has been subscribing. + * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. + */ + Set groupedTopics = new HashSet<>(); + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + groupedTopics.add(topicName.getPartitionedTopicName()); + } + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + // Case 1: Expand partitions. + if (partitionedTopics.contains(topicName.getPartitionedTopicName())) { + if (consumers.containsKey(tp)) { + continue; + } else { + if (topicName.getPartitionIndex() + 1 > + partitionedTopics.get(topicName.getPartitionedTopicName())){ + partitionedTopics.put(topicName.getPartitionedTopicName(), + topicName.getPartitionIndex() + 1); + } + CompletableFuture consumerFuture = subscribeAsync(tp, 0); + consumerFuture.exceptionally(ex -> { + log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); + return null; + }); + futures.add(consumerFuture); + } + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(tp)) { + // Case-2: Non-partitioned topic, but has been subscribing. + groupedTopics.remove(topicName.getPartitionedTopicName()); + } + } + // Case 3: Non-partitioned topic or Partitioned topic, but has not been subscribing. + for (String partitionedTopic: groupedTopics) { + CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); + consumerFuture.exceptionally(ex -> { + log.warn("[{}] Failed to subscribe to topics: {}", partitionedTopic, ex); return null; }); - return addFuture; + futures.add(consumerFuture); + } + return FutureUtil.waitForAll(futures); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java index 116a69b63e4ec..d8a35f119ac57 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java @@ -50,58 +50,58 @@ public void setUp() { } - @Test - public void testChangedUnfilteredResponse() { - PatternMultiTopicsConsumerImpl.updateSubscriptions( - Pattern.compile("tenant/my-ns/name-.*"), - mockTopicsHashSetter, - new GetTopicsResult(Arrays.asList( - "persistent://tenant/my-ns/name-1", - "persistent://tenant/my-ns/name-2", - "persistent://tenant/my-ns/non-matching"), - null, false, true), - mockListener, - Collections.emptyList()); - verify(mockListener).onTopicsAdded(Sets.newHashSet( - "persistent://tenant/my-ns/name-1", - "persistent://tenant/my-ns/name-2")); - verify(mockListener).onTopicsRemoved(Collections.emptySet()); - verify(mockTopicsHashSetter).accept(null); - } +// @Test +// public void testChangedUnfilteredResponse() { +// PatternMultiTopicsConsumerImpl.updateSubscriptions( +// Pattern.compile("tenant/my-ns/name-.*"), +// mockTopicsHashSetter, +// new GetTopicsResult(Arrays.asList( +// "persistent://tenant/my-ns/name-1", +// "persistent://tenant/my-ns/name-2", +// "persistent://tenant/my-ns/non-matching"), +// null, false, true), +// mockListener, +// Collections.emptyList()); +// verify(mockListener).onTopicsAdded(Sets.newHashSet( +// "persistent://tenant/my-ns/name-1", +// "persistent://tenant/my-ns/name-2")); +// verify(mockListener).onTopicsRemoved(Collections.emptySet()); +// verify(mockTopicsHashSetter).accept(null); +// } - @Test - public void testChangedFilteredResponse() { - PatternMultiTopicsConsumerImpl.updateSubscriptions( - Pattern.compile("tenant/my-ns/name-.*"), - mockTopicsHashSetter, - new GetTopicsResult(Arrays.asList( - "persistent://tenant/my-ns/name-0", - "persistent://tenant/my-ns/name-1", - "persistent://tenant/my-ns/name-2"), - "TOPICS_HASH", true, true), - mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); - verify(mockListener).onTopicsAdded(Sets.newHashSet( - "persistent://tenant/my-ns/name-1", - "persistent://tenant/my-ns/name-2")); - verify(mockListener).onTopicsRemoved(Collections.emptySet()); - verify(mockTopicsHashSetter).accept("TOPICS_HASH"); - } +// @Test +// public void testChangedFilteredResponse() { +// PatternMultiTopicsConsumerImpl.updateSubscriptions( +// Pattern.compile("tenant/my-ns/name-.*"), +// mockTopicsHashSetter, +// new GetTopicsResult(Arrays.asList( +// "persistent://tenant/my-ns/name-0", +// "persistent://tenant/my-ns/name-1", +// "persistent://tenant/my-ns/name-2"), +// "TOPICS_HASH", true, true), +// mockListener, +// Arrays.asList("persistent://tenant/my-ns/name-0")); +// verify(mockListener).onTopicsAdded(Sets.newHashSet( +// "persistent://tenant/my-ns/name-1", +// "persistent://tenant/my-ns/name-2")); +// verify(mockListener).onTopicsRemoved(Collections.emptySet()); +// verify(mockTopicsHashSetter).accept("TOPICS_HASH"); +// } - @Test - public void testUnchangedResponse() { - PatternMultiTopicsConsumerImpl.updateSubscriptions( - Pattern.compile("tenant/my-ns/name-.*"), - mockTopicsHashSetter, - new GetTopicsResult(Arrays.asList( - "persistent://tenant/my-ns/name-0", - "persistent://tenant/my-ns/name-1", - "persistent://tenant/my-ns/name-2"), - "TOPICS_HASH", true, false), - mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); - verify(mockListener, never()).onTopicsAdded(any()); - verify(mockListener, never()).onTopicsRemoved(any()); - verify(mockTopicsHashSetter).accept("TOPICS_HASH"); - } +// @Test +// public void testUnchangedResponse() { +// PatternMultiTopicsConsumerImpl.updateSubscriptions( +// Pattern.compile("tenant/my-ns/name-.*"), +// mockTopicsHashSetter, +// new GetTopicsResult(Arrays.asList( +// "persistent://tenant/my-ns/name-0", +// "persistent://tenant/my-ns/name-1", +// "persistent://tenant/my-ns/name-2"), +// "TOPICS_HASH", true, false), +// mockListener, +// Arrays.asList("persistent://tenant/my-ns/name-0")); +// verify(mockListener, never()).onTopicsAdded(any()); +// verify(mockListener, never()).onTopicsRemoved(any()); +// verify(mockTopicsHashSetter).accept("TOPICS_HASH"); +// } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java index 80f16e6c36717..26a295264fcae 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/lookup/GetTopicsResult.java @@ -18,9 +18,12 @@ */ package org.apache.pulsar.common.lookup; +import com.google.re2j.Pattern; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; +import java.util.Set; import lombok.Getter; import lombok.ToString; import org.apache.pulsar.common.api.proto.CommandGetTopicsOfNamespace; @@ -119,4 +122,25 @@ public List getTopics() { return topics; } } + + public GetTopicsResult filterTopics(Pattern topicsPattern) { + List topicsFiltered = TopicList.filterTopics(getTopics(), topicsPattern); + // If nothing changed. + if (topicsFiltered.equals(getTopics())) { + GetTopicsResult newObj = new GetTopicsResult(nonPartitionedOrPartitionTopics, null, true, true); + newObj.topics = topics; + return newObj; + } + // Filtered some topics. + Set topicsFilteredSet = new HashSet<>(topicsFiltered); + List newTps = new ArrayList<>(); + for (String tp: nonPartitionedOrPartitionTopics) { + if (topicsFilteredSet.contains(TopicName.get(tp).getPartitionedTopicName())) { + newTps.add(tp); + } + } + GetTopicsResult newObj = new GetTopicsResult(newTps, null, true, true); + newObj.topics = topicsFiltered; + return newObj; + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index e051e01495dbe..77ab2cc9a70f3 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -102,6 +102,10 @@ public static boolean isValid(String topic) { } } + public static String getPartitionPattern(String topic) { + return "^" + get(topic).getPartitionedTopicName().toString() + "-partition-[0-9]+$"; + } + @SuppressFBWarnings("DCN_NULLPOINTER_EXCEPTION") private TopicName(String completeTopicName) { try { From a7a752fbb516251d85ca631307b08a5cb8f74164 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 5 Jun 2024 23:48:47 +0800 Subject: [PATCH 02/30] the first commit --- .../apache/pulsar/client/impl/MultiTopicsConsumerImpl.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index f2c00ad2781bf..72d7f427e00d3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1073,12 +1073,12 @@ private void doSubscribeTopicPartitions(Schema schema, configurationData.setReceiverQueueSize(receiverQueueSize); CompletableFuture> partitionsFuture; - if (!createIfDoesNotExist) { - partitionsFuture = getExistsPartitions(topic); - } else { + if (createIfDoesNotExist) { partitionsFuture = CompletableFuture.completedFuture(IntStream.range(0, numPartitions) .mapToObj(i -> Integer.valueOf(i)) .collect(Collectors.toList())); + } else { + partitionsFuture = getExistsPartitions(topic); } subscribeAllPartitionsFuture = partitionsFuture.thenCompose(partitions -> { log.info("===> partitions: {}", partitions); From ea051d3f69478eca01ba08bdb498b67d3e3bc468 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 6 Jun 2024 03:14:33 +0800 Subject: [PATCH 03/30] the second commit --- .../pulsar/broker/service/TopicGCTest.java | 28 +-- .../client/impl/MultiTopicsConsumerImpl.java | 26 ++- .../impl/PatternMultiTopicsConsumerImpl.java | 212 +++++++++--------- .../PatternMultiTopicsConsumerImplTest.java | 104 ++++----- .../apache/pulsar/common/util/FutureUtil.java | 3 + 5 files changed, 190 insertions(+), 183 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 8cd3f82c53916..4d1f1114ecc8f 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -21,6 +21,7 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; +import java.time.Duration; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -68,9 +69,10 @@ protected void cleanup() throws Exception { @EqualsAndHashCode.Include protected void doInitConf() throws Exception { super.doInitConf(); - this.conf.setBrokerDeleteInactiveTopicsEnabled(true); - this.conf.setBrokerDeleteInactiveTopicsMode(InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); - this.conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); + conf.setBrokerDeleteInactiveTopicsEnabled(true); + conf.setBrokerDeleteInactiveTopicsMode( + InactiveTopicDeleteMode.delete_when_subscriptions_caught_up); + conf.setBrokerDeleteInactiveTopicsFrequencySeconds(10); } private enum SubscribeTopicType { @@ -195,7 +197,7 @@ public void testAppendCreateConsumerAfterOnePartGc(SubscribeTopicType subscribeT for (int i = 0; i < 2; i++) { Message msg = consumer1.receive(2, TimeUnit.SECONDS); assertNotNull(msg, "Expected at least received 2 messages."); - log.info("===> received msg[{}]: {}", i, msg.getValue()); + log.info("received msg[{}]: {}", i, msg.getValue()); TopicMessageId messageId = (TopicMessageId) msg.getMessageId(); if (messageId.getOwnerTopic().equals(partition1)) { consumer1.acknowledgeAsync(msg); @@ -225,12 +227,12 @@ public void testAppendCreateConsumerAfterOnePartGc(SubscribeTopicType subscribeT } @Test(timeOut = 180 * 1000) - public void testAfterAllPartDeleted() throws Exception { + public void testPhasePartDeletion() throws Exception { final String topic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); final String topicPattern = "persistent://public/default/tp.*"; final String partition0 = topic + "-partition-0"; final String partition1 = topic + "-partition-1"; - final String partition2 = topic + "-partition-1"; + final String partition2 = topic + "-partition-2"; final String subscription = "s1"; admin.topics().createPartitionedTopic(topic, 3); // Create consumer. @@ -247,7 +249,7 @@ public void testAfterAllPartDeleted() throws Exception { ConcurrentHashMap partitionedTopics = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); assertEquals(partitionedTopics.size(), 1); - assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(partitionedTopics.get(topic), 3); assertEquals(consumers.size(), 3); assertTrue(consumers.containsKey(partition0)); assertTrue(consumers.containsKey(partition1)); @@ -262,7 +264,7 @@ public void testAfterAllPartDeleted() throws Exception { ConcurrentHashMap partitionedTopics = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); assertEquals(partitionedTopics.size(), 1); - assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(partitionedTopics.get(topic), 3); assertEquals(consumers.size(), 2); assertTrue(consumers.containsKey(partition1)); assertTrue(consumers.containsKey(partition2)); @@ -276,12 +278,12 @@ public void testAfterAllPartDeleted() throws Exception { ConcurrentHashMap partitionedTopics = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); assertEquals(partitionedTopics.size(), 1); - assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(partitionedTopics.get(topic), 3); assertEquals(consumers.size(), 1); assertTrue(consumers.containsKey(partition2)); }); // Delete partitions the third time. - admin.topics().delete(partition1, true); + admin.topics().delete(partition2, true); // Check subscriptions. Awaitility.await().untilAsserted(() -> { ConcurrentHashMap> consumers @@ -334,7 +336,7 @@ public void testExpandPartitions() throws Exception { ConcurrentHashMap partitionedTopics = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); assertEquals(partitionedTopics.size(), 1); - assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(partitionedTopics.get(topic), 2); assertEquals(consumers.size(), 2); assertTrue(consumers.containsKey(partition0)); assertTrue(consumers.containsKey(partition1)); @@ -349,7 +351,7 @@ public void testExpandPartitions() throws Exception { ConcurrentHashMap partitionedTopics = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); assertEquals(partitionedTopics.size(), 1); - assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(partitionedTopics.get(topic), 3); assertEquals(consumers.size(), 3); assertTrue(consumers.containsKey(partition0)); assertTrue(consumers.containsKey(partition1)); @@ -365,7 +367,7 @@ public void testExpandPartitions() throws Exception { ConcurrentHashMap partitionedTopics = WhiteboxImpl.getInternalState(c1, "partitionedTopics"); assertEquals(partitionedTopics.size(), 1); - assertTrue(partitionedTopics.containsKey(topic)); + assertEquals(partitionedTopics.get(topic), 4); assertEquals(consumers.size(), 4); assertTrue(consumers.containsKey(partition0)); assertTrue(consumers.containsKey(partition1)); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 72d7f427e00d3..984ae56337ee1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -938,7 +938,6 @@ private void removeTopic(String topic) { * @param topicName topic name without the partition suffix. */ public CompletableFuture subscribeAsync(String topicName, boolean createTopicIfDoesNotExist) { - log.info("===> topicName: {}", topicName); TopicName topicNameInstance = getTopicName(topicName); if (topicNameInstance == null) { return FutureUtil.failedFuture( @@ -1013,8 +1012,12 @@ CompletableFuture subscribeAsync(String topicName, int numberPartitions) { new PulsarClientException.AlreadyClosedException("Topic name not valid")); } String fullTopicName = topicNameInstance.toString(); - if (consumers.containsKey(fullTopicName) - || partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { + if (consumers.containsKey(fullTopicName)) { + return FutureUtil.failedFuture( + new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); + } + if (!topicNameInstance.isPartitioned() + && partitionedTopics.containsKey(topicNameInstance.getPartitionedTopicName())) { return FutureUtil.failedFuture( new PulsarClientException.AlreadyClosedException("Already subscribed to " + topicName)); } @@ -1047,7 +1050,6 @@ private void doSubscribeTopicPartitions(Schema schema, String topicName, int numPartitions, boolean createIfDoesNotExist) { - log.info("===> doSubscribeTopicPartitions {}: {}", numPartitions, topicName); if (log.isDebugEnabled()) { log.debug("Subscribe to topic {} metadata.partitions: {}", topicName, numPartitions); } @@ -1073,15 +1075,14 @@ private void doSubscribeTopicPartitions(Schema schema, configurationData.setReceiverQueueSize(receiverQueueSize); CompletableFuture> partitionsFuture; - if (createIfDoesNotExist) { + if (createIfDoesNotExist || !TopicName.get(topicName).isPersistent()) { partitionsFuture = CompletableFuture.completedFuture(IntStream.range(0, numPartitions) .mapToObj(i -> Integer.valueOf(i)) .collect(Collectors.toList())); } else { - partitionsFuture = getExistsPartitions(topic); + partitionsFuture = getExistsPartitions(topicName.toString()); } subscribeAllPartitionsFuture = partitionsFuture.thenCompose(partitions -> { - log.info("===> partitions: {}", partitions); if (partitions.isEmpty()) { partitionedTopics.remove(topicName, numPartitions); return CompletableFuture.completedFuture(null); @@ -1101,7 +1102,6 @@ private void doSubscribeTopicPartitions(Schema schema, } consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); } - log.info("===> consumers: {}", consumers.keySet()); subscribeList.add(subFuture); } return FutureUtil.waitForAll(subscribeList); @@ -1158,7 +1158,9 @@ private void doSubscribeTopicPartitions(Schema schema, return; }) .exceptionally(ex -> { - handleSubscribeOneTopicError(topicName, ex, subscribeResult); + log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, + ex.getMessage()); + subscribeResult.completeExceptionally(ex); return null; }); } @@ -1549,11 +1551,11 @@ protected void setCurrentReceiverQueueSize(int newSize) { private CompletableFuture> getExistsPartitions(String topic) { TopicName topicName = TopicName.get(topic); if (!topicName.isPersistent()) { - return FutureUtil.failedFuture(new IllegalArgumentException("The API LookupService.getExistsPartitions does" - + " not support non-persistent topic yet.")); + return FutureUtil.failedFuture(new IllegalArgumentException("The method getExistsPartitions" + + " does not support non-persistent topic yet.")); } return client.getLookup().getTopicsUnderNamespace(topicName.getNamespaceObject(), - topicName.isPersistent() ? CommandGetTopicsOfNamespace.Mode.PERSISTENT : CommandGetTopicsOfNamespace.Mode.NON_PERSISTENT, + CommandGetTopicsOfNamespace.Mode.PERSISTENT, "^" + topicName.getPartitionedTopicName() + "$", null).thenApply(getTopicsResult -> { if (getTopicsResult.getNonPartitionedOrPartitionTopics() == null diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 3dc479b491f28..56ac411aa428c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -84,15 +84,15 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl updateSubscriptionFuture; + private volatile CompletableFuture updateFuture; /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). @@ -108,7 +108,7 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, ConsumerInterceptors interceptors) { super(client, conf, executorProvider, subscribeFuture, schema, interceptors, false /* createTopicIfDoesNotExist */); - updateSubscriptionFuture = subscribeFuture.thenAccept(__ -> {}); + updateFuture = subscribeFuture.thenAccept(__ -> {}).exceptionally(ex -> null); this.topicsPattern = topicsPattern; this.topicsHash = topicsHash; this.subscriptionMode = subscriptionMode; @@ -215,30 +215,27 @@ private CompletableFuture recheckTopicsChange() { }); } - private CompletableFuture updateSubscriptions(Pattern topicsPattern, + static CompletableFuture updateSubscriptions(Pattern topicsPattern, java.util.function.Consumer topicsHashSetter, GetTopicsResult getTopicsResult, TopicsChangedListener topicsChangedListener, List oldTopics) { - updateSubscriptionFuture = updateSubscriptionFuture.whenComplete((__, ex) -> {}).thenCompose(ignore -> { - topicsHashSetter.accept(getTopicsResult.getTopicsHash()); - if (!getTopicsResult.isChanged()) { - return CompletableFuture.completedFuture(null); - } + topicsHashSetter.accept(getTopicsResult.getTopicsHash()); + if (!getTopicsResult.isChanged()) { + return CompletableFuture.completedFuture(null); + } - List newTopics; - if (getTopicsResult.isFiltered()) { - newTopics = getTopicsResult.getNonPartitionedOrPartitionTopics(); - } else { - newTopics = getTopicsResult.filterTopics(topicsPattern).getNonPartitionedOrPartitionTopics(); - } + List newTopics; + if (getTopicsResult.isFiltered()) { + newTopics = getTopicsResult.getNonPartitionedOrPartitionTopics(); + } else { + newTopics = getTopicsResult.filterTopics(topicsPattern).getNonPartitionedOrPartitionTopics(); + } - final List> listenersCallback = new ArrayList<>(2); - listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); - listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); - return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); - }); - return updateSubscriptionFuture; + final List> listenersCallback = new ArrayList<>(2); + listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); + listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); + return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); } public Pattern getPattern() { @@ -273,50 +270,53 @@ private class PatternTopicsChangedListener implements TopicsChangedListener { */ @Override public CompletableFuture onTopicsRemoved(Collection removedTopics) { - log.info("===> removedTopics: {}", removedTopics); if (removedTopics.isEmpty()) { return CompletableFuture.completedFuture(null); } - // Unsubscribe and remove consumers in memory. - List> unsubscribeList = new ArrayList<>(removedTopics.size()); - Set partialRemoved = new HashSet<>(removedTopics.size()); - for (String tp : removedTopics) { - ConsumerImpl consumer = consumers.get(tp); - if (consumer != null) { - CompletableFuture unsubscribeFuture = new CompletableFuture<>(); - consumer.closeAsync().whenComplete((__, ex) -> { - if (ex != null) { - log.error("[{}] Failed to unsubscribe from topics: {}", tp, ex); - unsubscribeFuture.completeExceptionally(ex); - } else { - consumers.remove(tp, consumer); - unsubscribeFuture.complete(null); - } - }); - unsubscribeList.add(unsubscribeFuture); - partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); + CompletableFuture newFuture = updateFuture.thenCompose(ignore -> { + // Unsubscribe and remove consumers in memory. + List> unsubscribeList = new ArrayList<>(removedTopics.size()); + Set partialRemoved = new HashSet<>(removedTopics.size()); + for (String tp : removedTopics) { + ConsumerImpl consumer = consumers.get(tp); + if (consumer != null) { + CompletableFuture unsubscribeFuture = new CompletableFuture<>(); + consumer.closeAsync().whenComplete((__, ex) -> { + if (ex != null) { + log.error("[{}] Failed to unsubscribe from topics: {}", tp, ex); + unsubscribeFuture.completeExceptionally(ex); + } else { + consumers.remove(tp, consumer); + unsubscribeFuture.complete(null); + } + }); + unsubscribeList.add(unsubscribeFuture); + partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); + } } - } - // Remove partitioned topics in memory. - return FutureUtil.waitForAll(unsubscribeList).whenComplete((__, ex) -> { - for (String groupedTopicRemoved : partialRemoved) { - Integer partitions = partitionedTopics.get(groupedTopicRemoved); - if (partitions != null) { - boolean allPartitionsHasBeenRemoved = true; - for (int i = 0; i < partitions; i++) { - if (consumers.containsKey(TopicName.get(groupedTopicRemoved).getPartition(i))) { - allPartitionsHasBeenRemoved = false; - break; + // Remove partitioned topics in memory. + return FutureUtil.waitForAll(unsubscribeList).whenComplete((__, ex) -> { + for (String groupedTopicRemoved : partialRemoved) { + Integer partitions = partitionedTopics.get(groupedTopicRemoved); + if (partitions != null) { + boolean allPartitionsHasBeenRemoved = true; + for (int i = 0; i < partitions; i++) { + if (consumers.containsKey( + TopicName.get(groupedTopicRemoved).getPartition(i).toString())) { + allPartitionsHasBeenRemoved = false; + break; + } + } + if (allPartitionsHasBeenRemoved) { + partitionedTopics.remove(groupedTopicRemoved, partitions); } - } - if (allPartitionsHasBeenRemoved) { - partitionedTopics.remove(groupedTopicRemoved); } } - } + }); }); + return updateFuture = newFuture.exceptionally(ex -> null); } /** @@ -324,61 +324,61 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) */ @Override public CompletableFuture onTopicsAdded(Collection addedTopics) { - log.info("===> addedTopics: {}", addedTopics); if (addedTopics.isEmpty()) { return CompletableFuture.completedFuture(null); } - updateSubscriptionFuture.join(); - log.info("===> addedTopics 2: {}", addedTopics); - - List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); - - /** - * Three cases: - * 1. Expand partitions. - * 2. Non-partitioned topic, but has been subscribing. - * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. - */ - Set groupedTopics = new HashSet<>(); - for (String tp : addedTopics) { - TopicName topicName = TopicName.get(tp); - groupedTopics.add(topicName.getPartitionedTopicName()); - } - for (String tp : addedTopics) { - TopicName topicName = TopicName.get(tp); - // Case 1: Expand partitions. - if (partitionedTopics.contains(topicName.getPartitionedTopicName())) { - if (consumers.containsKey(tp)) { - continue; - } else { - if (topicName.getPartitionIndex() + 1 > - partitionedTopics.get(topicName.getPartitionedTopicName())){ - partitionedTopics.put(topicName.getPartitionedTopicName(), - topicName.getPartitionIndex() + 1); + CompletableFuture newFuture = updateFuture.thenCompose(ignore -> { + List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); + /** + * Three cases: + * 1. Expand partitions. + * 2. Non-partitioned topic, but has been subscribing. + * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. + */ + Set groupedTopics = new HashSet<>(); + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + groupedTopics.add(topicName.getPartitionedTopicName()); + } + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + // Case 1: Expand partitions. + if (partitionedTopics.containsKey(topicName.getPartitionedTopicName())) { + if (consumers.containsKey(tp.toString())) { + continue; + } else { + if (topicName.getPartitionIndex() + 1 > + partitionedTopics.get(topicName.getPartitionedTopicName())) { + partitionedTopics.put(topicName.getPartitionedTopicName(), + topicName.getPartitionIndex() + 1); + } + CompletableFuture consumerFuture = subscribeAsync(tp, 0); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); + } + }); + futures.add(consumerFuture); } - CompletableFuture consumerFuture = subscribeAsync(tp, 0); - consumerFuture.exceptionally(ex -> { - log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); - return null; - }); - futures.add(consumerFuture); + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(tp.toString())) { + // Case-2: Non-partitioned topic, but has been subscribing. + groupedTopics.remove(topicName.getPartitionedTopicName()); } - groupedTopics.remove(topicName.getPartitionedTopicName()); - } else if (consumers.containsKey(tp)) { - // Case-2: Non-partitioned topic, but has been subscribing. - groupedTopics.remove(topicName.getPartitionedTopicName()); } - } - // Case 3: Non-partitioned topic or Partitioned topic, but has not been subscribing. - for (String partitionedTopic: groupedTopics) { - CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); - consumerFuture.exceptionally(ex -> { - log.warn("[{}] Failed to subscribe to topics: {}", partitionedTopic, ex); - return null; - }); - futures.add(consumerFuture); - } - return FutureUtil.waitForAll(futures); + // Case 3: Non-partitioned topic or Partitioned topic, but has not been subscribing. + for (String partitionedTopic : groupedTopics) { + CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("[{}] Failed to subscribe to topics: {}", partitionedTopic, ex); + } + }); + futures.add(consumerFuture); + } + return FutureUtil.waitForAll(futures); + }); + return updateFuture = newFuture.exceptionally(ex -> null); } } @@ -398,7 +398,7 @@ public CompletableFuture closeAsync() { closeFutures.add(watcher.closeAsync()); } } - closeFutures.add(super.closeAsync()); + closeFutures.add(updateFuture.thenCompose(__ -> super.closeAsync())); return FutureUtil.waitForAll(closeFutures); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java index d8a35f119ac57..116a69b63e4ec 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java @@ -50,58 +50,58 @@ public void setUp() { } -// @Test -// public void testChangedUnfilteredResponse() { -// PatternMultiTopicsConsumerImpl.updateSubscriptions( -// Pattern.compile("tenant/my-ns/name-.*"), -// mockTopicsHashSetter, -// new GetTopicsResult(Arrays.asList( -// "persistent://tenant/my-ns/name-1", -// "persistent://tenant/my-ns/name-2", -// "persistent://tenant/my-ns/non-matching"), -// null, false, true), -// mockListener, -// Collections.emptyList()); -// verify(mockListener).onTopicsAdded(Sets.newHashSet( -// "persistent://tenant/my-ns/name-1", -// "persistent://tenant/my-ns/name-2")); -// verify(mockListener).onTopicsRemoved(Collections.emptySet()); -// verify(mockTopicsHashSetter).accept(null); -// } + @Test + public void testChangedUnfilteredResponse() { + PatternMultiTopicsConsumerImpl.updateSubscriptions( + Pattern.compile("tenant/my-ns/name-.*"), + mockTopicsHashSetter, + new GetTopicsResult(Arrays.asList( + "persistent://tenant/my-ns/name-1", + "persistent://tenant/my-ns/name-2", + "persistent://tenant/my-ns/non-matching"), + null, false, true), + mockListener, + Collections.emptyList()); + verify(mockListener).onTopicsAdded(Sets.newHashSet( + "persistent://tenant/my-ns/name-1", + "persistent://tenant/my-ns/name-2")); + verify(mockListener).onTopicsRemoved(Collections.emptySet()); + verify(mockTopicsHashSetter).accept(null); + } -// @Test -// public void testChangedFilteredResponse() { -// PatternMultiTopicsConsumerImpl.updateSubscriptions( -// Pattern.compile("tenant/my-ns/name-.*"), -// mockTopicsHashSetter, -// new GetTopicsResult(Arrays.asList( -// "persistent://tenant/my-ns/name-0", -// "persistent://tenant/my-ns/name-1", -// "persistent://tenant/my-ns/name-2"), -// "TOPICS_HASH", true, true), -// mockListener, -// Arrays.asList("persistent://tenant/my-ns/name-0")); -// verify(mockListener).onTopicsAdded(Sets.newHashSet( -// "persistent://tenant/my-ns/name-1", -// "persistent://tenant/my-ns/name-2")); -// verify(mockListener).onTopicsRemoved(Collections.emptySet()); -// verify(mockTopicsHashSetter).accept("TOPICS_HASH"); -// } + @Test + public void testChangedFilteredResponse() { + PatternMultiTopicsConsumerImpl.updateSubscriptions( + Pattern.compile("tenant/my-ns/name-.*"), + mockTopicsHashSetter, + new GetTopicsResult(Arrays.asList( + "persistent://tenant/my-ns/name-0", + "persistent://tenant/my-ns/name-1", + "persistent://tenant/my-ns/name-2"), + "TOPICS_HASH", true, true), + mockListener, + Arrays.asList("persistent://tenant/my-ns/name-0")); + verify(mockListener).onTopicsAdded(Sets.newHashSet( + "persistent://tenant/my-ns/name-1", + "persistent://tenant/my-ns/name-2")); + verify(mockListener).onTopicsRemoved(Collections.emptySet()); + verify(mockTopicsHashSetter).accept("TOPICS_HASH"); + } -// @Test -// public void testUnchangedResponse() { -// PatternMultiTopicsConsumerImpl.updateSubscriptions( -// Pattern.compile("tenant/my-ns/name-.*"), -// mockTopicsHashSetter, -// new GetTopicsResult(Arrays.asList( -// "persistent://tenant/my-ns/name-0", -// "persistent://tenant/my-ns/name-1", -// "persistent://tenant/my-ns/name-2"), -// "TOPICS_HASH", true, false), -// mockListener, -// Arrays.asList("persistent://tenant/my-ns/name-0")); -// verify(mockListener, never()).onTopicsAdded(any()); -// verify(mockListener, never()).onTopicsRemoved(any()); -// verify(mockTopicsHashSetter).accept("TOPICS_HASH"); -// } + @Test + public void testUnchangedResponse() { + PatternMultiTopicsConsumerImpl.updateSubscriptions( + Pattern.compile("tenant/my-ns/name-.*"), + mockTopicsHashSetter, + new GetTopicsResult(Arrays.asList( + "persistent://tenant/my-ns/name-0", + "persistent://tenant/my-ns/name-1", + "persistent://tenant/my-ns/name-2"), + "TOPICS_HASH", true, false), + mockListener, + Arrays.asList("persistent://tenant/my-ns/name-0")); + verify(mockListener, never()).onTopicsAdded(any()); + verify(mockListener, never()).onTopicsRemoved(any()); + verify(mockTopicsHashSetter).accept("TOPICS_HASH"); + } } diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java index 0628d494af3af..454eee0f966c5 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/util/FutureUtil.java @@ -54,6 +54,9 @@ public class FutureUtil { * @return a new CompletableFuture that is completed when all of the given CompletableFutures complete */ public static CompletableFuture waitForAll(Collection> futures) { + if (futures == null || futures.isEmpty()) { + return CompletableFuture.completedFuture(null); + } return CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])); } From e04ec2260886f89041c98dc0eee4c88a673be473 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 6 Jun 2024 03:19:26 +0800 Subject: [PATCH 04/30] - --- .../pulsar/broker/service/TopicGCTest.java | 1 - .../pulsar/client/impl/LookupServiceTest.java | 53 ------------------- .../pulsar/client/impl/LookupService.java | 1 - .../impl/PatternMultiTopicsConsumerImpl.java | 13 ++--- 4 files changed, 5 insertions(+), 63 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java index 4d1f1114ecc8f..172bd3702e129 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/TopicGCTest.java @@ -21,7 +21,6 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; -import java.time.Duration; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java index dec5b89963f65..5d5066db7aa86 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java @@ -125,57 +125,4 @@ public void testGetTopicsOfGetTopicsResult(boolean isUsingHttpLookup) throws Exc admin.topics().delete(nonPartitionedTopic, false); } -// @Test(dataProvider = "isUsingHttpLookup") -// public void testGetExistsPartitions(boolean isUsingHttpLookup) throws Exception { -// LookupService lookupService = getLookupService(isUsingHttpLookup); -// String nonPartitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); -// admin.topics().createNonPartitionedTopic(nonPartitionedTopic); -// String partitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); -// admin.topics().createPartitionedTopic(partitionedTopic, 3); -// String nonPersistentTopic = BrokerTestUtil.newUniqueName("non-persistent://public/default/tp"); -// -// assertEquals(lookupService.getExistsPartitions(nonPartitionedTopic).join(), Collections.emptyList()); -// assertEquals(lookupService.getExistsPartitions(partitionedTopic).join(), Arrays.asList(0, 1, 2)); -// try { -// lookupService.getExistsPartitions(nonPersistentTopic).join(); -// fail("Expected an error"); -// } catch (Exception ex) { -// assertTrue(ex.getMessage().contains("not support")); -// } -// -// // Cleanup. -// admin.topics().deletePartitionedTopic(partitionedTopic, false); -// admin.topics().delete(nonPartitionedTopic, false); -// } -// -// @Test(dataProvider = "isUsingHttpLookup") -// public void testGetExistsPartitionsIfDisabledBrokerFilter(boolean isUsingHttpLookup) throws Exception { -// cleanup(); -// enableBrokerSideSubscriptionPatternEvaluation = false; -// setup(); -// -// LookupService lookupService = getLookupService(isUsingHttpLookup); -// String nonPartitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); -// admin.topics().createNonPartitionedTopic(nonPartitionedTopic); -// String partitionedTopic = BrokerTestUtil.newUniqueName("persistent://public/default/tp"); -// admin.topics().createPartitionedTopic(partitionedTopic, 3); -// String nonPersistentTopic = BrokerTestUtil.newUniqueName("non-persistent://public/default/tp"); -// -// assertEquals(lookupService.getExistsPartitions(nonPartitionedTopic).join(), Collections.emptyList()); -// assertEquals(lookupService.getExistsPartitions(partitionedTopic).join(), Arrays.asList(0, 1, 2)); -// try { -// lookupService.getExistsPartitions(nonPersistentTopic).join(); -// fail("Expected an error"); -// } catch (Exception ex) { -// assertTrue(ex.getMessage().contains("not support")); -// } -// -// // Cleanup. -// admin.topics().deletePartitionedTopic(partitionedTopic, false); -// admin.topics().delete(nonPartitionedTopic, false); -// // Reset broker config. -// cleanup(); -// enableBrokerSideSubscriptionPatternEvaluation = true; -// setup(); -// } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index d5ef544a41844..ccd1f6b23f2f3 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -124,5 +124,4 @@ CompletableFuture getPartitionedTopicMetadata(TopicNam */ CompletableFuture getTopicsUnderNamespace(NamespaceName namespace, Mode mode, String topicPattern, String topicsHash); - } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 56ac411aa428c..7bd106f4b42c0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -190,22 +190,19 @@ private CompletableFuture recheckTopicsChange() { final int epoch = recheckPatternEpoch.incrementAndGet(); return client.getLookup().getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, topicsHash) .thenCompose(getTopicsResult -> { - // If "recheckTopicsChange" has been called more than one times, only make the last one take - // affects. Use "synchronized (recheckPatternTaskBackoff)" instead of + // If "recheckTopicsChange" has been called more than one times, only make the last one take affects. + // Use "synchronized (recheckPatternTaskBackoff)" instead of // `synchronized(PatternMultiTopicsConsumerImpl.this)` to avoid locking in a wider range. synchronized (recheckPatternTaskBackoff) { if (recheckPatternEpoch.get() > epoch) { return CompletableFuture.completedFuture(null); } if (log.isDebugEnabled()) { - log.debug( - "Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", - namespaceName, getTopicsResult.getTopics().size(), - getTopicsResult.getTopicsHash(), + log.debug("Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", + namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> - log.debug("Get topics under namespace {}, topic: {}", namespaceName, - topicName)); + log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); } final List oldTopics = new ArrayList<>(getPartitions()); From a72b0488a32ff154e00955f38bc7043243dfa8d7 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 14 Jun 2024 19:16:05 +0800 Subject: [PATCH 05/30] support topic name contains characters --- .../apache/pulsar/client/impl/MultiTopicsConsumerImpl.java | 2 +- .../java/org/apache/pulsar/common/naming/TopicName.java | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 984ae56337ee1..383b22db0f587 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1556,7 +1556,7 @@ private CompletableFuture> getExistsPartitions(String topic) { } return client.getLookup().getTopicsUnderNamespace(topicName.getNamespaceObject(), CommandGetTopicsOfNamespace.Mode.PERSISTENT, - "^" + topicName.getPartitionedTopicName() + "$", + TopicName.getPattern(topicName.getPartitionedTopicName()), null).thenApply(getTopicsResult -> { if (getTopicsResult.getNonPartitionedOrPartitionTopics() == null || getTopicsResult.getNonPartitionedOrPartitionTopics().isEmpty()) { diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index 77ab2cc9a70f3..2298a8d06a880 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -28,6 +28,7 @@ import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.util.Codec; @@ -103,7 +104,11 @@ public static boolean isValid(String topic) { } public static String getPartitionPattern(String topic) { - return "^" + get(topic).getPartitionedTopicName().toString() + "-partition-[0-9]+$"; + return "^" + Pattern.quote(get(topic).getPartitionedTopicName().toString()) + "-partition-[0-9]+$"; + } + + public static String getPattern(String topic) { + return "^" + Pattern.quote(get(topic).getPartitionedTopicName().toString()) + "$"; } @SuppressFBWarnings("DCN_NULLPOINTER_EXCEPTION") From bed9bd42a3a4032a299c03944f2c963102a47110 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 14 Jun 2024 19:19:41 +0800 Subject: [PATCH 06/30] close the orphan consumer --- .../pulsar/client/impl/MultiTopicsConsumerImpl.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 383b22db0f587..259e8b744a709 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1100,7 +1100,14 @@ private void doSubscribeTopicPartitions(Schema schema, } else { newConsumer.resume(); } - consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + Consumer originalValue = consumers.putIfAbsent(newConsumer.getTopic(), newConsumer); + if (originalValue != null) { + newConsumer.closeAsync().exceptionally(ex -> { + log.error("[{}] [{}] Failed to close the orphan consumer", + partitionName, subscription, ex); + return null; + }); + } } subscribeList.add(subFuture); } From 78c9a24dc3119ca1234301dcb33347539c9bf82f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 18 Jun 2024 01:19:38 +0800 Subject: [PATCH 07/30] address comment --- .../impl/PatternConsumerUpdateQueue.java | 187 +++++++++++++++ .../impl/PatternMultiTopicsConsumerImpl.java | 213 ++++++++---------- .../pulsar/client/impl/TopicListWatcher.java | 16 +- .../impl/PatternConsumerUpdateQueueTest.java | 137 +++++++++++ .../client/impl/TopicListWatcherTest.java | 11 +- 5 files changed, 428 insertions(+), 136 deletions(-) create mode 100644 pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java create mode 100644 pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java new file mode 100644 index 0000000000000..ac9f07d5a1a69 --- /dev/null +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -0,0 +1,187 @@ +package org.apache.pulsar.client.impl; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.tuple.Pair; + +/** + * Used to make all tasks that will modify subscriptions will be executed one by one, and skip the unnecessary updating. + * + * So far, four three scenarios that will modify subscriptions: + * 1. When start pattern consumer. + * 2. After topic list watcher reconnected, it will call {@link PatternMultiTopicsConsumerImpl#recheckTopicsChange()}. + * this scenario only exists in the version >= 2.11 (both client-version and broker version are >= 2.11). + * 3. A scheduled task will call {@link PatternMultiTopicsConsumerImpl#recheckTopicsChange()}, this scenario only + * exists in the version < 2.11. + * 4. The topics change events will trigger a + * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsRemoved(Collection)} or + * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsAdded(Collection)}. + * + * When you are using a release >= 2.11, there are three scenarios: [1, 2, 4]. + * When you are using a release < 2.11, there is only one scenario: [3] and all the event will run in the same thread. + */ +public class PatternConsumerUpdateQueue { + + private static final Pair> RECHECK_OP = + Pair.of(UpdateSubscriptionType.RECHECK, null); + + private final LinkedBlockingQueue>> pendingTasks; + + private final PatternMultiTopicsConsumerImpl patternConsumer; + + private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + + /** + * Whether there is a task is in progress, this variable is used to confirm whether a next-task triggering is + * needed. + */ + private CompletableFuture taskInProgress = null; + + /** + * Whether there is a recheck task in queue. + * - Since recheck task will do all changes, it can be used to compress multiple tasks to one. + * - To avoid skipping the newest changes, once the recheck task is starting to work, this variable will be set + * to "false". + */ + private boolean recheckTaskInQueue = false; + + private long lastRecheckTaskStartingTimestamp = 0; + + private boolean closed; + + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer) { + this(patternConsumer, patternConsumer.topicsChangeListener); + } + + /** This constructor is only for test. **/ + @VisibleForTesting + public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer, + PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener) { + this.patternConsumer = patternConsumer; + this.topicsChangeListener = topicsChangeListener; + this.pendingTasks = new LinkedBlockingQueue<>(); + } + + synchronized void appendTopicsAddedOp(Collection topics) { + if (topics == null || topics.isEmpty()) { + return; + } + doAppend(Pair.of(UpdateSubscriptionType.TOPICS_ADDED, topics)); + } + + synchronized void appendTopicsRemovedOp(Collection topics) { + if (topics == null || topics.isEmpty()) { + return; + } + doAppend(Pair.of(UpdateSubscriptionType.TOPICS_REMOVED, topics)); + } + + synchronized void appendRecheckOp() { + doAppend(RECHECK_OP); + } + + synchronized void doAppend(Pair> task) { + // Once there is a recheck task in queue, it means other tasks can be skipped. + if (recheckTaskInQueue) { + return; + } + + // Once there are too many tasks in queue, compress them as a recheck task. + if (pendingTasks.size() >= 30 && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + appendRecheckOp(); + return; + } + + pendingTasks.add(task); + if (task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + recheckTaskInQueue = true; + } + + // If no task is in-progress, trigger a task execution. + if (taskInProgress == null) { + triggerNextTask(); + } + } + + synchronized void triggerNextTask() { + if (closed) { + return; + } + + Pair> task = pendingTasks.poll(); + + // No pending task. + if (task == null) { + taskInProgress = null; + return; + } + + // If there is a recheck task in queue, skip others and only call the recheck task. + if (recheckTaskInQueue && !task.getLeft().equals(UpdateSubscriptionType.RECHECK)) { + triggerNextTask(); + return; + } + + // Execute pending task. + CompletableFuture newTaskFuture = null; + switch (task.getLeft()) { + case TOPICS_ADDED: { + newTaskFuture = topicsChangeListener.onTopicsAdded(task.getRight()); + break; + } + case TOPICS_REMOVED: { + newTaskFuture = topicsChangeListener.onTopicsRemoved(task.getRight()); + break; + } + case RECHECK: { + recheckTaskInQueue = false; + lastRecheckTaskStartingTimestamp = System.currentTimeMillis(); + newTaskFuture = patternConsumer.recheckTopicsChange(); + } + } + + // Trigger next pending task. + taskInProgress = newTaskFuture; + newTaskFuture.thenAccept(ignore -> { + triggerNextTask(); + }).exceptionally(ex -> { + /** + * Once a updating fails, trigger a delayed new recheck task to guarantee all things is correct. + * - Skip if there is already a recheck task in queue. + * - Skip if the last recheck task has been executed after the current time. + */ + // TODO log. + // Skip if there is already a recheck task in queue. + synchronized (PatternConsumerUpdateQueue.this) { + if (recheckTaskInQueue || PatternConsumerUpdateQueue.this.closed) { + return null; + } + } + // Skip if the last recheck task has been executed after the current time. + long failedTime = System.currentTimeMillis(); + patternConsumer.client.timer().newTimeout(timeout -> { + if (lastRecheckTaskStartingTimestamp <= failedTime) { + appendRecheckOp(); + } + }, 30, TimeUnit.SECONDS); + return null; + }); + } + + public synchronized CompletableFuture cancelAllAndWaitForTheRunningTask() { + this.closed = true; + return taskInProgress.thenAccept(__ -> {}).exceptionally(ex -> null); + } + + private enum UpdateSubscriptionType { + /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ + TOPICS_ADDED, + /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ + TOPICS_REMOVED, + /** A fully check for pattern consumer. **/ + RECHECK; + } +} diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 7bd106f4b42c0..68859e709528c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -32,6 +32,8 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.client.api.Consumer; @@ -51,7 +53,7 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl implements TimerTask { private final Pattern topicsPattern; - private final TopicsChangedListener topicsChangeListener; + final TopicsChangedListener topicsChangeListener; private final Mode subscriptionMode; private final CompletableFuture watcherFuture = new CompletableFuture<>(); protected NamespaceName namespaceName; @@ -69,30 +71,7 @@ public class PatternMultiTopicsConsumerImpl extends MultiTopicsConsumerImpl= 2.11. - * 3. A scheduled task {@link #recheckPatternTimeout} will call {@link #recheckTopicsChange()}, this scenario only - * exists in the version < 2.11. - * 4. The topics change events will trigger a {@link #recheckTopicsChange()}. - * - * So when you are using a release >= 2.11, there are three scenarios: [1, 2, 4]. - * - The events related scenario-4 will be executed at the same thread, because it will be triggered by - * {@link ClientCnx}; - * - The events related scenario-2 will be executed after switching {@link ClientCnx}, so this event will happen - * after the events related to scenario-4, this guarantees the variable {@link #updateFuture} will be - * updated before the new {@link ClientCnx} was set. - * - If the update subscription task related scenario-2 fails, it will schedule a retry task. Since it checks all - * topics, so all things will be fine if it is later than any events related scenario-4. - * - * When you are using a release < 2.11, there are three scenarios: [1, 3, 4]. Since it checks all topics, - * so all things will be fine. TODO compare and set. - */ - private volatile CompletableFuture updateFuture; + private PatternConsumerUpdateQueue updateTaskQueue; /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). @@ -108,10 +87,10 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, ConsumerInterceptors interceptors) { super(client, conf, executorProvider, subscribeFuture, schema, interceptors, false /* createTopicIfDoesNotExist */); - updateFuture = subscribeFuture.thenAccept(__ -> {}).exceptionally(ex -> null); this.topicsPattern = topicsPattern; this.topicsHash = topicsHash; this.subscriptionMode = subscriptionMode; + this.updateTaskQueue = new PatternConsumerUpdateQueue(this); this.recheckPatternTaskBackoff = new BackoffBuilder() .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) @@ -128,7 +107,7 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, .newTimeout(this, Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); if (subscriptionMode == Mode.PERSISTENT) { long watcherId = client.newTopicListWatcherId(); - new TopicListWatcher(topicsChangeListener, client, topicsPattern, watcherId, + new TopicListWatcher(updateTaskQueue, client, topicsPattern, watcherId, namespaceName, topicsHash, watcherFuture, () -> recheckTopicsChangeAfterReconnect()); watcherFuture .thenAccept(__ -> recheckPatternTimeout.cancel()) @@ -174,18 +153,10 @@ public void run(Timeout timeout) throws Exception { if (timeout.isCancelled()) { return; } - recheckTopicsChange().exceptionally(ex -> { - log.warn("[{}] Failed to recheck topics change: {}", topic, ex.getMessage()); - return null; - }).thenAccept(__ -> { - // schedule the next re-check task - this.recheckPatternTimeout = client.timer() - .newTimeout(PatternMultiTopicsConsumerImpl.this, - Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); - }); + updateTaskQueue.appendRecheckOp(); } - private CompletableFuture recheckTopicsChange() { + CompletableFuture recheckTopicsChange() { String pattern = topicsPattern.pattern(); final int epoch = recheckPatternEpoch.incrementAndGet(); return client.getLookup().getTopicsUnderNamespace(namespaceName, subscriptionMode, pattern, topicsHash) @@ -271,49 +242,46 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) return CompletableFuture.completedFuture(null); } - CompletableFuture newFuture = updateFuture.thenCompose(ignore -> { - // Unsubscribe and remove consumers in memory. - List> unsubscribeList = new ArrayList<>(removedTopics.size()); - Set partialRemoved = new HashSet<>(removedTopics.size()); - for (String tp : removedTopics) { - ConsumerImpl consumer = consumers.get(tp); - if (consumer != null) { - CompletableFuture unsubscribeFuture = new CompletableFuture<>(); - consumer.closeAsync().whenComplete((__, ex) -> { - if (ex != null) { - log.error("[{}] Failed to unsubscribe from topics: {}", tp, ex); - unsubscribeFuture.completeExceptionally(ex); - } else { - consumers.remove(tp, consumer); - unsubscribeFuture.complete(null); - } - }); - unsubscribeList.add(unsubscribeFuture); - partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); - } + // Unsubscribe and remove consumers in memory. + List> unsubscribeList = new ArrayList<>(removedTopics.size()); + Set partialRemoved = new HashSet<>(removedTopics.size()); + for (String tp : removedTopics) { + ConsumerImpl consumer = consumers.get(tp); + if (consumer != null) { + CompletableFuture unsubscribeFuture = new CompletableFuture<>(); + consumer.closeAsync().whenComplete((__, ex) -> { + if (ex != null) { + log.error("[{}] Failed to unsubscribe from topics: {}", tp, ex); + unsubscribeFuture.completeExceptionally(ex); + } else { + consumers.remove(tp, consumer); + unsubscribeFuture.complete(null); + } + }); + unsubscribeList.add(unsubscribeFuture); + partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); } + } - // Remove partitioned topics in memory. - return FutureUtil.waitForAll(unsubscribeList).whenComplete((__, ex) -> { - for (String groupedTopicRemoved : partialRemoved) { - Integer partitions = partitionedTopics.get(groupedTopicRemoved); - if (partitions != null) { - boolean allPartitionsHasBeenRemoved = true; - for (int i = 0; i < partitions; i++) { - if (consumers.containsKey( - TopicName.get(groupedTopicRemoved).getPartition(i).toString())) { - allPartitionsHasBeenRemoved = false; - break; - } - } - if (allPartitionsHasBeenRemoved) { - partitionedTopics.remove(groupedTopicRemoved, partitions); + // Remove partitioned topics in memory. + return FutureUtil.waitForAll(unsubscribeList).whenComplete((__, ex) -> { + for (String groupedTopicRemoved : partialRemoved) { + Integer partitions = partitionedTopics.get(groupedTopicRemoved); + if (partitions != null) { + boolean allPartitionsHasBeenRemoved = true; + for (int i = 0; i < partitions; i++) { + if (consumers.containsKey( + TopicName.get(groupedTopicRemoved).getPartition(i).toString())) { + allPartitionsHasBeenRemoved = false; + break; } } + if (allPartitionsHasBeenRemoved) { + partitionedTopics.remove(groupedTopicRemoved, partitions); + } } - }); + } }); - return updateFuture = newFuture.exceptionally(ex -> null); } /** @@ -324,58 +292,55 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { if (addedTopics.isEmpty()) { return CompletableFuture.completedFuture(null); } - CompletableFuture newFuture = updateFuture.thenCompose(ignore -> { - List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); - /** - * Three cases: - * 1. Expand partitions. - * 2. Non-partitioned topic, but has been subscribing. - * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. - */ - Set groupedTopics = new HashSet<>(); - for (String tp : addedTopics) { - TopicName topicName = TopicName.get(tp); - groupedTopics.add(topicName.getPartitionedTopicName()); - } - for (String tp : addedTopics) { - TopicName topicName = TopicName.get(tp); - // Case 1: Expand partitions. - if (partitionedTopics.containsKey(topicName.getPartitionedTopicName())) { - if (consumers.containsKey(tp.toString())) { - continue; - } else { - if (topicName.getPartitionIndex() + 1 > - partitionedTopics.get(topicName.getPartitionedTopicName())) { - partitionedTopics.put(topicName.getPartitionedTopicName(), - topicName.getPartitionIndex() + 1); - } - CompletableFuture consumerFuture = subscribeAsync(tp, 0); - consumerFuture.whenComplete((__, ex) -> { - if (ex != null) { - log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); - } - }); - futures.add(consumerFuture); + List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); + /** + * Three cases: + * 1. Expand partitions. + * 2. Non-partitioned topic, but has been subscribing. + * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. + */ + Set groupedTopics = new HashSet<>(); + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + groupedTopics.add(topicName.getPartitionedTopicName()); + } + for (String tp : addedTopics) { + TopicName topicName = TopicName.get(tp); + // Case 1: Expand partitions. + if (partitionedTopics.containsKey(topicName.getPartitionedTopicName())) { + if (consumers.containsKey(tp.toString())) { + continue; + } else { + if (topicName.getPartitionIndex() + 1 > + partitionedTopics.get(topicName.getPartitionedTopicName())) { + partitionedTopics.put(topicName.getPartitionedTopicName(), + topicName.getPartitionIndex() + 1); } - groupedTopics.remove(topicName.getPartitionedTopicName()); - } else if (consumers.containsKey(tp.toString())) { - // Case-2: Non-partitioned topic, but has been subscribing. - groupedTopics.remove(topicName.getPartitionedTopicName()); + CompletableFuture consumerFuture = subscribeAsync(tp, 0); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); + } + }); + futures.add(consumerFuture); } + groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(tp.toString())) { + // Case-2: Non-partitioned topic, but has been subscribing. + groupedTopics.remove(topicName.getPartitionedTopicName()); } - // Case 3: Non-partitioned topic or Partitioned topic, but has not been subscribing. - for (String partitionedTopic : groupedTopics) { - CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); - consumerFuture.whenComplete((__, ex) -> { - if (ex != null) { - log.warn("[{}] Failed to subscribe to topics: {}", partitionedTopic, ex); - } - }); - futures.add(consumerFuture); - } - return FutureUtil.waitForAll(futures); - }); - return updateFuture = newFuture.exceptionally(ex -> null); + } + // Case 3: Non-partitioned topic or Partitioned topic, but has not been subscribing. + for (String partitionedTopic : groupedTopics) { + CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); + consumerFuture.whenComplete((__, ex) -> { + if (ex != null) { + log.warn("[{}] Failed to subscribe to topics: {}", partitionedTopic, ex); + } + }); + futures.add(consumerFuture); + } + return FutureUtil.waitForAll(futures); } } @@ -395,7 +360,7 @@ public CompletableFuture closeAsync() { closeFutures.add(watcher.closeAsync()); } } - closeFutures.add(updateFuture.thenCompose(__ -> super.closeAsync())); + closeFutures.add(updateTaskQueue.cancelAllAndWaitForTheRunningTask().thenCompose(__ -> super.closeAsync())); return FutureUtil.waitForAll(closeFutures); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java index 15922d1180ce0..0007f98b253a0 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/TopicListWatcher.java @@ -43,7 +43,7 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. AtomicLongFieldUpdater .newUpdater(TopicListWatcher.class, "createWatcherDeadline"); - private final PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener; + private final PatternConsumerUpdateQueue patternConsumerUpdateQueue; private final String name; private final ConnectionHandler connectionHandler; private final Pattern topicsPattern; @@ -63,13 +63,13 @@ public class TopicListWatcher extends HandlerState implements ConnectionHandler. /*** * @param topicsPattern The regexp for the topic name(not contains partition suffix). */ - public TopicListWatcher(PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener, + public TopicListWatcher(PatternConsumerUpdateQueue patternConsumerUpdateQueue, PulsarClientImpl client, Pattern topicsPattern, long watcherId, NamespaceName namespace, String topicsHash, CompletableFuture watcherFuture, Runnable recheckTopicsChangeAfterReconnect) { super(client, topicsPattern.pattern()); - this.topicsChangeListener = topicsChangeListener; + this.patternConsumerUpdateQueue = patternConsumerUpdateQueue; this.name = "Watcher(" + topicsPattern + ")"; this.connectionHandler = new ConnectionHandler(this, new BackoffBuilder() @@ -277,13 +277,7 @@ private void cleanupAtClose(CompletableFuture closeFuture, Throwable excep } public void handleCommandWatchTopicUpdate(CommandWatchTopicUpdate update) { - List deleted = update.getDeletedTopicsList(); - if (!deleted.isEmpty()) { - topicsChangeListener.onTopicsRemoved(deleted); - } - List added = update.getNewTopicsList(); - if (!added.isEmpty()) { - topicsChangeListener.onTopicsAdded(added); - } + patternConsumerUpdateQueue.appendTopicsRemovedOp(update.getDeletedTopicsList()); + patternConsumerUpdateQueue.appendTopicsAddedOp(update.getNewTopicsList()); } } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java new file mode 100644 index 0000000000000..7844ba7b5061d --- /dev/null +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.pulsar.client.impl; + +import static org.mockito.ArgumentMatchers.anyCollection; +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 java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import lombok.AllArgsConstructor; +import org.awaitility.Awaitility; +import org.testng.annotations.Test; + +@Test(groups = "utils") +public class PatternConsumerUpdateQueueTest { + + private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, + CompletableFuture customizedPartialUpdateFuture) { + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.recheckTopicsChange()).thenReturn(customizedRecheckFuture); + + PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener = + mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); + when(topicsChangeListener.onTopicsAdded(anyCollection())).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(anyCollection())).thenReturn(customizedPartialUpdateFuture); + + PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, topicsChangeListener); + return new QueueInstance(queue, patternConsumer, topicsChangeListener); + } + + private QueueInstance createInstance() { + CompletableFuture completedFuture = CompletableFuture.completedFuture(null); + return createInstance(completedFuture, completedFuture); + } + + @AllArgsConstructor + private static class QueueInstance { + private PatternConsumerUpdateQueue queue; + private PatternMultiTopicsConsumerImpl mockedConsumer; + private PatternMultiTopicsConsumerImpl.TopicsChangedListener mockedListener; + } + + @Test + public void testTopicsChangedEvents() { + QueueInstance instance = createInstance(); + + Collection topics = Arrays.asList("a"); + for (int i = 0; i < 10; i++) { + instance.queue.appendTopicsAddedOp(topics); + instance.queue.appendTopicsRemovedOp(topics); + } + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedListener, times(10)).onTopicsAdded(topics); + verify(instance.mockedListener, times(10)).onTopicsRemoved(topics); + }); + } + + @Test + public void testRecheckTask() { + QueueInstance instance = createInstance(); + + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + } + + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedConsumer, times(10)).recheckTopicsChange(); + }); + } + + @Test + public void testDelayedRecheckTask() { + CompletableFuture recheckFuture = new CompletableFuture<>(); + CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture); + + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + } + + recheckFuture.complete(null); + Awaitility.await().untilAsserted(() -> { + // The first task will be running, and never completed until all tasks have been added. + // Since the first was started, the second one will not be skipped. + // The others after the second task will be skipped. + // So the times that called "recheckTopicsChange" will be 2. + verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); + }); + } + + @Test + public void testCompositeTasks() { + CompletableFuture recheckFuture = new CompletableFuture<>(); + CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture); + + Collection topics = Arrays.asList("a"); + for (int i = 0; i < 10; i++) { + instance.queue.appendRecheckOp(); + instance.queue.appendTopicsAddedOp(topics); + instance.queue.appendTopicsRemovedOp(topics); + } + recheckFuture.complete(null); + Awaitility.await().untilAsserted(() -> { + // The first task will be running, and never completed until all tasks have been added. + // Since the first was started, the second one will not be skipped. + // The others after the second task will be skipped. + // So the times that called "recheckTopicsChange" will be 2. + verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); + // The tasks after the second "recheckTopicsChange" will be skipped due to there is a previous + // "recheckTopicsChange" that has not been executed. + // The tasks between the fist "recheckTopicsChange" and the second "recheckTopicsChange" will be skipped + // due to there is a following "recheckTopicsChange". + verify(instance.mockedListener, times(0)).onTopicsAdded(topics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(topics); + }); + } +} diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index 74a71f3da850d..63334323b7ce4 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -30,6 +30,7 @@ import org.apache.pulsar.common.api.proto.CommandWatchTopicUpdate; import org.apache.pulsar.common.naming.NamespaceName; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; @@ -68,8 +69,16 @@ public void setup() { thenReturn(clientCnxFuture.thenApply(clientCnx -> Pair.of(clientCnx, false))); when(client.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); when(connectionPool.getConnection(any(), any(), anyInt())).thenReturn(clientCnxFuture); + + CompletableFuture completedFuture = CompletableFuture.completedFuture(null); + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.recheckTopicsChange()).thenReturn(completedFuture); + when(listener.onTopicsAdded(anyCollection())).thenReturn(completedFuture); + when(listener.onTopicsRemoved(anyCollection())).thenReturn(completedFuture); + PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, listener); + watcherFuture = new CompletableFuture<>(); - watcher = new TopicListWatcher(listener, client, + watcher = new TopicListWatcher(queue, client, Pattern.compile(topic), 7, NamespaceName.get("tenant/ns"), null, watcherFuture, () -> {}); } From a5742a25478bea60be02fac6c2c0295f4d9635fd Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 19 Jun 2024 21:27:06 +0800 Subject: [PATCH 08/30] improve comments --- .../pulsar/client/impl/PatternConsumerUpdateQueue.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index ac9f07d5a1a69..af5d1f8de4a66 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -20,8 +20,9 @@ * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsRemoved(Collection)} or * {@link PatternMultiTopicsConsumerImpl#topicsChangeListener#onTopicsAdded(Collection)}. * - * When you are using a release >= 2.11, there are three scenarios: [1, 2, 4]. - * When you are using a release < 2.11, there is only one scenario: [3] and all the event will run in the same thread. + * When you are using this client connect to the broker whose version >= 2.11, there are three scenarios: [1, 2, 4]. + * When you are using this client connect to the broker whose version < 2.11, there is only one scenario: [3] and all + * the event will run in the same thread. */ public class PatternConsumerUpdateQueue { From 7cb42fe8c9e920e93ce06a200bc743cbf7a74699 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 20 Jun 2024 13:53:51 +0800 Subject: [PATCH 09/30] address comments --- .../pulsar/client/impl/PatternConsumerUpdateQueue.java | 3 +++ .../pulsar/client/impl/PatternMultiTopicsConsumerImpl.java | 5 ++--- .../java/org/apache/pulsar/client/impl/PulsarClientImpl.java | 2 ++ 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index af5d1f8de4a66..8c6bc3122dc4c 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -174,6 +174,9 @@ synchronized void triggerNextTask() { public synchronized CompletableFuture cancelAllAndWaitForTheRunningTask() { this.closed = true; + if (taskInProgress == null) { + return CompletableFuture.completedFuture(null); + } return taskInProgress.thenAccept(__ -> {}).exceptionally(ex -> null); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 68859e709528c..4fc94d5f53922 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -32,8 +32,6 @@ import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.LinkedBlockingDeque; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import org.apache.pulsar.client.api.Consumer; @@ -44,6 +42,7 @@ import org.apache.pulsar.common.lookup.GetTopicsResult; import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.apache.pulsar.common.topics.TopicList; import org.apache.pulsar.common.util.Backoff; import org.apache.pulsar.common.util.BackoffBuilder; @@ -316,7 +315,7 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { partitionedTopics.put(topicName.getPartitionedTopicName(), topicName.getPartitionIndex() + 1); } - CompletableFuture consumerFuture = subscribeAsync(tp, 0); + CompletableFuture consumerFuture = subscribeAsync(tp, PartitionedTopicMetadata.NON_PARTITIONED); consumerFuture.whenComplete((__, ex) -> { if (ex != null) { log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index f4afb2931cc9e..37be762068138 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -598,6 +598,8 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), pattern); } conf.getTopicNames().addAll(topicsList); + // Pattern consumer has his unique check mechanism, so do not need the feature "autoUpdatePartitions". + conf.setAutoUpdatePartitions(false); ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(pattern, getTopicsResult.getTopicsHash(), PulsarClientImpl.this, From 24374027ee5771c6ba0477166c807dcdf3bbe7fe Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 20 Jun 2024 13:59:23 +0800 Subject: [PATCH 10/30] address comments --- .../pulsar/client/impl/PatternMultiTopicsConsumerImpl.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 4fc94d5f53922..bf0ff495d44d1 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -297,6 +297,9 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { * 1. Expand partitions. * 2. Non-partitioned topic, but has been subscribing. * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. + * Note: The events that triggered by {@link TopicsPartitionChangedListener} after expanding partitions has + * been disabled through "conf.setAutoUpdatePartitions(false)" when creating + * {@link PatternMultiTopicsConsumerImpl}. */ Set groupedTopics = new HashSet<>(); for (String tp : addedTopics) { From 22a5a869311f772a81f593cee6dc018ee9213ae1 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 26 Jun 2024 10:07:00 +0800 Subject: [PATCH 11/30] address comments --- .../apache/pulsar/client/impl/PatternConsumerUpdateQueue.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index 8c6bc3122dc4c..ec373606dfbe4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -49,7 +49,7 @@ public class PatternConsumerUpdateQueue { */ private boolean recheckTaskInQueue = false; - private long lastRecheckTaskStartingTimestamp = 0; + private volatile long lastRecheckTaskStartingTimestamp = 0; private boolean closed; From 0c6b56e9331560fea41019a26065e14a4edcf975 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 26 Jun 2024 22:14:27 +0800 Subject: [PATCH 12/30] checkstyle --- .../impl/PatternConsumerUpdateQueue.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index ec373606dfbe4..d4ab49a30f508 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -1,3 +1,21 @@ +/* + * 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.pulsar.client.impl; import com.google.common.annotations.VisibleForTesting; From 265b9f1f960cc49d35c2319bf657851c6e25e548 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 26 Jun 2024 23:57:18 +0800 Subject: [PATCH 13/30] checkstyle --- .../pulsar/client/impl/PatternMultiTopicsConsumerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index bf0ff495d44d1..0e5885f145919 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -313,8 +313,8 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { if (consumers.containsKey(tp.toString())) { continue; } else { - if (topicName.getPartitionIndex() + 1 > - partitionedTopics.get(topicName.getPartitionedTopicName())) { + if (topicName.getPartitionIndex() + 1 + > partitionedTopics.get(topicName.getPartitionedTopicName())) { partitionedTopics.put(topicName.getPartitionedTopicName(), topicName.getPartitionIndex() + 1); } From 7e331cd5d42056465a2dc6669cea8a6ec3dd8b2b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 28 Jun 2024 01:42:30 +0800 Subject: [PATCH 14/30] checkstyle --- .../pulsar/client/impl/PatternConsumerUpdateQueue.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index d4ab49a30f508..8a3a25c093c70 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -77,6 +77,7 @@ public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer /** This constructor is only for test. **/ @VisibleForTesting + @SuppressWarnings("EI_EXPOSE_REP2") public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer, PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener) { this.patternConsumer = patternConsumer; @@ -159,6 +160,10 @@ synchronized void triggerNextTask() { recheckTaskInQueue = false; lastRecheckTaskStartingTimestamp = System.currentTimeMillis(); newTaskFuture = patternConsumer.recheckTopicsChange(); + break; + } + default: { + throw new RuntimeException("Un-support UpdateSubscriptionType"); } } From 2849534654a6e3251868b267afc553c9b7fac152 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 28 Jun 2024 01:57:27 +0800 Subject: [PATCH 15/30] checkstyle --- .../apache/pulsar/client/impl/PatternConsumerUpdateQueue.java | 1 - 1 file changed, 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index 8a3a25c093c70..d624acecf954f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -77,7 +77,6 @@ public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer /** This constructor is only for test. **/ @VisibleForTesting - @SuppressWarnings("EI_EXPOSE_REP2") public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer, PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener) { this.patternConsumer = patternConsumer; From dde10edbe8bfd2eb7b258275005b071c80b44cdb Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 28 Jun 2024 01:59:32 +0800 Subject: [PATCH 16/30] checkstyle --- .../apache/pulsar/client/impl/PatternConsumerUpdateQueue.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index d624acecf954f..b16c4c54afd91 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -19,6 +19,7 @@ package org.apache.pulsar.client.impl; import com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.Collection; import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; @@ -42,6 +43,7 @@ * When you are using this client connect to the broker whose version < 2.11, there is only one scenario: [3] and all * the event will run in the same thread. */ +@SuppressFBWarnings("EI_EXPOSE_REP2") public class PatternConsumerUpdateQueue { private static final Pair> RECHECK_OP = From 166281c3ab00bf008a2ea180ddf9a69297bf252c Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 28 Jun 2024 15:56:15 +0800 Subject: [PATCH 17/30] fix test --- .../pulsar/client/impl/PatternMultiTopicsConsumerImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 0e5885f145919..44edcfb67d7bf 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -89,7 +89,6 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, this.topicsPattern = topicsPattern; this.topicsHash = topicsHash; this.subscriptionMode = subscriptionMode; - this.updateTaskQueue = new PatternConsumerUpdateQueue(this); this.recheckPatternTaskBackoff = new BackoffBuilder() .setInitialTime(client.getConfiguration().getInitialBackoffIntervalNanos(), TimeUnit.NANOSECONDS) .setMax(client.getConfiguration().getMaxBackoffIntervalNanos(), TimeUnit.NANOSECONDS) @@ -102,6 +101,7 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, checkArgument(getNameSpaceFromPattern(topicsPattern).toString().equals(this.namespaceName.toString())); this.topicsChangeListener = new PatternTopicsChangedListener(); + this.updateTaskQueue = new PatternConsumerUpdateQueue(this); this.recheckPatternTimeout = client.timer() .newTimeout(this, Math.max(1, conf.getPatternAutoDiscoveryPeriod()), TimeUnit.SECONDS); if (subscriptionMode == Mode.PERSISTENT) { From e128704d0e5eb7fca7b7d4280b61d33010c63381 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 28 Jun 2024 17:23:57 +0800 Subject: [PATCH 18/30] fix stuck --- .../client/impl/TopicsConsumerImplTest.java | 1 + .../impl/PatternConsumerUpdateQueue.java | 17 ++-- .../impl/PatternConsumerUpdateQueueTest.java | 77 ++++++++++++++++++- 3 files changed, 87 insertions(+), 8 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index c343ab0d6e294..564ccbe64b6f9 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Sets; import io.netty.util.Timeout; +import java.time.Duration; import lombok.Cleanup; import org.apache.bookkeeper.mledger.Position; import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index b16c4c54afd91..1783bbee9b37b 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -24,6 +24,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.lang3.tuple.Pair; /** @@ -43,6 +44,7 @@ * When you are using this client connect to the broker whose version < 2.11, there is only one scenario: [3] and all * the event will run in the same thread. */ +@Slf4j @SuppressFBWarnings("EI_EXPOSE_REP2") public class PatternConsumerUpdateQueue { @@ -132,7 +134,7 @@ synchronized void triggerNextTask() { return; } - Pair> task = pendingTasks.poll(); + final Pair> task = pendingTasks.poll(); // No pending task. if (task == null) { @@ -167,7 +169,10 @@ synchronized void triggerNextTask() { throw new RuntimeException("Un-support UpdateSubscriptionType"); } } - + if (log.isDebugEnabled()) { + log.debug("[{} {}] Pattern consumer [{}] update subscriptions", task.getLeft(), + task.getRight() == null ? "" : task.getRight(), patternConsumer.getSubscription()); + } // Trigger next pending task. taskInProgress = newTaskFuture; newTaskFuture.thenAccept(ignore -> { @@ -178,7 +183,8 @@ synchronized void triggerNextTask() { * - Skip if there is already a recheck task in queue. * - Skip if the last recheck task has been executed after the current time. */ - // TODO log. + log.error("[{} {}] Pattern consumer [{}] failed to update subscriptions", task.getLeft(), task.getRight(), + patternConsumer.getSubscription(), ex); // Skip if there is already a recheck task in queue. synchronized (PatternConsumerUpdateQueue.this) { if (recheckTaskInQueue || PatternConsumerUpdateQueue.this.closed) { @@ -187,11 +193,12 @@ synchronized void triggerNextTask() { } // Skip if the last recheck task has been executed after the current time. long failedTime = System.currentTimeMillis(); - patternConsumer.client.timer().newTimeout(timeout -> { + patternConsumer.getClient().timer().newTimeout(timeout -> { if (lastRecheckTaskStartingTimestamp <= failedTime) { appendRecheckOp(); } - }, 30, TimeUnit.SECONDS); + }, 10, TimeUnit.SECONDS); + triggerNextTask(); return null; }); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java index 7844ba7b5061d..743d419c531b3 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java @@ -23,10 +23,17 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import io.netty.util.HashedWheelTimer; +import java.io.Closeable; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; +import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import lombok.AllArgsConstructor; +import org.apache.pulsar.client.util.ExecutorProvider; +import org.apache.pulsar.common.util.FutureUtil; import org.awaitility.Awaitility; import org.testng.annotations.Test; @@ -35,13 +42,34 @@ public class PatternConsumerUpdateQueueTest { private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, CompletableFuture customizedPartialUpdateFuture) { + return createInstance(customizedRecheckFuture, customizedPartialUpdateFuture, null, null); + } + + private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, + CompletableFuture customizedPartialUpdateFuture, + Collection successTopics, + Collection errorTopics) { + HashedWheelTimer timer = new HashedWheelTimer(new ExecutorProvider.ExtendedThreadFactory("timer-x", + Thread.currentThread().isDaemon()), 1, TimeUnit.MILLISECONDS); + PulsarClientImpl client = mock(PulsarClientImpl.class); + when(client.timer()).thenReturn(timer); + PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); when(patternConsumer.recheckTopicsChange()).thenReturn(customizedRecheckFuture); + when(patternConsumer.getClient()).thenReturn(client); PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener = mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); - when(topicsChangeListener.onTopicsAdded(anyCollection())).thenReturn(customizedPartialUpdateFuture); - when(topicsChangeListener.onTopicsRemoved(anyCollection())).thenReturn(customizedPartialUpdateFuture); + if (successTopics == null && errorTopics == null) { + when(topicsChangeListener.onTopicsAdded(anyCollection())).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(anyCollection())).thenReturn(customizedPartialUpdateFuture); + } else { + CompletableFuture ex = FutureUtil.failedFuture(new RuntimeException("mock error")); + when(topicsChangeListener.onTopicsAdded(successTopics)).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsRemoved(successTopics)).thenReturn(customizedPartialUpdateFuture); + when(topicsChangeListener.onTopicsAdded(errorTopics)).thenReturn(ex); + when(topicsChangeListener.onTopicsRemoved(errorTopics)).thenReturn(ex); + } PatternConsumerUpdateQueue queue = new PatternConsumerUpdateQueue(patternConsumer, topicsChangeListener); return new QueueInstance(queue, patternConsumer, topicsChangeListener); @@ -53,10 +81,15 @@ private QueueInstance createInstance() { } @AllArgsConstructor - private static class QueueInstance { + private static class QueueInstance implements Closeable { private PatternConsumerUpdateQueue queue; private PatternMultiTopicsConsumerImpl mockedConsumer; private PatternMultiTopicsConsumerImpl.TopicsChangedListener mockedListener; + + @Override + public void close() { + mockedConsumer.getClient().timer().stop(); + } } @Test @@ -72,6 +105,9 @@ public void testTopicsChangedEvents() { verify(instance.mockedListener, times(10)).onTopicsAdded(topics); verify(instance.mockedListener, times(10)).onTopicsRemoved(topics); }); + + // cleanup. + instance.close(); } @Test @@ -85,6 +121,9 @@ public void testRecheckTask() { Awaitility.await().untilAsserted(() -> { verify(instance.mockedConsumer, times(10)).recheckTopicsChange(); }); + + // cleanup. + instance.close(); } @Test @@ -105,6 +144,9 @@ public void testDelayedRecheckTask() { // So the times that called "recheckTopicsChange" will be 2. verify(instance.mockedConsumer, times(2)).recheckTopicsChange(); }); + + // cleanup. + instance.close(); } @Test @@ -133,5 +175,34 @@ public void testCompositeTasks() { verify(instance.mockedListener, times(0)).onTopicsAdded(topics); verify(instance.mockedListener, times(0)).onTopicsRemoved(topics); }); + + // cleanup. + instance.close(); + } + + @Test + public void testErrorTask() { + CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); + Collection successTopics = Arrays.asList("a"); + Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, successTopics, + errorTopics); + + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + instance.queue.appendTopicsAddedOp(errorTopics); + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + + Awaitility.await().atMost(Duration.ofSeconds(60)).untilAsserted(() -> { + verify(instance.mockedListener, times(2)).onTopicsAdded(successTopics); + verify(instance.mockedListener, times(2)).onTopicsRemoved(successTopics); + verify(instance.mockedListener, times(1)).onTopicsAdded(errorTopics); + // After an error task will push a recheck task to offset. + verify(instance.mockedConsumer, times(1)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); } } From 3c0cd89c387cafb25e0a8c6c0f1aab7d2be2d731 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 28 Jun 2024 17:55:16 +0800 Subject: [PATCH 19/30] fix test --- .../apache/pulsar/client/impl/TopicsConsumerImplTest.java | 4 +--- .../pulsar/client/impl/PatternConsumerUpdateQueue.java | 6 +++++- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java index 564ccbe64b6f9..83cb5f2a4400b 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/TopicsConsumerImplTest.java @@ -1322,7 +1322,6 @@ public void testPartitionsUpdatesForMultipleTopics() throws Exception { Assert.assertEquals(consumer.allTopicPartitionsNumber.intValue(), 2); admin.topics().updatePartitionedTopic(topicName0, 5); - consumer.getPartitionsAutoUpdateTimeout().task().run(consumer.getPartitionsAutoUpdateTimeout()); Awaitility.await().untilAsserted(() -> { Assert.assertEquals(consumer.getPartitionsOfTheTopicMap(), 5); @@ -1342,9 +1341,8 @@ public void testPartitionsUpdatesForMultipleTopics() throws Exception { }); admin.topics().updatePartitionedTopic(topicName1, 5); - consumer.getPartitionsAutoUpdateTimeout().task().run(consumer.getPartitionsAutoUpdateTimeout()); - Awaitility.await().untilAsserted(() -> { + Awaitility.await().atMost(Duration.ofSeconds(30)).untilAsserted(() -> { Assert.assertEquals(consumer.getPartitionsOfTheTopicMap(), 10); Assert.assertEquals(consumer.allTopicPartitionsNumber.intValue(), 10); }); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index 1783bbee9b37b..3e526e0c7ea66 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -107,6 +107,10 @@ synchronized void appendRecheckOp() { } synchronized void doAppend(Pair> task) { + if (log.isDebugEnabled()) { + log.debug("[{} {}] Pattern consumer [{}] try to append task", task.getLeft(), + task.getRight() == null ? "" : task.getRight(), patternConsumer.getSubscription()); + } // Once there is a recheck task in queue, it means other tasks can be skipped. if (recheckTaskInQueue) { return; @@ -170,7 +174,7 @@ synchronized void triggerNextTask() { } } if (log.isDebugEnabled()) { - log.debug("[{} {}] Pattern consumer [{}] update subscriptions", task.getLeft(), + log.debug("[{} {}] Pattern consumer [{}] updating subscriptions", task.getLeft(), task.getRight() == null ? "" : task.getRight(), patternConsumer.getSubscription()); } // Trigger next pending task. From d4f2250b4287d0325c34b995c10a4f450067b453 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Tue, 2 Jul 2024 09:28:02 +0800 Subject: [PATCH 20/30] fix race condition --- .../api/PatternMultiTopicsConsumerTest.java | 37 +++++++++++++ .../pulsar/client/impl/ConsumerBase.java | 6 +++ .../client/impl/MultiTopicsConsumerImpl.java | 4 +- .../impl/PatternConsumerUpdateQueue.java | 30 +++++++++-- .../impl/PatternMultiTopicsConsumerImpl.java | 6 +++ .../impl/PatternConsumerUpdateQueueTest.java | 53 ++++++++++++++++--- 6 files changed, 124 insertions(+), 12 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java index 00a47c3957150..475477ac52149 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/api/PatternMultiTopicsConsumerTest.java @@ -18,11 +18,14 @@ */ package org.apache.pulsar.client.api; +import static org.testng.Assert.fail; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.pulsar.broker.BrokerTestUtil; +import org.apache.pulsar.client.impl.PatternMultiTopicsConsumerImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testng.Assert; @@ -95,4 +98,38 @@ private void testWithConsumer(Consumer consumer) throws Exception { consumer.close(); } + @Test(timeOut = 30000) + public void testFailedSubscribe() throws Exception { + final String topicName1 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String topicName2 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String topicName3 = BrokerTestUtil.newUniqueName("persistent://public/default/tp_test"); + final String subName = "s1"; + admin.topics().createPartitionedTopic(topicName1, 2); + admin.topics().createPartitionedTopic(topicName2, 3); + admin.topics().createNonPartitionedTopic(topicName3); + + // Register a exclusive consumer to makes the pattern consumer failed to subscribe. + Consumer c1 = pulsarClient.newConsumer(Schema.STRING).topic(topicName3).subscriptionType(SubscriptionType.Exclusive) + .subscriptionName(subName).subscribe(); + + try { + PatternMultiTopicsConsumerImpl consumer = + (PatternMultiTopicsConsumerImpl) pulsarClient.newConsumer(Schema.STRING) + .topicsPattern("persistent://public/default/tp_test.*") + .subscriptionType(SubscriptionType.Failover) + .subscriptionName(subName) + .subscribe(); + fail("Expected a consumer busy error."); + } catch (Exception ex) { + log.info("consumer busy", ex); + } + + c1.close(); + // Verify all internal consumer will be closed. + // If delete topic without "-f" work, it means the internal consumers were closed. + admin.topics().delete(topicName3); + admin.topics().deletePartitionedTopic(topicName2); + admin.topics().deletePartitionedTopic(topicName1); + } + } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java index 05081dcaa07ea..74abb82bfe809 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/ConsumerBase.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Queues; import io.netty.util.Timeout; import java.nio.charset.StandardCharsets; @@ -1285,5 +1286,10 @@ public boolean hasBatchReceiveTimeout() { return batchReceiveTimeout != null; } + @VisibleForTesting + CompletableFuture> getSubscribeFuture() { + return subscribeFuture; + } + private static final Logger log = LoggerFactory.getLogger(ConsumerBase.class); } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java index 259e8b744a709..e8cbf71e500c9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiTopicsConsumerImpl.java @@ -1167,7 +1167,7 @@ private void doSubscribeTopicPartitions(Schema schema, .exceptionally(ex -> { log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, ex.getMessage()); - subscribeResult.completeExceptionally(ex); + handleSubscribeOneTopicError(topicName, ex, subscribeResult); return null; }); } @@ -1190,7 +1190,7 @@ private ConsumerImpl createInternalConsumer(ConsumerConfigurationData conf } // handling failure during subscribe new topic, unsubscribe success created partitions - private void handleSubscribeOneTopicError(String topicName, + protected void handleSubscribeOneTopicError(String topicName, Throwable error, CompletableFuture subscribeFuture) { log.warn("[{}] Failed to subscribe for topic [{}] in topics consumer {}", topic, topicName, error.getMessage()); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index 3e526e0c7ea66..0b87fd2897f0f 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -61,7 +61,7 @@ public class PatternConsumerUpdateQueue { * Whether there is a task is in progress, this variable is used to confirm whether a next-task triggering is * needed. */ - private CompletableFuture taskInProgress = null; + private Pair> taskInProgress = null; /** * Whether there is a recheck task in queue. @@ -86,6 +86,9 @@ public PatternConsumerUpdateQueue(PatternMultiTopicsConsumerImpl patternConsumer this.patternConsumer = patternConsumer; this.topicsChangeListener = topicsChangeListener; this.pendingTasks = new LinkedBlockingQueue<>(); + // To avoid subscribing and topics changed events execute concurrently, let the change events starts after the + // subscribing task. + doAppend(Pair.of(UpdateSubscriptionType.CONSUMER_INIT, null)); } synchronized void appendTopicsAddedOp(Collection topics) { @@ -155,6 +158,21 @@ synchronized void triggerNextTask() { // Execute pending task. CompletableFuture newTaskFuture = null; switch (task.getLeft()) { + case CONSUMER_INIT: { + newTaskFuture = patternConsumer.getSubscribeFuture().thenAccept(__ -> {}).exceptionally(ex -> { + // If the subscribe future was failed, the consumer will be closed. + synchronized (PatternConsumerUpdateQueue.this) { + this.closed = true; + patternConsumer.closeAsync().exceptionally(ex2 -> { + log.error("Pattern consumer failed to close, this error may left orphan consumers." + + " Subscription: {}", patternConsumer.getSubscription()); + return null; + }); + } + return null; + }); + break; + } case TOPICS_ADDED: { newTaskFuture = topicsChangeListener.onTopicsAdded(task.getRight()); break; @@ -178,7 +196,7 @@ synchronized void triggerNextTask() { task.getRight() == null ? "" : task.getRight(), patternConsumer.getSubscription()); } // Trigger next pending task. - taskInProgress = newTaskFuture; + taskInProgress = Pair.of(task.getLeft(), newTaskFuture); newTaskFuture.thenAccept(ignore -> { triggerNextTask(); }).exceptionally(ex -> { @@ -212,10 +230,16 @@ public synchronized CompletableFuture cancelAllAndWaitForTheRunningTask() if (taskInProgress == null) { return CompletableFuture.completedFuture(null); } - return taskInProgress.thenAccept(__ -> {}).exceptionally(ex -> null); + // If the in-progress task is consumer init task, it means nothing is in-progress. + if (taskInProgress.getLeft().equals(UpdateSubscriptionType.CONSUMER_INIT)) { + return CompletableFuture.completedFuture(null); + } + return taskInProgress.getRight().thenAccept(__ -> {}).exceptionally(ex -> null); } private enum UpdateSubscriptionType { + /** A marker that indicates the consumer's subscribe task.**/ + CONSUMER_INIT, /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ TOPICS_ADDED, /** Triggered by {@link PatternMultiTopicsConsumerImpl#topicsChangeListener}.**/ diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 44edcfb67d7bf..f003d98a31fa8 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -371,5 +371,11 @@ Timeout getRecheckPatternTimeout() { return recheckPatternTimeout; } + protected void handleSubscribeOneTopicError(String topicName, + Throwable error, + CompletableFuture subscribeFuture) { + subscribeFuture.completeExceptionally(error); + } + private static final Logger log = LoggerFactory.getLogger(PatternMultiTopicsConsumerImpl.class); } diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java index 743d419c531b3..01f0be6a85ef6 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueueTest.java @@ -41,12 +41,15 @@ public class PatternConsumerUpdateQueueTest { private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, - CompletableFuture customizedPartialUpdateFuture) { - return createInstance(customizedRecheckFuture, customizedPartialUpdateFuture, null, null); + CompletableFuture customizedPartialUpdateFuture, + CompletableFuture customizedConsumerInitFuture) { + return createInstance(customizedRecheckFuture, customizedPartialUpdateFuture, customizedConsumerInitFuture, + null, null); } private QueueInstance createInstance(CompletableFuture customizedRecheckFuture, CompletableFuture customizedPartialUpdateFuture, + CompletableFuture customizedConsumerInitFuture, Collection successTopics, Collection errorTopics) { HashedWheelTimer timer = new HashedWheelTimer(new ExecutorProvider.ExtendedThreadFactory("timer-x", @@ -57,6 +60,11 @@ private QueueInstance createInstance(CompletableFuture customizedRecheckFu PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); when(patternConsumer.recheckTopicsChange()).thenReturn(customizedRecheckFuture); when(patternConsumer.getClient()).thenReturn(client); + if (customizedConsumerInitFuture != null) { + when(patternConsumer.getSubscribeFuture()).thenReturn(customizedConsumerInitFuture); + } else { + when(patternConsumer.getSubscribeFuture()).thenReturn(CompletableFuture.completedFuture(null)); + } PatternMultiTopicsConsumerImpl.TopicsChangedListener topicsChangeListener = mock(PatternMultiTopicsConsumerImpl.TopicsChangedListener.class); @@ -77,7 +85,7 @@ private QueueInstance createInstance(CompletableFuture customizedRecheckFu private QueueInstance createInstance() { CompletableFuture completedFuture = CompletableFuture.completedFuture(null); - return createInstance(completedFuture, completedFuture); + return createInstance(completedFuture, completedFuture, completedFuture); } @AllArgsConstructor @@ -130,7 +138,8 @@ public void testRecheckTask() { public void testDelayedRecheckTask() { CompletableFuture recheckFuture = new CompletableFuture<>(); CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); - QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture); + CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); for (int i = 0; i < 10; i++) { instance.queue.appendRecheckOp(); @@ -153,7 +162,8 @@ public void testDelayedRecheckTask() { public void testCompositeTasks() { CompletableFuture recheckFuture = new CompletableFuture<>(); CompletableFuture partialUpdateFuture = CompletableFuture.completedFuture(null); - QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture); + CompletableFuture consumerInitFuture = CompletableFuture.completedFuture(null); + QueueInstance instance = createInstance(recheckFuture, partialUpdateFuture, consumerInitFuture); Collection topics = Arrays.asList("a"); for (int i = 0; i < 10; i++) { @@ -185,8 +195,8 @@ public void testErrorTask() { CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); Collection successTopics = Arrays.asList("a"); Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); - QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, successTopics, - errorTopics); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, + immediatelyCompleteFuture, successTopics, errorTopics); instance.queue.appendTopicsAddedOp(successTopics); instance.queue.appendTopicsRemovedOp(successTopics); @@ -205,4 +215,33 @@ public void testErrorTask() { // cleanup. instance.close(); } + + @Test + public void testFailedSubscribe() { + CompletableFuture immediatelyCompleteFuture = CompletableFuture.completedFuture(null); + CompletableFuture consumerInitFuture = new CompletableFuture<>(); + Collection successTopics = Arrays.asList("a"); + Collection errorTopics = Arrays.asList(UUID.randomUUID().toString()); + QueueInstance instance = createInstance(immediatelyCompleteFuture, immediatelyCompleteFuture, + consumerInitFuture, successTopics, errorTopics); + + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + instance.queue.appendTopicsAddedOp(errorTopics); + instance.queue.appendTopicsAddedOp(successTopics); + instance.queue.appendTopicsRemovedOp(successTopics); + + // Consumer init failed after multi topics changes. + // All the topics changes events should be skipped. + consumerInitFuture.completeExceptionally(new RuntimeException("mocked ex")); + Awaitility.await().untilAsserted(() -> { + verify(instance.mockedListener, times(0)).onTopicsAdded(successTopics); + verify(instance.mockedListener, times(0)).onTopicsRemoved(successTopics); + verify(instance.mockedListener, times(0)).onTopicsAdded(errorTopics); + verify(instance.mockedConsumer, times(0)).recheckTopicsChange(); + }); + + // cleanup. + instance.close(); + } } From 5ea4e41c0fd3a8633be118189de8456bae40078e Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 3 Jul 2024 10:50:51 +0800 Subject: [PATCH 21/30] fix Pattern --- .../main/java/org/apache/pulsar/common/naming/TopicName.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java index 2298a8d06a880..d264eab9574ef 100644 --- a/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java +++ b/pulsar-common/src/main/java/org/apache/pulsar/common/naming/TopicName.java @@ -23,12 +23,12 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import com.google.common.util.concurrent.UncheckedExecutionException; +import com.google.re2j.Pattern; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.List; import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.regex.Pattern; import org.apache.commons.lang3.StringUtils; import org.apache.pulsar.common.util.Codec; From c390775d9d6dad2f885bbb3ddf67165ba7963296 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 3 Jul 2024 10:57:10 +0800 Subject: [PATCH 22/30] fix test --- .../java/org/apache/pulsar/client/impl/TopicListWatcherTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java index 63334323b7ce4..7daf316c4c576 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/TopicListWatcherTest.java @@ -72,6 +72,7 @@ public void setup() { CompletableFuture completedFuture = CompletableFuture.completedFuture(null); PatternMultiTopicsConsumerImpl patternConsumer = mock(PatternMultiTopicsConsumerImpl.class); + when(patternConsumer.getSubscribeFuture()).thenReturn(completedFuture); when(patternConsumer.recheckTopicsChange()).thenReturn(completedFuture); when(listener.onTopicsAdded(anyCollection())).thenReturn(completedFuture); when(listener.onTopicsRemoved(anyCollection())).thenReturn(completedFuture); From 24b59e803092e09bee2b3b111ba959bb86c699b3 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Wed, 3 Jul 2024 22:37:27 +0800 Subject: [PATCH 23/30] fix bugs and improve logs --- .../impl/PatternTopicsConsumerImplTest.java | 11 ++-- .../impl/PatternConsumerUpdateQueue.java | 18 +++-- .../impl/PatternMultiTopicsConsumerImpl.java | 65 ++++++++++++------- .../pulsar/client/impl/PulsarClientImpl.java | 13 +++- .../PatternMultiTopicsConsumerImplTest.java | 6 +- 5 files changed, 73 insertions(+), 40 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index c5504a0c02a0c..26a7bea3a8c6a 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -53,6 +53,7 @@ import org.apache.pulsar.common.api.proto.BaseCommand; import org.apache.pulsar.common.api.proto.CommandWatchTopicListSuccess; import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.awaitility.Awaitility; import org.awaitility.reflect.WhiteboxImpl; @@ -1024,17 +1025,17 @@ public void testAutoUnsubscribePatternConsumer() throws Exception { // 6. remove producer 1,3; verify only consumer 2 left // seems no direct way to verify auto-unsubscribe, because this patternConsumer also referenced the topic. - List topicNames = Lists.newArrayList(topicName2); + String tp2p0 = TopicName.get(topicName2).getPartition(0).toString(); + String tp2p1 = TopicName.get(topicName2).getPartition(1).toString(); + List topicNames = Lists.newArrayList(tp2p0, tp2p1); NamespaceService nss = pulsar.getNamespaceService(); doReturn(CompletableFuture.completedFuture(topicNames)).when(nss) .getListOfPersistentTopics(NamespaceName.get("my-property/my-ns")); // 7. call recheckTopics to unsubscribe topic 1,3, verify topics number: 2=6-1-3 log.debug("recheck topics change"); - PatternMultiTopicsConsumerImpl consumer1 = ((PatternMultiTopicsConsumerImpl) consumer); - Timeout recheckPatternTimeout = spy(consumer1.getRecheckPatternTimeout()); - doReturn(false).when(recheckPatternTimeout).isCancelled(); - consumer1.run(recheckPatternTimeout); + PatternConsumerUpdateQueue taskQueue = WhiteboxImpl.getInternalState(consumer, "updateTaskQueue"); + taskQueue.appendRecheckOp(); Thread.sleep(100); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getPartitions().size(), 2); assertEquals(((PatternMultiTopicsConsumerImpl) consumer).getConsumers().size(), 2); diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java index 0b87fd2897f0f..d6eba6463a07d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternConsumerUpdateQueue.java @@ -111,8 +111,8 @@ synchronized void appendRecheckOp() { synchronized void doAppend(Pair> task) { if (log.isDebugEnabled()) { - log.debug("[{} {}] Pattern consumer [{}] try to append task", task.getLeft(), - task.getRight() == null ? "" : task.getRight(), patternConsumer.getSubscription()); + log.debug("Pattern consumer [{}] try to append task. {} {}", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); } // Once there is a recheck task in queue, it means other tasks can be skipped. if (recheckTaskInQueue) { @@ -192,21 +192,25 @@ synchronized void triggerNextTask() { } } if (log.isDebugEnabled()) { - log.debug("[{} {}] Pattern consumer [{}] updating subscriptions", task.getLeft(), - task.getRight() == null ? "" : task.getRight(), patternConsumer.getSubscription()); + log.debug("Pattern consumer [{}] starting task. {} {} ", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); } // Trigger next pending task. taskInProgress = Pair.of(task.getLeft(), newTaskFuture); newTaskFuture.thenAccept(ignore -> { - triggerNextTask(); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] task finished. {} {} ", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight()); + } + triggerNextTask(); }).exceptionally(ex -> { /** * Once a updating fails, trigger a delayed new recheck task to guarantee all things is correct. * - Skip if there is already a recheck task in queue. * - Skip if the last recheck task has been executed after the current time. */ - log.error("[{} {}] Pattern consumer [{}] failed to update subscriptions", task.getLeft(), task.getRight(), - patternConsumer.getSubscription(), ex); + log.error("Pattern consumer [{}] task finished. {} {}. But it failed", patternConsumer.getSubscription(), + task.getLeft(), task.getRight() == null ? "" : task.getRight(), ex); // Skip if there is already a recheck task in queue. synchronized (PatternConsumerUpdateQueue.this) { if (recheckTaskInQueue || PatternConsumerUpdateQueue.this.closed) { diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index f003d98a31fa8..f4584371df1a6 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -111,11 +111,13 @@ public PatternMultiTopicsConsumerImpl(Pattern topicsPattern, watcherFuture .thenAccept(__ -> recheckPatternTimeout.cancel()) .exceptionally(ex -> { - log.warn("Unable to create topic list watcher. Falling back to only polling for new topics", ex); + log.warn("Pattern consumer [{}] unable to create topic list watcher. Falling back to only polling" + + " for new topics", conf.getSubscriptionName(), ex); return null; }); } else { - log.debug("Not creating topic list watcher for subscription mode {}", subscriptionMode); + log.debug("Pattern consumer [{}] not creating topic list watcher for subscription mode {}", + conf.getSubscriptionName(), subscriptionMode); watcherFuture.complete(null); } } @@ -133,17 +135,7 @@ private void recheckTopicsChangeAfterReconnect() { return; } // Do check. - recheckTopicsChange().whenComplete((ignore, ex) -> { - if (ex != null) { - log.warn("[{}] Failed to recheck topics change: {}", topic, ex.getMessage()); - long delayMs = recheckPatternTaskBackoff.next(); - client.timer().newTimeout(timeout -> { - recheckTopicsChangeAfterReconnect(); - }, delayMs, TimeUnit.MILLISECONDS); - } else { - recheckPatternTaskBackoff.reset(); - } - }); + updateTaskQueue.appendRecheckOp(); } // TimerTask to recheck topics change, and trigger subscribe/unsubscribe based on the change. @@ -168,7 +160,9 @@ CompletableFuture recheckTopicsChange() { return CompletableFuture.completedFuture(null); } if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}, topicsHash: {}, filtered: {}", + log.debug("Pattern consumer [{}] get topics under namespace {}, topics.size: {}," + + " topicsHash: {}, filtered: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> @@ -177,7 +171,7 @@ CompletableFuture recheckTopicsChange() { final List oldTopics = new ArrayList<>(getPartitions()); return updateSubscriptions(topicsPattern, this::setTopicsHash, getTopicsResult, - topicsChangeListener, oldTopics); + topicsChangeListener, oldTopics, subscription); } }); } @@ -186,7 +180,8 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, java.util.function.Consumer topicsHashSetter, GetTopicsResult getTopicsResult, TopicsChangedListener topicsChangedListener, - List oldTopics) { + List oldTopics, + String subscriptionForLog) { topicsHashSetter.accept(getTopicsResult.getTopicsHash()); if (!getTopicsResult.isChanged()) { return CompletableFuture.completedFuture(null); @@ -200,8 +195,16 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, } final List> listenersCallback = new ArrayList<>(2); - listenersCallback.add(topicsChangedListener.onTopicsAdded(TopicList.minus(newTopics, oldTopics))); - listenersCallback.add(topicsChangedListener.onTopicsRemoved(TopicList.minus(oldTopics, newTopics))); + Set topicsAdded = TopicList.minus(newTopics, oldTopics); + Set topicsRemoved = TopicList.minus(oldTopics, newTopics); + log.info("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: []", + subscriptionForLog, topicsAdded, topicsRemoved); + if (!topicsAdded.isEmpty()) { + listenersCallback.add(topicsChangedListener.onTopicsAdded(topicsAdded)); + } + if (!topicsRemoved.isEmpty()) { + listenersCallback.add(topicsChangedListener.onTopicsRemoved(topicsRemoved)); + } return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); } @@ -244,13 +247,15 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) // Unsubscribe and remove consumers in memory. List> unsubscribeList = new ArrayList<>(removedTopics.size()); Set partialRemoved = new HashSet<>(removedTopics.size()); + Set partialRemovedForLog = new HashSet<>(removedTopics.size()); for (String tp : removedTopics) { ConsumerImpl consumer = consumers.get(tp); if (consumer != null) { CompletableFuture unsubscribeFuture = new CompletableFuture<>(); consumer.closeAsync().whenComplete((__, ex) -> { if (ex != null) { - log.error("[{}] Failed to unsubscribe from topics: {}", tp, ex); + log.error("Pattern consumer [{}] failed to unsubscribe from topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), tp, ex); unsubscribeFuture.completeExceptionally(ex); } else { consumers.remove(tp, consumer); @@ -259,11 +264,15 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) }); unsubscribeList.add(unsubscribeFuture); partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); + partialRemovedForLog.add(TopicName.get(tp).toString()); } } + log.info("Pattern consumer [{}] remove topics. {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), + partialRemovedForLog); // Remove partitioned topics in memory. - return FutureUtil.waitForAll(unsubscribeList).whenComplete((__, ex) -> { + return FutureUtil.waitForAll(unsubscribeList).handle((__, ex) -> { + List removedPartitionedTopicsForLog = new ArrayList<>(); for (String groupedTopicRemoved : partialRemoved) { Integer partitions = partitionedTopics.get(groupedTopicRemoved); if (partitions != null) { @@ -276,10 +285,16 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) } } if (allPartitionsHasBeenRemoved) { + removedPartitionedTopicsForLog.add(String.format("%s with %s partitions", + groupedTopicRemoved, partitions)); partitionedTopics.remove(groupedTopicRemoved, partitions); } } } + log.info("Pattern consumer [{}] remove partitioned topics because all partitions have been removed. {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), + removedPartitionedTopicsForLog); + return null; }); } @@ -302,6 +317,7 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { * {@link PatternMultiTopicsConsumerImpl}. */ Set groupedTopics = new HashSet<>(); + List expendPartitionsForLog = new ArrayList<>(); for (String tp : addedTopics) { TopicName topicName = TopicName.get(tp); groupedTopics.add(topicName.getPartitionedTopicName()); @@ -318,10 +334,12 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { partitionedTopics.put(topicName.getPartitionedTopicName(), topicName.getPartitionIndex() + 1); } + expendPartitionsForLog.add(tp); CompletableFuture consumerFuture = subscribeAsync(tp, PartitionedTopicMetadata.NON_PARTITIONED); consumerFuture.whenComplete((__, ex) -> { if (ex != null) { - log.warn("[{}] Failed to subscribe to topics: {}", tp, ex); + log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), tp, ex); } }); futures.add(consumerFuture); @@ -337,11 +355,14 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); consumerFuture.whenComplete((__, ex) -> { if (ex != null) { - log.warn("[{}] Failed to subscribe to topics: {}", partitionedTopic, ex); + log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), partitionedTopic, ex); } }); futures.add(consumerFuture); } + log.info("Pattern consumer [{}] add topics. expend partitions {}, new subscribing {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), expendPartitionsForLog, groupedTopics); return FutureUtil.waitForAll(futures); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index 37be762068138..df3c94ee70e92 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -585,12 +585,13 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo lookup.getTopicsUnderNamespace(namespaceName, subscriptionMode, regex, null) .thenAccept(getTopicsResult -> { if (log.isDebugEnabled()) { - log.debug("Get topics under namespace {}, topics.size: {}," - + " topicsHash: {}, changed: {}, filtered: {}", + log.debug("Pattern consumer [{}] get topics under namespace {}, topics.size: {}," + + " topicsHash: {}, changed: {}, filtered: {}", conf.getSubscriptionName(), namespaceName, getTopicsResult.getTopics().size(), getTopicsResult.getTopicsHash(), getTopicsResult.isChanged(), getTopicsResult.isFiltered()); getTopicsResult.getTopics().forEach(topicName -> - log.debug("Get topics under namespace {}, topic: {}", namespaceName, topicName)); + log.debug("Pattern consumer [{}] get topics under namespace {}, topic: {}", + conf.getSubscriptionName(), namespaceName, topicName)); } List topicsList = getTopicsResult.getTopics(); @@ -598,6 +599,12 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo topicsList = TopicList.filterTopics(getTopicsResult.getTopics(), pattern); } conf.getTopicNames().addAll(topicsList); + + if (log.isInfoEnabled()) { + log.info("Pattern consumer [{}] initialize topics. {}", conf.getSubscriptionName(), + getTopicsResult.getNonPartitionedOrPartitionTopics()); + } + // Pattern consumer has his unique check mechanism, so do not need the feature "autoUpdatePartitions". conf.setAutoUpdatePartitions(false); ConsumerBase consumer = new PatternMultiTopicsConsumerImpl<>(pattern, diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java index 116a69b63e4ec..3dfb23f31954a 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImplTest.java @@ -61,7 +61,7 @@ public void testChangedUnfilteredResponse() { "persistent://tenant/my-ns/non-matching"), null, false, true), mockListener, - Collections.emptyList()); + Collections.emptyList(), ""); verify(mockListener).onTopicsAdded(Sets.newHashSet( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); @@ -80,7 +80,7 @@ public void testChangedFilteredResponse() { "persistent://tenant/my-ns/name-2"), "TOPICS_HASH", true, true), mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); + Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener).onTopicsAdded(Sets.newHashSet( "persistent://tenant/my-ns/name-1", "persistent://tenant/my-ns/name-2")); @@ -99,7 +99,7 @@ public void testUnchangedResponse() { "persistent://tenant/my-ns/name-2"), "TOPICS_HASH", true, false), mockListener, - Arrays.asList("persistent://tenant/my-ns/name-0")); + Arrays.asList("persistent://tenant/my-ns/name-0"), ""); verify(mockListener, never()).onTopicsAdded(any()); verify(mockListener, never()).onTopicsRemoved(any()); verify(mockTopicsHashSetter).accept("TOPICS_HASH"); From d41c62c1b7770b36d1f00f682e481f3a7d35751f Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 4 Jul 2024 00:06:31 +0800 Subject: [PATCH 24/30] checkstyle --- .../pulsar/client/impl/PatternTopicsConsumerImplTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java index 26a7bea3a8c6a..9c19fadffb137 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/PatternTopicsConsumerImplTest.java @@ -19,7 +19,6 @@ package org.apache.pulsar.client.impl; import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.spy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; @@ -35,7 +34,6 @@ import java.util.regex.Pattern; import java.util.stream.IntStream; -import io.netty.util.Timeout; import org.apache.pulsar.broker.BrokerTestUtil; import org.apache.pulsar.broker.namespace.NamespaceService; import org.apache.pulsar.client.api.Consumer; From af01c4f27af19a95f75c60360ea78e961c487eae Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 4 Jul 2024 16:01:17 +0800 Subject: [PATCH 25/30] fix test --- .../client/impl/PatternMultiTopicsConsumerImpl.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index f4584371df1a6..63a7030b66ac2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -197,14 +197,10 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, final List> listenersCallback = new ArrayList<>(2); Set topicsAdded = TopicList.minus(newTopics, oldTopics); Set topicsRemoved = TopicList.minus(oldTopics, newTopics); - log.info("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: []", + log.info("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: {}", subscriptionForLog, topicsAdded, topicsRemoved); - if (!topicsAdded.isEmpty()) { - listenersCallback.add(topicsChangedListener.onTopicsAdded(topicsAdded)); - } - if (!topicsRemoved.isEmpty()) { - listenersCallback.add(topicsChangedListener.onTopicsRemoved(topicsRemoved)); - } + listenersCallback.add(topicsChangedListener.onTopicsAdded(topicsAdded)); + listenersCallback.add(topicsChangedListener.onTopicsRemoved(topicsRemoved)); return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); } From 71648f9c6340b3b99dbdef102afbca36c10bbe61 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 4 Jul 2024 18:13:15 +0800 Subject: [PATCH 26/30] Add conflict topics check --- .../impl/PatternMultiTopicsConsumerImpl.java | 50 +++++++++++++------ 1 file changed, 36 insertions(+), 14 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 63a7030b66ac2..9ffc43c09ef9d 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -245,22 +245,23 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) Set partialRemoved = new HashSet<>(removedTopics.size()); Set partialRemovedForLog = new HashSet<>(removedTopics.size()); for (String tp : removedTopics) { - ConsumerImpl consumer = consumers.get(tp); + TopicName topicName = TopicName.get(tp); + ConsumerImpl consumer = consumers.get(topicName.toString()); if (consumer != null) { CompletableFuture unsubscribeFuture = new CompletableFuture<>(); consumer.closeAsync().whenComplete((__, ex) -> { if (ex != null) { log.error("Pattern consumer [{}] failed to unsubscribe from topics: {}", - PatternMultiTopicsConsumerImpl.this.getSubscription(), tp, ex); + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName.toString(), ex); unsubscribeFuture.completeExceptionally(ex); } else { - consumers.remove(tp, consumer); + consumers.remove(topicName.toString(), consumer); unsubscribeFuture.complete(null); } }); unsubscribeList.add(unsubscribeFuture); - partialRemoved.add(TopicName.get(tp).getPartitionedTopicName()); - partialRemovedForLog.add(TopicName.get(tp).toString()); + partialRemoved.add(topicName.getPartitionedTopicName()); + partialRemovedForLog.add(topicName.toString()); } } log.info("Pattern consumer [{}] remove topics. {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), @@ -304,11 +305,17 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { } List> futures = Lists.newArrayListWithExpectedSize(addedTopics.size()); /** - * Three cases: + * Three normal cases: * 1. Expand partitions. * 2. Non-partitioned topic, but has been subscribing. * 3. Non-partitioned topic or Partitioned topic, but has not been subscribing. - * Note: The events that triggered by {@link TopicsPartitionChangedListener} after expanding partitions has + * Two unexpected cases: + * Error-1: Received adding non-partitioned topic event, but has subscribed a partitioned topic with the + * same name. + * Error-2: Received adding partitioned topic event, but has subscribed a non-partitioned topic with the + * same name. + * + * Note: The events that triggered by {@link TopicsPartitionChangedListener} after expanding partitions has * been disabled through "conf.setAutoUpdatePartitions(false)" when creating * {@link PatternMultiTopicsConsumerImpl}. */ @@ -322,31 +329,46 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { TopicName topicName = TopicName.get(tp); // Case 1: Expand partitions. if (partitionedTopics.containsKey(topicName.getPartitionedTopicName())) { - if (consumers.containsKey(tp.toString())) { - continue; + if (consumers.containsKey(topicName.toString())) { + // Already subscribed. + } else if (topicName.getPartitionIndex() < 0) { + // Error-1: Received adding non-partitioned topic event, but has subscribed a partitioned topic + // with the same name. + log.error("Pattern consumer [{}] skip to subscribe to the non-partitioned topic {}, because has" + + "subscribed a partitioned topic with the same name", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName.toString()); } else { if (topicName.getPartitionIndex() + 1 > partitionedTopics.get(topicName.getPartitionedTopicName())) { partitionedTopics.put(topicName.getPartitionedTopicName(), topicName.getPartitionIndex() + 1); } - expendPartitionsForLog.add(tp); - CompletableFuture consumerFuture = subscribeAsync(tp, PartitionedTopicMetadata.NON_PARTITIONED); + expendPartitionsForLog.add(topicName.toString()); + CompletableFuture consumerFuture = subscribeAsync(topicName.toString(), + PartitionedTopicMetadata.NON_PARTITIONED); consumerFuture.whenComplete((__, ex) -> { if (ex != null) { log.warn("Pattern consumer [{}] Failed to subscribe to topics: {}", - PatternMultiTopicsConsumerImpl.this.getSubscription(), tp, ex); + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName, ex); } }); futures.add(consumerFuture); } groupedTopics.remove(topicName.getPartitionedTopicName()); - } else if (consumers.containsKey(tp.toString())) { + } else if (consumers.containsKey(topicName.toString())) { // Case-2: Non-partitioned topic, but has been subscribing. groupedTopics.remove(topicName.getPartitionedTopicName()); + } else if (consumers.containsKey(topicName.getPartitionedTopicName()) + && topicName.getPartitionIndex() >= 0) { + // Error-2: Received adding partitioned topic event, but has subscribed a non-partitioned topic + // with the same name. + log.error("Pattern consumer [{}] skip to subscribe to the partitioned topic {}, because has" + + "subscribed a non-partitioned topic with the same name", + PatternMultiTopicsConsumerImpl.this.getSubscription(), topicName); + groupedTopics.remove(topicName.getPartitionedTopicName()); } } - // Case 3: Non-partitioned topic or Partitioned topic, but has not been subscribing. + // Case 3: Non-partitioned topic or Partitioned topic, which has not been subscribed. for (String partitionedTopic : groupedTopics) { CompletableFuture consumerFuture = subscribeAsync(partitionedTopic, false); consumerFuture.whenComplete((__, ex) -> { From 61cd302135d6b7085978e028aa3f9615b7b0e426 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 4 Jul 2024 22:35:06 +0800 Subject: [PATCH 27/30] revert unnecessary change --- .../java/org/apache/pulsar/client/impl/LookupServiceTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java index 5d5066db7aa86..59cb7ae03d0e3 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/LookupServiceTest.java @@ -108,8 +108,8 @@ public void testGetTopicsOfGetTopicsResult(boolean isUsingHttpLookup) throws Exc // Verify the new method "GetTopicsResult.nonPartitionedOrPartitionTopics" works as expected. Collection nonPartitionedOrPartitionTopics = lookupService.getTopicsUnderNamespace(NamespaceName.get("public/default"), - Mode.PERSISTENT, "public/default/.*", null).join() - .getNonPartitionedOrPartitionTopics(); + Mode.PERSISTENT, "public/default/.*", null).join() + .getNonPartitionedOrPartitionTopics(); assertTrue(nonPartitionedOrPartitionTopics.contains(nonPartitionedTopic)); assertFalse(nonPartitionedOrPartitionTopics.contains(partitionedTopic)); assertFalse(nonPartitionedOrPartitionTopics.contains(nonPersistentTopic)); From fb8a84a40904cbe71606dc2c18d465f6b7a2278b Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Thu, 4 Jul 2024 22:38:25 +0800 Subject: [PATCH 28/30] change log level to debug --- .../impl/PatternMultiTopicsConsumerImpl.java | 27 ++++++++++++------- .../pulsar/client/impl/PulsarClientImpl.java | 4 +-- 2 files changed, 20 insertions(+), 11 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 9ffc43c09ef9d..98cb1750774f4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -197,8 +197,10 @@ static CompletableFuture updateSubscriptions(Pattern topicsPattern, final List> listenersCallback = new ArrayList<>(2); Set topicsAdded = TopicList.minus(newTopics, oldTopics); Set topicsRemoved = TopicList.minus(oldTopics, newTopics); - log.info("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: {}", - subscriptionForLog, topicsAdded, topicsRemoved); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] Recheck pattern consumer's topics. topicsAdded: {}, topicsRemoved: {}", + subscriptionForLog, topicsAdded, topicsRemoved); + } listenersCallback.add(topicsChangedListener.onTopicsAdded(topicsAdded)); listenersCallback.add(topicsChangedListener.onTopicsRemoved(topicsRemoved)); return FutureUtil.waitForAll(Collections.unmodifiableList(listenersCallback)); @@ -264,8 +266,11 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) partialRemovedForLog.add(topicName.toString()); } } - log.info("Pattern consumer [{}] remove topics. {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), - partialRemovedForLog); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] remove topics. {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), + partialRemovedForLog); + } // Remove partitioned topics in memory. return FutureUtil.waitForAll(unsubscribeList).handle((__, ex) -> { @@ -288,9 +293,11 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) } } } - log.info("Pattern consumer [{}] remove partitioned topics because all partitions have been removed. {}", - PatternMultiTopicsConsumerImpl.this.getSubscription(), - removedPartitionedTopicsForLog); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] remove partitioned topics because all partitions have been removed." + + " {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), + removedPartitionedTopicsForLog); + } return null; }); } @@ -379,8 +386,10 @@ public CompletableFuture onTopicsAdded(Collection addedTopics) { }); futures.add(consumerFuture); } - log.info("Pattern consumer [{}] add topics. expend partitions {}, new subscribing {}", - PatternMultiTopicsConsumerImpl.this.getSubscription(), expendPartitionsForLog, groupedTopics); + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] add topics. expend partitions {}, new subscribing {}", + PatternMultiTopicsConsumerImpl.this.getSubscription(), expendPartitionsForLog, groupedTopics); + } return FutureUtil.waitForAll(futures); } } diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java index df3c94ee70e92..120bdeb569c69 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PulsarClientImpl.java @@ -600,8 +600,8 @@ private CompletableFuture> patternTopicSubscribeAsync(ConsumerCo } conf.getTopicNames().addAll(topicsList); - if (log.isInfoEnabled()) { - log.info("Pattern consumer [{}] initialize topics. {}", conf.getSubscriptionName(), + if (log.isDebugEnabled()) { + log.debug("Pattern consumer [{}] initialize topics. {}", conf.getSubscriptionName(), getTopicsResult.getNonPartitionedOrPartitionTopics()); } From f279c6f623b171ff1a2ac5511133c8555e3bb4a3 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Fri, 5 Jul 2024 08:18:50 +0800 Subject: [PATCH 29/30] checkstyle --- .../pulsar/client/impl/PatternMultiTopicsConsumerImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java index 98cb1750774f4..70ba3e33963f4 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/PatternMultiTopicsConsumerImpl.java @@ -294,8 +294,8 @@ public CompletableFuture onTopicsRemoved(Collection removedTopics) } } if (log.isDebugEnabled()) { - log.debug("Pattern consumer [{}] remove partitioned topics because all partitions have been removed." - + " {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), + log.debug("Pattern consumer [{}] remove partitioned topics because all partitions have been" + + " removed. {}", PatternMultiTopicsConsumerImpl.this.getSubscription(), removedPartitionedTopicsForLog); } return null; From 8a7bafa656b69d47ccf6815dba5d650a1b077367 Mon Sep 17 00:00:00 2001 From: fengyubiao Date: Mon, 8 Jul 2024 22:48:06 +0800 Subject: [PATCH 30/30] address comments --- .../org/apache/pulsar/client/impl/LookupService.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java index ccd1f6b23f2f3..2fe457059c1e9 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/LookupService.java @@ -117,7 +117,14 @@ CompletableFuture getPartitionedTopicMetadata(TopicNam InetSocketAddress resolveHost(); /** - * Returns all the topics name for a given namespace. + * Returns all the topics that matches {@param topicPattern} for a given namespace. + * + * Note: {@param topicPattern} it relate to the topic name(without the partition suffix). For example: + * - There is a partitioned topic "tp-a" with two partitions. + * - tp-a-partition-0 + * - tp-a-partition-1 + * - If {@param topicPattern} is "tp-a", the consumer will subscribe to the two partitions. + * - if {@param topicPattern} is "tp-a-partition-0", the consumer will subscribe nothing. * * @param namespace : namespace-name * @return