diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java index bd4917da3b119..3ab25eb098cdf 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/ServerCnx.java @@ -2178,7 +2178,8 @@ protected void handleGetLastMessageId(CommandGetLastMessageId getLastMessageId) (PositionImpl) markDeletePosition, partitionIndex, requestId, - consumer.getSubscription().getName()); + consumer.getSubscription().getName(), + consumer.readCompacted()); }).exceptionally(e -> { writeAndFlush(Commands.newError(getLastMessageId.getRequestId(), ServerError.UnknownError, "Failed to recover Transaction Buffer.")); @@ -2196,15 +2197,17 @@ private void getLargestBatchIndexWhenPossible( PositionImpl markDeletePosition, int partitionIndex, long requestId, - String subscriptionName) { + String subscriptionName, + boolean readCompacted) { PersistentTopic persistentTopic = (PersistentTopic) topic; ManagedLedgerImpl ml = (ManagedLedgerImpl) persistentTopic.getManagedLedger(); // If it's not pointing to a valid entry, respond messageId of the current position. // If the compaction cursor reach the end of the topic, respond messageId from compacted ledger - CompletableFuture compactionHorizonFuture = - persistentTopic.getTopicCompactionService().getLastCompactedPosition(); + CompletableFuture compactionHorizonFuture = readCompacted + ? persistentTopic.getTopicCompactionService().getLastCompactedPosition() : + CompletableFuture.completedFuture(null); compactionHorizonFuture.whenComplete((compactionHorizon, ex) -> { if (ex != null) { @@ -2213,8 +2216,22 @@ private void getLargestBatchIndexWhenPossible( return; } - if (lastPosition.getEntryId() == -1 || (compactionHorizon != null - && lastPosition.compareTo((PositionImpl) compactionHorizon) <= 0)) { + if (lastPosition.getEntryId() == -1 || !ml.ledgerExists(lastPosition.getLedgerId())) { + // there is no entry in the original topic + if (compactionHorizon != null) { + // if readCompacted is true, we need to read the last entry from compacted topic + handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, + markDeletePosition); + } else { + // if readCompacted is false, we need to return MessageId.earliest + writeAndFlush(Commands.newGetLastMessageIdResponse(requestId, -1, -1, partitionIndex, -1, + markDeletePosition != null ? markDeletePosition.getLedgerId() : -1, + markDeletePosition != null ? markDeletePosition.getEntryId() : -1)); + } + return; + } + + if (compactionHorizon != null && lastPosition.compareTo((PositionImpl) compactionHorizon) <= 0) { handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, markDeletePosition); return; @@ -2249,7 +2266,8 @@ public String toString() { batchSizeFuture.whenComplete((batchSize, e) -> { if (e != null) { - if (e.getCause() instanceof ManagedLedgerException.NonRecoverableLedgerException) { + if (e.getCause() instanceof ManagedLedgerException.NonRecoverableLedgerException + && readCompacted) { handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex, markDeletePosition); } else { diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java index 317b1a227e585..6c2d848bb7c2d 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/compaction/GetLastMessageIdCompactedTest.java @@ -20,6 +20,8 @@ import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotEquals; + import java.util.ArrayList; import java.util.List; import java.util.Optional; @@ -32,6 +34,7 @@ import org.apache.pulsar.broker.service.persistent.PersistentTopic; import org.apache.pulsar.client.api.CompressionType; import org.apache.pulsar.client.api.Consumer; +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.ProducerBuilder; @@ -415,4 +418,28 @@ public void testGetLastMessageIdAfterCompactionAllNullMsg(boolean enabledBatch) producer.close(); admin.topics().delete(topicName, false); } + + @Test(dataProvider = "enabledBatch") + public void testReaderStuckWithCompaction(boolean enabledBatch) throws Exception { + String topicName = "persistent://public/default/" + BrokerTestUtil.newUniqueName("tp"); + String subName = "sub"; + Producer producer = createProducer(enabledBatch, topicName); + producer.newMessage().key("k0").value("v0").sendAsync(); + producer.newMessage().key("k0").value("v1").sendAsync(); + producer.flush(); + + triggerCompactionAndWait(topicName); + triggerLedgerSwitch(topicName); + clearAllTheLedgersOutdated(topicName); + + var reader = pulsarClient.newReader(Schema.STRING) + .topic(topicName) + .subscriptionName(subName) + .startMessageId(MessageId.earliest) + .create(); + while (reader.hasMessageAvailable()) { + Message message = reader.readNext(5, TimeUnit.SECONDS); + assertNotEquals(message, null); + } + } }