From aec0bcecafa07e6f83f69d50edb6c50b860c4fe4 Mon Sep 17 00:00:00 2001 From: Cong Zhao Date: Fri, 26 Jan 2024 17:26:45 +0800 Subject: [PATCH] [fix][broker][branch-3.0] Avoid consumers receiving acknowledged messages from compacted topic after reconnection (#21187) --- .../bookkeeper/mledger/ManagedCursor.java | 4 + .../mledger/impl/ManagedCursorImpl.java | 10 +- ...sistentDispatcherSingleActiveConsumer.java | 12 +- .../service/persistent/PersistentTopic.java | 5 +- .../pulsar/compaction/CompactedTopicImpl.java | 6 +- .../service/ReplicatorSubscriptionTest.java | 2 + .../broker/transaction/TransactionTest.java | 1 + .../apache/pulsar/client/impl/ReaderTest.java | 28 +++ .../pulsar/compaction/CompactionTest.java | 164 +++++++++++++++++- 9 files changed, 218 insertions(+), 14 deletions(-) diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java index d1ffdf6d2d763..bc6a1e9a782d6 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedCursor.java @@ -517,6 +517,10 @@ void markDelete(Position position, Map properties) */ void rewind(); + default void rewind(boolean readCompacted) { + rewind(); + } + /** * Move the cursor to a different read position. * diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java index d0fdfe50165d3..fd6d921ab346e 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedCursorImpl.java @@ -677,7 +677,7 @@ private void recoveredCursor(PositionImpl position, Map properties LedgerHandle recoveredFromCursorLedger) { // if the position was at a ledger that didn't exist (since it will be deleted if it was previously empty), // we need to move to the next existing ledger - if (!ledger.ledgerExists(position.getLedgerId())) { + if (position.getEntryId() == -1L && !ledger.ledgerExists(position.getLedgerId())) { Long nextExistingLedger = ledger.getNextValidLedger(position.getLedgerId()); if (nextExistingLedger == null) { log.info("[{}] [{}] Couldn't find next next valid ledger for recovery {}", ledger.getName(), name, @@ -2518,9 +2518,15 @@ public Position getPersistentMarkDeletedPosition() { @Override public void rewind() { + rewind(false); + } + + @Override + public void rewind(boolean readCompacted) { lock.writeLock().lock(); try { - PositionImpl newReadPosition = ledger.getNextValidPosition(markDeletePosition); + PositionImpl newReadPosition = + readCompacted ? markDeletePosition.getNext() : ledger.getNextValidPosition(markDeletePosition); PositionImpl oldReadPosition = readPosition; log.info("[{}-{}] Rewind from {} to {}", ledger.getName(), name, oldReadPosition, newReadPosition); diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java index eacc568f0a48f..bf6482bda0129 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java @@ -105,9 +105,9 @@ protected void scheduleReadOnActiveConsumer() { if (log.isDebugEnabled()) { log.debug("[{}] Rewind cursor and read more entries without delay", name); } - cursor.rewind(); - Consumer activeConsumer = ACTIVE_CONSUMER_UPDATER.get(this); + cursor.rewind(activeConsumer != null && activeConsumer.readCompacted()); + notifyActiveConsumerChanged(activeConsumer); readMoreEntries(activeConsumer); return; @@ -125,9 +125,9 @@ protected void scheduleReadOnActiveConsumer() { log.debug("[{}] Rewind cursor and read more entries after {} ms delay", name, serviceConfig.getActiveConsumerFailoverDelayTimeMillis()); } - cursor.rewind(); - Consumer activeConsumer = ACTIVE_CONSUMER_UPDATER.get(this); + cursor.rewind(activeConsumer != null && activeConsumer.readCompacted()); + notifyActiveConsumerChanged(activeConsumer); readMoreEntries(activeConsumer); readOnActiveConsumerTask = null; @@ -198,7 +198,7 @@ public synchronized void internalReadEntriesComplete(final List entries, log.debug("[{}] rewind because no available consumer found", name); } entries.forEach(Entry::release); - cursor.rewind(); + cursor.rewind(currentConsumer != null ? currentConsumer.readCompacted() : readConsumer.readCompacted()); if (currentConsumer != null) { notifyActiveConsumerChanged(currentConsumer); readMoreEntries(currentConsumer); @@ -293,7 +293,7 @@ private synchronized void internalRedeliverUnacknowledgedMessages(Consumer consu } cursor.cancelPendingReadRequest(); havePendingRead = false; - cursor.rewind(); + cursor.rewind(consumer.readCompacted()); if (log.isDebugEnabled()) { log.debug("[{}-{}] Cursor rewinded, redelivering unacknowledged messages. ", name, consumer); } diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java index 3f75c02d48d4b..18485d27d336d 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentTopic.java @@ -975,7 +975,9 @@ public CompletableFuture subscribe(final TransportCnx cnx, String subs } private CompletableFuture getDurableSubscription(String subscriptionName, - InitialPosition initialPosition, long startMessageRollbackDurationSec, boolean replicated, + InitialPosition initialPosition, + long startMessageRollbackDurationSec, + boolean replicated, Map subscriptionProperties) { CompletableFuture subscriptionFuture = new CompletableFuture<>(); if (checkMaxSubscriptionsPerTopicExceed(subscriptionName)) { @@ -985,7 +987,6 @@ private CompletableFuture getDurableSubscription(String subscripti } Map properties = PersistentSubscription.getBaseCursorProperties(replicated); - ledger.asyncOpenCursor(Codec.encode(subscriptionName), initialPosition, properties, subscriptionProperties, new OpenCursorCallback() { @Override diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java index 2252065413559..b906653099867 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/compaction/CompactedTopicImpl.java @@ -96,7 +96,11 @@ public void asyncReadEntriesOrWait(ManagedCursor cursor, ReadEntriesCallback callback, Consumer consumer) { synchronized (this) { PositionImpl cursorPosition; - if (isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId())){ + boolean readFromEarliest = isFirstRead && MessageId.earliest.equals(consumer.getStartMessageId()) + && (!cursor.isDurable() || cursor.getName().equals(Compactor.COMPACTION_SUBSCRIPTION) + || cursor.getMarkDeletedPosition() == null + || cursor.getMarkDeletedPosition().getEntryId() == -1L); + if (readFromEarliest){ cursorPosition = PositionImpl.EARLIEST; } else { cursorPosition = (PositionImpl) cursor.getReadPosition(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java index fe519827be74a..4cc3a9ada7d04 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/ReplicatorSubscriptionTest.java @@ -52,6 +52,7 @@ import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.common.policies.data.PartitionedTopicStats; import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; @@ -868,6 +869,7 @@ public void testReplicatedSubscriptionWithCompaction() throws Exception { .topic(topicName) .subscriptionName("sub2") .subscriptionType(SubscriptionType.Exclusive) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .readCompacted(true) .subscribe(); List result = new ArrayList<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java index c947ba27069f1..1ad5ea14e0a02 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/transaction/TransactionTest.java @@ -1877,6 +1877,7 @@ public void testReadCommittedWithCompaction() throws Exception{ .topic(topic) .subscriptionName("sub") .subscriptionType(SubscriptionType.Exclusive) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .readCompacted(true) .subscribe(); List result = new ArrayList<>(); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java index 951f99af1a464..2f91d79258182 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/client/impl/ReaderTest.java @@ -733,4 +733,32 @@ public void testReaderListenerAcknowledgement() admin.topics().deletePartitionedTopic(partitionedTopic); } + @Test + public void testReaderReconnectedFromNextEntry() throws Exception { + final String topic = "persistent://my-property/my-ns/testReaderReconnectedFromNextEntry"; + Reader reader = pulsarClient.newReader(Schema.STRING).topic(topic).receiverQueueSize(1) + .startMessageId(MessageId.earliest).create(); + Producer producer = pulsarClient.newProducer(Schema.STRING).topic(topic).create(); + + // Send 3 and consume 1. + producer.send("1"); + producer.send("2"); + producer.send("3"); + Message msg1 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg1.getValue(), "1"); + + // Trigger reader reconnect. + admin.topics().unload(topic); + + // For non-durable we are going to restart from the next entry. + Message msg2 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg2.getValue(), "2"); + Message msg3 = reader.readNext(2, TimeUnit.SECONDS); + assertEquals(msg3.getValue(), "3"); + + // cleanup. + reader.close(); + producer.close(); + admin.topics().delete(topic, false); + } } diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java index 545dd97675c2d..d3de2187d959d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/CompactionTest.java @@ -1878,6 +1878,7 @@ public void testReceiverQueueSize() throws Exception { ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.STRING) .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); //Give some time to consume @@ -1915,6 +1916,7 @@ public void testDispatcherMaxReadSizeBytes() throws Exception { ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.BYTES) .topic(topicName).readCompacted(true).receiverQueueSize(receiveQueueSize).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) .subscribe(); PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getTopicReference(topicName).get(); @@ -2175,9 +2177,11 @@ public void testCompactionWithTTL() throws Exception { }); @Cleanup - Consumer consumer = - pulsarClient.newConsumer(Schema.STRING).topic(topicName).subscriptionName(subName).readCompacted(true) - .subscribe(); + Consumer consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName) + .subscriptionName("sub-2") + .readCompacted(true) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); List result = new ArrayList<>(); while (true) { @@ -2191,4 +2195,158 @@ public void testCompactionWithTTL() throws Exception { Assert.assertEquals(result, List.of("V3", "V4", "V5")); } + + @Test + public void testAcknowledgeWithReconnection() throws Exception { + final String topicName = "persistent://my-property/use/my-ns/testAcknowledge" + UUID.randomUUID(); + final String subName = "my-sub"; + @Cleanup + PulsarClient client = newPulsarClient(lookupUrl.toString(), 100); + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + List expected = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + producer.newMessage().key(String.valueOf(i)).value(String.valueOf(i)).send(); + expected.add(String.valueOf(i)); + } + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + // trim the topic + admin.topics().unload(topicName); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); + assertEquals(internalStats.numberOfEntries, 0); + }); + + ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.STRING) + .topic(topicName).readCompacted(true).receiverQueueSize(1).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .isAckReceiptEnabled(true) + .subscribe(); + + List results = new ArrayList<>(); + for (int i = 0; i < 5; i++) { + Message message = consumer.receive(3, TimeUnit.SECONDS); + if (message == null) { + break; + } + results.add(message.getValue()); + consumer.acknowledge(message); + } + + Awaitility.await().untilAsserted(() -> + assertEquals(admin.topics().getStats(topicName, true).getSubscriptions().get(subName).getMsgBacklog(), + 5)); + + // Make consumer reconnect to broker + admin.topics().unload(topicName); + + // Wait for consumer to reconnect and clear incomingMessages + consumer.pause(); + Awaitility.await().untilAsserted(() -> { + Assert.assertEquals(consumer.numMessagesInQueue(), 0); + }); + consumer.resume(); + + for (int i = 0; i < 5; i++) { + Message message = consumer.receive(3, TimeUnit.SECONDS); + if (message == null) { + break; + } + results.add(message.getValue()); + consumer.acknowledge(message); + } + + Awaitility.await().untilAsserted(() -> + assertEquals(admin.topics().getStats(topicName, true).getSubscriptions().get(subName).getMsgBacklog(), + 0)); + + Assert.assertEquals(results, expected); + + Message message = consumer.receive(3, TimeUnit.SECONDS); + Assert.assertNull(message); + + // Make consumer reconnect to broker + admin.topics().unload(topicName); + + producer.newMessage().key("K").value("V").send(); + Message message2 = consumer.receive(3, TimeUnit.SECONDS); + Assert.assertEquals(message2.getValue(), "V"); + consumer.acknowledge(message2); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName); + Assert.assertEquals(internalStats.lastConfirmedEntry, + internalStats.cursors.get(subName).markDeletePosition); + }); + + consumer.close(); + producer.close(); + } + + @Test + public void testEarliestSubsAfterRollover() throws Exception { + final String topicName = "persistent://my-property/use/my-ns/testEarliestSubsAfterRollover" + UUID.randomUUID(); + final String subName = "my-sub"; + @Cleanup + PulsarClient client = newPulsarClient(lookupUrl.toString(), 100); + @Cleanup + Producer producer = pulsarClient.newProducer(Schema.STRING) + .enableBatching(false).topic(topicName).create(); + + List expected = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + producer.newMessage().key(String.valueOf(i)).value(String.valueOf(i)).send(); + expected.add(String.valueOf(i)); + } + producer.flush(); + + admin.topics().triggerCompaction(topicName); + + Awaitility.await().untilAsserted(() -> { + assertEquals(admin.topics().compactionStatus(topicName).status, + LongRunningProcessStatus.Status.SUCCESS); + }); + + // trim the topic + admin.topics().unload(topicName); + + Awaitility.await().untilAsserted(() -> { + PersistentTopicInternalStats internalStats = admin.topics().getInternalStats(topicName, false); + assertEquals(internalStats.numberOfEntries, 0); + }); + + // Make ml.getFirstPosition() return new ledger first position + producer.newMessage().key("K").value("V").send(); + expected.add("V"); + + @Cleanup + ConsumerImpl consumer = (ConsumerImpl) client.newConsumer(Schema.STRING) + .topic(topicName).readCompacted(true).receiverQueueSize(1).subscriptionName(subName) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .isAckReceiptEnabled(true) + .subscribe(); + + List results = new ArrayList<>(); + while (true) { + Message message = consumer.receive(3, TimeUnit.SECONDS); + if (message == null) { + break; + } + + results.add(message.getValue()); + consumer.acknowledge(message); + } + + Assert.assertEquals(results, expected); + } }