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] Expire messages according to ledger close time to avoid client clock skew #21940

Merged
merged 15 commits into from
Feb 22, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import java.util.Objects;
import java.util.Optional;
import java.util.SortedMap;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import java.util.concurrent.atomic.LongAdder;
import javax.annotation.Nullable;
Expand All @@ -30,8 +31,10 @@
import org.apache.bookkeeper.mledger.ManagedLedgerException.LedgerNotExistException;
import org.apache.bookkeeper.mledger.ManagedLedgerException.NonRecoverableLedgerException;
import org.apache.bookkeeper.mledger.Position;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.mledger.proto.MLDataFormats;
import org.apache.pulsar.broker.service.MessageExpirer;
import org.apache.pulsar.client.impl.MessageImpl;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
Expand Down Expand Up @@ -78,7 +81,9 @@ public boolean expireMessages(int messageTTLInSeconds) {
if (expirationCheckInProgressUpdater.compareAndSet(this, FALSE, TRUE)) {
log.info("[{}][{}] Starting message expiry check, ttl= {} seconds", topicName, subName,
messageTTLInSeconds);

if (checkExpiryByLedgerClosureTime(cursor, messageTTLInSeconds)) {
return true;
}
cursor.asyncFindNewestMatching(ManagedCursor.FindPositionConstraint.SearchActiveEntries, entry -> {
try {
long entryTimestamp = Commands.getEntryTimestamp(entry.getDataBuffer());
Expand All @@ -100,6 +105,35 @@ public boolean expireMessages(int messageTTLInSeconds) {
}
}

private boolean checkExpiryByLedgerClosureTime(ManagedCursor cursor, int messageTTLInSeconds) {
if (cursor instanceof ManagedCursorImpl managedCursor) {
ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) managedCursor.getManagedLedger();
Position deletedPosition = managedCursor.getMarkDeletedPosition();
SortedMap<Long, MLDataFormats.ManagedLedgerInfo.LedgerInfo> ledgerInfoSortedMap =
managedLedger.getLedgersInfo()
.subMap(deletedPosition.getLedgerId(), managedLedger.getLedgersInfo().lastKey());
MLDataFormats.ManagedLedgerInfo.LedgerInfo info = null;
for (MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo : ledgerInfoSortedMap.values()) {
if (!ledgerInfo.hasTimestamp() || !MessageImpl.isEntryExpired(messageTTLInSeconds,
ledgerInfo.getTimestamp())) {
break;
}
info = ledgerInfo;
}
if (info != null && info.getLedgerId() > -1) {
PositionImpl position = PositionImpl.get(info.getLedgerId(), info.getEntries() - 1);
PositionImpl newPosition = managedLedger.getNextValidPosition(position);
315157973 marked this conversation as resolved.
Show resolved Hide resolved
if (((PositionImpl) managedLedger.getLastConfirmedEntry()).compareTo(newPosition) < 0) {
findEntryComplete(managedLedger.getLastConfirmedEntry(), null);
} else {
findEntryComplete(newPosition, null);
}
return true;
}
}
return false;
}

@Override
public boolean expireMessages(Position messagePosition) {
// If it's beyond last position of this topic, do nothing.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -81,8 +81,11 @@
public class PersistentMessageFinderTest extends MockedBookKeeperTestCase {

public static byte[] createMessageWrittenToLedger(String msg) {
return createMessageWrittenToLedger(msg, System.currentTimeMillis());
}
public static byte[] createMessageWrittenToLedger(String msg, long messageTimestamp) {
MessageMetadata messageMetadata = new MessageMetadata()
.setPublishTime(System.currentTimeMillis())
.setPublishTime(messageTimestamp)
.setProducerName("createMessageWrittenToLedger")
.setSequenceId(1);
ByteBuf data = UnpooledByteBufAllocator.DEFAULT.heapBuffer().writeBytes(msg.getBytes());
Expand Down Expand Up @@ -437,6 +440,29 @@ void testMessageExpiryWithTimestampNonRecoverableException() throws Exception {

}

@Test
public void testIncorrectClientClock() throws Exception {
315157973 marked this conversation as resolved.
Show resolved Hide resolved
final String ledgerAndCursorName = "testIncorrectClientClock";
int maxTTLSeconds = 1;
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setMaxEntriesPerLedger(1);
ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open(ledgerAndCursorName, config);
ManagedCursorImpl c1 = (ManagedCursorImpl) ledger.openCursor(ledgerAndCursorName);
// set client clock to 10 days later
long incorrectPublishTimestamp = System.currentTimeMillis() + TimeUnit.DAYS.toMillis(10);
for (int i = 0; i < 10; i++) {
ledger.addEntry(createMessageWrittenToLedger("msg" + i, incorrectPublishTimestamp));
}
assertEquals(ledger.getLedgersInfoAsList().size(), 10);
PersistentTopic mock = mock(PersistentTopic.class);
when(mock.getName()).thenReturn("topicname");
when(mock.getLastPosition()).thenReturn(PositionImpl.EARLIEST);
PersistentMessageExpiryMonitor monitor = new PersistentMessageExpiryMonitor(mock, c1.getName(), c1, null);
Thread.sleep(TimeUnit.SECONDS.toMillis(maxTTLSeconds));
monitor.expireMessages(maxTTLSeconds);
assertEquals(c1.getNumberOfEntriesInBacklog(true), 0);
}

@Test
void testMessageExpiryWithPosition() throws Exception {
final String ledgerAndCursorName = "testPersistentMessageExpiryWithPositionNonRecoverableLedgers";
Expand Down
Loading