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 @@ -1503,7 +1503,12 @@ protected void internalResetCursor(Position proposedReadPosition,

final AsyncCallbacks.ResetCursorCallback callback = resetCursorCallback;

final Position newMarkDeletePosition = ledger.getPreviousPosition(newReadPosition);
final Position newMarkDeletePosition;
if (isCompactionCursor()) {
newMarkDeletePosition = markDeletePosition;
} else {
newMarkDeletePosition = ledger.getPreviousPosition(newReadPosition);
}

Runnable alignAcknowledgeStatusAfterPersisted = () -> {
// Correct the variable "messagesConsumedCounter".
Expand Down Expand Up @@ -1593,7 +1598,6 @@ public void operationFailed(ManagedLedgerException exception) {

persistentMarkDeletePosition = null;
inProgressMarkDeletePersistPosition = null;
lastMarkDeleteEntry = new MarkDeleteEntry(newMarkDeletePosition, getProperties(), null, null);
internalAsyncMarkDelete(newMarkDeletePosition, isCompactionCursor() ? getProperties() : Collections.emptyMap(),
new MarkDeleteCallback() {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
*/
package org.apache.pulsar.compaction;

import com.google.common.annotations.VisibleForTesting;
import io.netty.buffer.ByteBuf;
import java.io.IOException;
import java.time.Duration;
Expand Down Expand Up @@ -59,6 +60,8 @@
*/
public abstract class AbstractTwoPhaseCompactor<T> extends Compactor {

@VisibleForTesting
static Runnable injectionAfterSeekInPhaseTwo = () -> {};
private static final Logger log = LoggerFactory.getLogger(AbstractTwoPhaseCompactor.class);
protected static final int MAX_OUTSTANDING = 500;
protected final Duration phaseOneLoopReadTimeout;
Expand Down Expand Up @@ -188,6 +191,7 @@ private CompletableFuture<Long> phaseTwoSeekThenLoop(RawReader reader, MessageId
CompletableFuture<Long> promise = new CompletableFuture<>();

reader.seekAsync(from).thenCompose((v) -> {
injectionAfterSeekInPhaseTwo.run();
Semaphore outstanding = new Semaphore(MAX_OUTSTANDING);
CompletableFuture<Void> loopPromise = new CompletableFuture<>();
phaseTwoLoop(reader, to, latestForKey, ledger, outstanding, loopPromise, MessageId.earliest);
Expand Down Expand Up @@ -436,4 +440,4 @@ protected static class PhaseOneResult<T> {
public long getPhaseOneLoopReadTimeoutInSeconds() {
return phaseOneLoopReadTimeout.getSeconds();
}
}
}
Loading
Loading