Skip to content
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 @@ -245,8 +245,7 @@ public void findEntryComplete(Position position, Object ctx) {
.attr("position", position)
.log("Expiring all messages until position");
Position prevMarkDeletePos = cursor.getMarkDeletedPosition();
cursor.asyncMarkDelete(position, cursor.getProperties(), markDeleteCallback,
cursor.getNumberOfEntriesInBacklog(false));
cursor.asyncMarkDelete(position, null, markDeleteCallback, cursor.getNumberOfEntriesInBacklog(false));
if (!Objects.equals(cursor.getMarkDeletedPosition(), prevMarkDeletePos) && subscription != null) {
subscription.updateLastMarkDeleteAdvancedTimestamp();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.pulsar.broker.service;

import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.nullable;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
Expand All @@ -34,11 +35,14 @@
import io.netty.buffer.UnpooledByteBufAllocator;
import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
Expand Down Expand Up @@ -1098,4 +1102,58 @@ public void testGetFindPositionRange_SingleClosedLedger() {
assertNull(range.getRight());
assertEquals(range.getLeft(), PositionFactory.create(1, 9));
}

@Test
@SuppressWarnings("unchecked")
void testExpireMessagesNeverLoseMarkDeleteProperties() throws Exception {
final String ledgerAndCursorName = "testExpireMessagesNeverLoseMarkDeleteProperties";

ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setRetentionSizeInMB(10);
config.setRetentionTime(1, TimeUnit.HOURS);
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config);
ManagedCursorImpl cursor = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName);
ManagedCursorImpl spyCursor = spy(cursor);

Position pos1 = ledger.addEntry(createMessageWrittenToLedger("msg-1"));
Position pos2 = ledger.addEntry(createMessageWrittenToLedger("msg-2"));

CountDownLatch expiryMarkDeleteEnteredLatch = new CountDownLatch(1);
CountDownLatch cursorMarkDeleteCompletedLatch = new CountDownLatch(1);
CountDownLatch expiryMarkDeleteCompletedLatch = new CountDownLatch(1);

doAnswer(invocation -> {
Map<String, Long> invocationProperties = invocation.getArgument(1);
// Pause the expiry-triggered mark-delete so the user markDelete() can complete first.
if (invocationProperties == null || invocationProperties.isEmpty()) {
expiryMarkDeleteEnteredLatch.countDown();
assertTrue(cursorMarkDeleteCompletedLatch.await(5, TimeUnit.SECONDS));
try {
return invocation.callRealMethod();
} finally {
expiryMarkDeleteCompletedLatch.countDown();
}
}

return invocation.callRealMethod();
}).when(spyCursor)
.asyncMarkDelete(any(Position.class), nullable(Map.class), any(AsyncCallbacks.MarkDeleteCallback.class),
nullable(Object.class));

PersistentTopic topic = mockPersistentTopic("topicname");
PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(topic,
spyCursor.getName(), spyCursor, null);

CompletableFuture.runAsync(() -> monitor.findEntryComplete(pos2, null));
assertTrue(expiryMarkDeleteEnteredLatch.await(5, TimeUnit.SECONDS));

Map<String, Long> properties = new HashMap<>();
properties.put("test-property", 1L);
spyCursor.markDelete(pos1, properties);
cursorMarkDeleteCompletedLatch.countDown();

assertTrue(expiryMarkDeleteCompletedLatch.await(5, TimeUnit.SECONDS));
assertEquals(spyCursor.getMarkDeletedPosition(), pos2);
assertEquals(spyCursor.getProperties(), properties);
}
}
Loading