Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[fix][broker] Fix consumer receive individual acknowledged messages from compacted topic after reconnection #23495

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
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 @@ -893,6 +893,8 @@ default ManagedCursorAttributes getManagedCursorAttributes() {

boolean isMessageDeleted(Position position);

boolean isMessageIndividualDeleted(Position position);

ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException;

long[] getBatchPositionAckSet(Position position);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3550,6 +3550,16 @@ public boolean isMessageDeleted(Position position) {
}
}

@Override
public boolean isMessageIndividualDeleted(Position position) {
lock.readLock().lock();
try {
return individualDeletedMessages.contains(position.getLedgerId(), position.getEntryId());
} finally {
lock.readLock().unlock();
}
}

//this method will return a copy of the position's ack set
@Override
public long[] getBatchPositionAckSet(Position position) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,4 +76,9 @@ public long getNumberOfEntries(Range<Position> range) {
public boolean isMessageDeleted(Position position) {
return false;
}

@Override
public boolean isMessageIndividualDeleted(Position position) {
return false;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -422,6 +422,11 @@ public boolean isMessageDeleted(Position position) {
return false;
}

@Override
public boolean isMessageIndividualDeleted(Position position) {
return false;
}

@Override
public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException {
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,9 @@
import static com.google.common.base.Preconditions.checkArgument;
import static org.apache.pulsar.common.protocol.Commands.DEFAULT_CONSUMER_EPOCH;
import com.google.common.annotations.Beta;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import javax.annotation.Nullable;
Expand All @@ -38,6 +40,8 @@

public class CompactedTopicUtils {

private static final String COMPACTION_CURSOR_NAME = "__compaction";

@Beta
public static void asyncReadCompactedEntries(TopicCompactionService topicCompactionService,
ManagedCursor cursor, int maxEntries,
Expand Down Expand Up @@ -88,15 +92,27 @@ public static void asyncReadCompactedEntries(TopicCompactionService topicCompact
return;
}

List<Entry> unAckedEntries = new ArrayList<>();
long entriesSize = 0;
for (Entry entry : entries) {
entriesSize += entry.getLength();
if (COMPACTION_CURSOR_NAME.equals(cursor.getName())) {
for (Entry entry : entries) {
entriesSize += entry.getLength();
}
unAckedEntries = entries;
} else {
for (Entry entry : entries) {
Position position = entry.getPosition();
if (!cursor.isMessageIndividualDeleted(position)) {
unAckedEntries.add(entry);
entriesSize += entry.getLength();
}
}
}
cursor.updateReadStats(entries.size(), entriesSize);
cursor.updateReadStats(unAckedEntries.size(), entriesSize);

Entry lastEntry = entries.get(entries.size() - 1);
cursor.seek(lastEntry.getPosition().getNext(), true);
callback.readEntriesComplete(entries, readEntriesCtx);
callback.readEntriesComplete(unAckedEntries, readEntriesCtx);
});
}).exceptionally((exception) -> {
exception = FutureUtil.unwrapCompletionException(exception);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -426,6 +426,11 @@ public boolean isMessageDeleted(Position position) {
return false;
}

@Override
public boolean isMessageIndividualDeleted(Position position) {
return false;
}

@Override
public ManagedCursor duplicateNonDurableCursor(String nonDurableCursorName) throws ManagedLedgerException {
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2364,4 +2364,77 @@ public void testEarliestSubsAfterRollover() throws Exception {

Assert.assertEquals(results, expected);
}

@Test
public void testIndividualAcknowledgeWithReconnection() throws Exception {
final String topicName = "persistent://my-property/use/my-ns/testIndividualAcknowledge" + UUID.randomUUID();
final String subName = "sub1";
final int numMessages = 10;

pulsarClient.newConsumer().topic(topicName).subscriptionName(subName)
.receiverQueueSize(1).readCompacted(true).subscribe().close();

Map<String, String> expected = new HashMap<>();

@Cleanup
Producer<String> producer = pulsarClient.newProducer(Schema.STRING)
.topic(topicName)
.enableBatching(false)
.messageRoutingMode(MessageRoutingMode.SinglePartition)
.create();

for (int i = 0; i < numMessages; i++) {
String key = "key" + new Random().nextInt(4);
String value = ("my-message-" + i);
producer.newMessage().key(key).value(value).send();
expected.put(key, value);
}
producer.flush();

// compact the topic
compact(topicName);

@Cleanup
Consumer<String> consumer = pulsarClient.newConsumer(Schema.STRING).topic(topicName)
.subscriptionInitialPosition(SubscriptionInitialPosition.Earliest)
.subscriptionName(subName).receiverQueueSize(1).readCompacted(true).subscribe();

Map<String, String> results = new HashMap<>();
while (true) {
Message<String> message = consumer.receive(3, TimeUnit.SECONDS);
if (message == null) {
break;
}
results.put(message.getKey(), message.getValue());
consumer.acknowledge(message);
}
Awaitility.await().untilAsserted(() ->
assertEquals(admin.topics().getStats(topicName, true).getSubscriptions().get(subName).getMsgBacklog(),
numMessages - expected.size()));

// unload the topic
admin.topics().unload(topicName);

// wait the consumer reconnect
Awaitility.await().until(() -> admin.topics().getStats(topicName).getSubscriptions() != null);

// should not receive message
int count = 0;
while (true) {
Message<String> message = consumer.receive(3, TimeUnit.SECONDS);
if (message == null) {
break;
}
count++;
results.put(message.getKey(), message.getValue());
consumer.acknowledge(message);
}
assertEquals(count, 0);

Awaitility.await().untilAsserted(() ->
assertEquals(admin.topics().getStats(topicName, true).getSubscriptions().get(subName).getMsgBacklog(),
numMessages - expected.size()));

assertEquals(results, expected);
}
}
Loading