Skip to content

[fix] [broker] Fix negative subscription/consumer's unack-messages #24496

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

Open
wants to merge 2 commits 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 @@ -22,6 +22,8 @@
import java.util.List;
import java.util.Map;
import java.util.Optional;
import lombok.AllArgsConstructor;
import lombok.Data;
import org.apache.bookkeeper.common.annotation.InterfaceAudience;
import org.apache.bookkeeper.common.annotation.InterfaceStability;

Expand Down Expand Up @@ -118,6 +120,25 @@ interface DeleteCallback {
void deleteFailed(ManagedLedgerException exception, Object ctx);
}

interface CursorDeleteCallback extends DeleteCallback {
void deleteComplete(Object ctx, List<PositionAckState> positionAckStates);
}

@AllArgsConstructor
@Data
class PositionAckState {
Position position;
BatchMsgAckResType batchMsgAckResType;
int batchMessageAckCount;
}

enum BatchMsgAckResType {
AckAllAtOnce,
FirstPartialAck,
PartialAck,
LatestPartialAck;
}

interface TerminateCallback {
void terminateComplete(Position lastCommittedPosition, Object ctx);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,9 @@
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.DEFAULT_LEDGER_DELETE_RETRIES;
import static org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl.createManagedLedgerException;
import static org.apache.bookkeeper.mledger.util.Errors.isNoSuchLedgerExistsException;
import static org.apache.bookkeeper.mledger.AsyncCallbacks.BatchMsgAckResType;
import static org.apache.bookkeeper.mledger.AsyncCallbacks.DeleteCallback;
import static org.apache.bookkeeper.mledger.AsyncCallbacks.PositionAckState;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.MoreObjects;
import com.google.common.collect.Collections2;
Expand Down Expand Up @@ -2351,7 +2354,7 @@ public void deleteFailed(ManagedLedgerException exception, Object ctx) {


@Override
public void asyncDelete(Iterable<Position> positions, AsyncCallbacks.DeleteCallback callback, Object ctx) {
public void asyncDelete(Iterable<Position> positions, DeleteCallback callback, Object ctx) {
if (isClosed()) {
callback.deleteFailed(new ManagedLedgerException
.CursorAlreadyClosedException("Cursor was already closed"), ctx);
Expand All @@ -2362,12 +2365,12 @@ public void asyncDelete(Iterable<Position> positions, AsyncCallbacks.DeleteCallb

lock.writeLock().lock();
boolean skipMarkDeleteBecauseAckedNothing = false;
List<PositionAckState> positionAckNotices = new ArrayList<>();
try {
if (log.isDebugEnabled()) {
log.debug("[{}] [{}] Deleting individual messages at {}. Current status: {} - md-position: {}",
ledger.getName(), name, positions, individualDeletedMessages, markDeletePosition);
}

for (Position pos : positions) {
Position position = requireNonNull(pos);
if (ledger.getLastConfirmedEntry().compareTo(position) < 0) {
Expand All @@ -2391,8 +2394,12 @@ public void asyncDelete(Iterable<Position> positions, AsyncCallbacks.DeleteCallb
}
long[] ackSet = AckSetStateUtil.getAckSetArrayOrNull(position);
if (ackSet == null || ackSet.length == 0) {
if (batchDeletedIndexes != null) {
batchDeletedIndexes.remove(position);
BitSet bitSet;
if (batchDeletedIndexes == null || (bitSet = batchDeletedIndexes.remove(position)) == null) {
positionAckNotices.add(new PositionAckState(position, BatchMsgAckResType.AckAllAtOnce, -1));
} else {
positionAckNotices.add(new PositionAckState(position, BatchMsgAckResType.LatestPartialAck,
bitSet.cardinality()));
}
// Add a range (prev, pos] to the set. Adding the previous entry as an open limit to the range will
// make the RangeSet recognize the "continuity" between adjacent Positions.
Expand All @@ -2413,18 +2420,27 @@ public void asyncDelete(Iterable<Position> positions, AsyncCallbacks.DeleteCallb
individualDeletedMessages);
}
} else if (batchDeletedIndexes != null) {
final var givenBitSet = BitSet.valueOf(ackSet);
final var bitSet = batchDeletedIndexes.computeIfAbsent(position, __ -> givenBitSet);
if (givenBitSet != bitSet) {
bitSet.and(givenBitSet);
}
if (bitSet.isEmpty()) {
Position previousPosition = ledger.getPreviousPosition(position);
individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(),
previousPosition.getEntryId(),
position.getLedgerId(), position.getEntryId());
MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this);
batchDeletedIndexes.remove(position);
final var ackingBitSet = BitSet.valueOf(ackSet);
final var combinedBitSet = batchDeletedIndexes.computeIfAbsent(position, __ -> ackingBitSet);
if (ackingBitSet != combinedBitSet) {
int unAckedBefore = combinedBitSet.cardinality();
combinedBitSet.and(ackingBitSet);
int unAckedAfter = combinedBitSet.cardinality();
if (combinedBitSet.isEmpty()) {
Position previousPosition = ledger.getPreviousPosition(position);
individualDeletedMessages.addOpenClosed(previousPosition.getLedgerId(),
previousPosition.getEntryId(),
position.getLedgerId(), position.getEntryId());
MSG_CONSUMED_COUNTER_UPDATER.incrementAndGet(this);
batchDeletedIndexes.remove(position);
positionAckNotices.add(new PositionAckState(position, BatchMsgAckResType.LatestPartialAck,
unAckedBefore - unAckedAfter));
} else {
positionAckNotices.add(new PositionAckState(position, BatchMsgAckResType.PartialAck,
unAckedBefore - unAckedAfter));
}
} else {
positionAckNotices.add(new PositionAckState(position, BatchMsgAckResType.FirstPartialAck, -1));
}
}
}
Expand Down Expand Up @@ -2478,15 +2494,15 @@ public void asyncDelete(Iterable<Position> positions, AsyncCallbacks.DeleteCallb
} finally {
lock.writeLock().unlock();
if (skipMarkDeleteBecauseAckedNothing) {
callback.deleteComplete(ctx);
completeDeleteCallback(callback, ctx, positionAckNotices);
}
}

// Apply rate limiting to mark-delete operations
if (markDeleteLimiter != null && !markDeleteLimiter.tryAcquire()) {
isDirty = true;
updateLastMarkDeleteEntryToLatest(newMarkDeletePosition, null);
callback.deleteComplete(ctx);
completeDeleteCallback(callback, ctx, positionAckNotices);
return;
}

Expand All @@ -2497,7 +2513,7 @@ public void asyncDelete(Iterable<Position> positions, AsyncCallbacks.DeleteCallb
internalAsyncMarkDelete(newMarkDeletePosition, properties, new MarkDeleteCallback() {
@Override
public void markDeleteComplete(Object ctx) {
callback.deleteComplete(ctx);
completeDeleteCallback(callback, ctx, positionAckNotices);
}

@Override
Expand All @@ -2517,6 +2533,15 @@ public void markDeleteFailed(ManagedLedgerException exception, Object ctx) {
}
}

private void completeDeleteCallback(DeleteCallback deleteCallback, Object ctx,
List<PositionAckState> positionAckStates) {
if (deleteCallback instanceof AsyncCallbacks.CursorDeleteCallback cursorDeleteCallback) {
cursorDeleteCallback.deleteComplete(ctx, positionAckStates);
} else {
deleteCallback.deleteComplete(ctx);
}
}

// update lastMarkDeleteEntry field if newPosition is later than the current lastMarkDeleteEntry.newPosition
private void updateLastMarkDeleteEntryToLatest(final Position newPosition,
final Map<String, Long> properties) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,7 @@ public int filterEntriesForConsumer(@Nullable MessageMetadata[] metadataArray, i

private void individualAcknowledgeMessageIfNeeded(List<Position> positions, Map<String, Long> properties) {
if (!(subscription instanceof PulsarCompactorSubscription)) {
subscription.acknowledgeMessage(positions, AckType.Individual, properties);
subscription.acknowledgeMessage(positions, AckType.Individual, properties, null, false);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -549,7 +549,7 @@ public CompletableFuture<Void> messageAcked(CommandAck ack) {
.thenApply(unused -> 1L);
} else {
List<Position> positionsAcked = Collections.singletonList(position);
subscription.acknowledgeMessage(positionsAcked, AckType.Cumulative, properties);
subscription.acknowledgeMessage(positionsAcked, AckType.Cumulative, properties, this, false);
future = CompletableFuture.completedFuture(1L);
}
} else {
Expand All @@ -562,74 +562,47 @@ public CompletableFuture<Void> messageAcked(CommandAck ack) {

return future
.thenApply(v -> {
this.messageAckRate.recordEvent(v);
this.messageAckCounter.add(v);
// The case that is typed individual ack without transaction will deal metrics after a callback
// that after cursor deleting positions, so we may receive a 0 value here.
ackMetricRecord(v);
return null;
});
}

public void ackMetricRecord(long msgCountAcked) {
if (msgCountAcked > 0) {
this.messageAckRate.recordEvent(msgCountAcked);
this.messageAckCounter.add(msgCountAcked);
}
}

//this method is for individual ack not carry the transaction
private CompletableFuture<Long> individualAckNormal(CommandAck ack, Map<String, Long> properties) {
List<Pair<Consumer, Position>> positionsAcked = new ArrayList<>();
long totalAckCount = 0;
List<Position> positionsAcked = new ArrayList<>();
for (int i = 0; i < ack.getMessageIdsCount(); i++) {
MessageIdData msgId = ack.getMessageIdAt(i);
Position position;
ObjectIntPair<Consumer> ackOwnerConsumerAndBatchSize =
getAckOwnerConsumerAndBatchSize(msgId.getLedgerId(), msgId.getEntryId());
Consumer ackOwnerConsumer = ackOwnerConsumerAndBatchSize.left();
long ackedCount;
int batchSize = ackOwnerConsumerAndBatchSize.rightInt();
if (msgId.getAckSetsCount() > 0) {
long[] ackSets = new long[msgId.getAckSetsCount()];
for (int j = 0; j < msgId.getAckSetsCount(); j++) {
ackSets[j] = msgId.getAckSetAt(j);
}
position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), ackSets);
ackedCount = getAckedCountForBatchIndexLevelEnabled(position, batchSize, ackSets, ackOwnerConsumer);
if (isTransactionEnabled()) {
//sync the batch position bit set point, in order to delete the position in pending acks
if (Subscription.isIndividualAckMode(subType)) {
((PersistentSubscription) subscription)
.syncBatchPositionBitSetForPendingAck(position);
((PersistentSubscription) subscription).syncBatchPositionBitSetForPendingAck(position);
}
}
addAndGetUnAckedMsgs(ackOwnerConsumer, -(int) ackedCount);
} else {
position = PositionFactory.create(msgId.getLedgerId(), msgId.getEntryId());
ackedCount = getAckedCountForMsgIdNoAckSets(batchSize, position, ackOwnerConsumer);
if (checkCanRemovePendingAcksAndHandle(ackOwnerConsumer, position, msgId)) {
addAndGetUnAckedMsgs(ackOwnerConsumer, -(int) ackedCount);
updateBlockedConsumerOnUnackedMsgs(ackOwnerConsumer);
}
position = AckSetStateUtil.createPositionWithAckSet(msgId.getLedgerId(), msgId.getEntryId(), null);
}

positionsAcked.add(Pair.of(ackOwnerConsumer, position));
positionsAcked.add(position);

checkAckValidationError(ack, position);

totalAckCount += ackedCount;
}
subscription.acknowledgeMessage(positionsAcked.stream()
.map(Pair::getRight)
.collect(Collectors.toList()), AckType.Individual, properties);
CompletableFuture<Long> completableFuture = new CompletableFuture<>();
completableFuture.complete(totalAckCount);
if (isTransactionEnabled() && Subscription.isIndividualAckMode(subType)) {
completableFuture.whenComplete((v, e) -> positionsAcked.forEach(positionPair -> {
Consumer ackOwnerConsumer = positionPair.getLeft();
Position position = positionPair.getRight();
//check if the position can remove from the consumer pending acks.
// the bit set is empty in pending ack handle.
if (AckSetStateUtil.hasAckSet(position)) {
if (((PersistentSubscription) subscription)
.checkIsCanDeleteConsumerPendingAck(position)) {
removePendingAcks(ackOwnerConsumer, position);
}
}
}));
}
return completableFuture;
subscription.acknowledgeMessage(positionsAcked, AckType.Individual, properties, this, false);
return CompletableFuture.completedFuture(0L);
}


Expand Down Expand Up @@ -745,6 +718,7 @@ private long getAckedCountForTransactionAck(int batchSize, long[] ackSets) {
}

private long getUnAckedCountForBatchIndexLevelEnabled(Position position, int batchSize) {
// TODO compare with the cursor.
long unAckedCount = batchSize;
if (isAcknowledgmentAtBatchIndexLevelEnabled) {
long[] cursorAckSet = getCursorAckSet(position);
Expand Down Expand Up @@ -1194,11 +1168,24 @@ public Subscription getSubscription() {
return subscription;
}

private int addAndGetUnAckedMsgs(Consumer consumer, int ackedMessages) {
public int addAndGetUnAckedMsgs(Consumer consumer, int ackedMessages) {
int unackedMsgs = 0;
if (isPersistentTopic && Subscription.isIndividualAckMode(subType)) {
subscription.addUnAckedMessages(ackedMessages);
unackedMsgs = UNACKED_MESSAGES_UPDATER.addAndGet(consumer, ackedMessages);
if (log.isDebugEnabled()) {
if (ackedMessages > 0) {
log.debug("[{}][{}]{}-{}-{} delivered out {} messages, un-ack-msg: {}",
topicName, consumer.subscription.getName(),
consumer.cnx(), consumer.consumerId(), consumer.consumerName(),
ackedMessages, consumer.getUnackedMessages());
} else {
log.debug("[{}][{}]{}-{}-{} acknowledged/redelivered {} messages, un-ack-msg: {}",
topicName, consumer.subscription.getName(),
consumer.cnx(), consumer.consumerId(), consumer.consumerName(),
-ackedMessages, consumer.getUnackedMessages());
}
}
}
if (unackedMsgs < 0 && System.currentTimeMillis() - negativeUnackedMsgsTimestamp >= 10_000) {
negativeUnackedMsgsTimestamp = System.currentTimeMillis();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -304,11 +304,15 @@ public boolean remove(long ledgerId, long entryId, int batchSize, int stickyKeyH
* @return true if the pending ack was removed, false otherwise
*/
public boolean remove(long ledgerId, long entryId) {
return removeAndReturn(ledgerId, entryId) != null;
}

public IntIntPair removeAndReturn(long ledgerId, long entryId) {
try {
writeLock.lock();
Long2ObjectSortedMap<IntIntPair> ledgerMap = pendingAcks.get(ledgerId);
if (ledgerMap == null) {
return false;
return null;
}
IntIntPair removedEntry = ledgerMap.remove(entryId);
boolean removed = removedEntry != null;
Expand All @@ -319,7 +323,7 @@ public boolean remove(long ledgerId, long entryId) {
if (removed && ledgerMap.isEmpty()) {
pendingAcks.remove(ledgerId);
}
return removed;
return removedEntry;
} finally {
writeLock.unlock();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.pulsar.common.api.proto.CommandAck.AckType;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
import org.apache.pulsar.common.api.proto.ReplicatedSubscriptionsSnapshot;
import org.jspecify.annotations.Nullable;

public interface Subscription extends MessageExpirer {

Expand All @@ -48,7 +49,13 @@ default void removeConsumer(Consumer consumer) throws BrokerServiceException {

void consumerFlow(Consumer consumer, int additionalNumberOfMessages);

void acknowledgeMessage(List<Position> positions, AckType ackType, Map<String, Long> properties);
/**
* @param ackFrom It can be null, and it will always be null if {@param ackType} is {@link AckType#Cumulative}.
* The performance will be improved, if this param is the owner consumer that received the messages
* who are being acked when {@param ackType} is {@link AckType#Individual}.
*/
void acknowledgeMessage(List<Position> positions, AckType ackType, Map<String, Long> properties,
@Nullable Consumer ackFrom, boolean triggeredByTxnCommit);

String getTopicName();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,8 @@ public void consumerFlow(Consumer consumer, int additionalNumberOfMessages) {
}

@Override
public void acknowledgeMessage(List<Position> position, AckType ackType, Map<String, Long> properties) {
public void acknowledgeMessage(List<Position> position, AckType ackType, Map<String, Long> properties,
Consumer ackFrom, boolean triggerByTxnCommit) {
// No-op
}

Expand Down
Loading
Loading