Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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."));
Expand All @@ -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<Position> compactionHorizonFuture =
persistentTopic.getTopicCompactionService().getLastCompactedPosition();
CompletableFuture<Position> compactionHorizonFuture = readCompacted
? persistentTopic.getTopicCompactionService().getLastCompactedPosition() :
CompletableFuture.completedFuture(null);

compactionHorizonFuture.whenComplete((compactionHorizon, ex) -> {
if (ex != null) {
Expand All @@ -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;
Comment thread
thetumbled marked this conversation as resolved.
}

if (compactionHorizon != null && lastPosition.compareTo((PositionImpl) compactionHorizon) <= 0) {
handleLastMessageIdFromCompactionService(persistentTopic, requestId, partitionIndex,
markDeletePosition);
return;
Expand Down Expand Up @@ -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 {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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<String> 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<String> message = reader.readNext(5, TimeUnit.SECONDS);
assertNotEquals(message, null);
}
}
}