Skip to content

Commit

Permalink
fix the closed ledger did not delete after expired (apache#9136)
Browse files Browse the repository at this point in the history
### Motivation
Fixes apache#9057

When current ledger closed, if there is no incoming traffic, the read position of the cursor is still point to the last entry of the closed ledger,  that casue the `slowestReaderLedgerId` point to the closed ledger in `internalTrimConsumedLedgers()` and fail to delete the closed ledger.

### Modifications
When close current ledger, if cursor's mark delete position point to the last entry of current ledger, move the read position to the new created ledger. 

### Verifying this change
add test case: testDeletionAfterLedgerClosedAndRetention()
  • Loading branch information
wangjialing218 authored Jan 10, 2021
1 parent ca93c1a commit 0e5c536
Show file tree
Hide file tree
Showing 3 changed files with 46 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1374,6 +1374,8 @@ public synchronized void createComplete(int rc, final LedgerHandle lh, Object ct
} else {
log.info("[{}] Created new ledger {}", name, lh.getId());
ledgers.put(lh.getId(), LedgerInfo.newBuilder().setLedgerId(lh.getId()).setTimestamp(0).build());
final long previousEntries = currentLedgerEntries;
final long previousLedgerId = currentLedger.getId();
currentLedger = lh;
currentLedgerEntries = 0;
currentLedgerSize = 0;
Expand All @@ -1391,6 +1393,14 @@ public void operationComplete(Void v, Stat stat) {
mbean.addLedgerSwitchLatencySample(System.currentTimeMillis() - lastLedgerCreationInitiationTimestamp,
TimeUnit.MILLISECONDS);
}
// Move cursor read point to new ledger
for (ManagedCursor cursor : cursors) {
PositionImpl markDeletedPosition = (PositionImpl) cursor.getMarkDeletedPosition();
if (markDeletedPosition.getLedgerId() == previousLedgerId && markDeletedPosition.getEntryId() + 1 >= previousEntries) {
// All entries in last ledger are marked delete, move read point to the new ledger
updateCursor((ManagedCursorImpl) cursor, PositionImpl.get(currentLedger.getId(), -1));
}
}
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
*/
package org.apache.bookkeeper.mledger.impl;

import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.anyString;
Expand All @@ -34,12 +33,10 @@
import static org.testng.Assert.assertSame;
import static org.testng.Assert.assertTrue;
import static org.testng.Assert.fail;

import com.google.common.base.Charsets;
import com.google.common.collect.Sets;

import io.netty.buffer.ByteBufAllocator;

import io.netty.util.concurrent.DefaultThreadFactory;
import java.lang.reflect.Field;
import java.nio.charset.Charset;
import java.security.GeneralSecurityException;
Expand All @@ -64,8 +61,6 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Predicate;

import io.netty.util.concurrent.DefaultThreadFactory;
import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
Expand Down Expand Up @@ -108,6 +103,7 @@
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.common.api.proto.CommandSubscribe.InitialPosition;
import org.apache.pulsar.common.policies.data.EnsemblePlacementPolicyConfig;
import org.apache.pulsar.metadata.api.Stat;
import org.apache.pulsar.metadata.impl.ZKMetadataStore;
import org.apache.zookeeper.CreateMode;
Expand Down Expand Up @@ -1836,6 +1832,34 @@ public void testDeletionAfterRetention() throws Exception {
ml.close();
}

@Test
public void testDeletionAfterLedgerClosedAndRetention() throws Exception {
ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(bkc, bkc.getZkHandle());
ManagedLedgerConfig config = new ManagedLedgerConfig();
config.setRetentionSizeInMB(0);
config.setMaxEntriesPerLedger(1);
config.setRetentionTime(1, TimeUnit.SECONDS);
config.setMaximumRolloverTime(1, TimeUnit.SECONDS);

ManagedLedgerImpl ml = (ManagedLedgerImpl) factory.open("deletion_after_retention_test_ledger", config);
ManagedCursor c1 = ml.openCursor("testCursor1");
ManagedCursor c2 = ml.openCursor("testCursor2");
ml.addEntry("iamaverylongmessagethatshouldnotberetained".getBytes());
c1.skipEntries(1, IndividualDeletedEntries.Exclude);
c2.skipEntries(1, IndividualDeletedEntries.Exclude);
// let current ledger close
ml.rollCurrentLedgerIfFull();
// let retention expire
Thread.sleep(1500);
// delete the expired ledger
ml.internalTrimConsumedLedgers(CompletableFuture.completedFuture(null));

// the closed and expired ledger should be deleted
assertTrue(ml.getLedgersInfoAsList().size() <= 1);
assertEquals(ml.getTotalSize(), 0);
ml.close();
}

/**
* Set retention time = 0 and create a empty ledger,
* first position can't higher than last after trim ledgers.
Expand Down Expand Up @@ -2862,7 +2886,7 @@ public void testManagedLedgerRollOverIfFull() throws Exception {
// all the messages have benn acknowledged
// and all the ledgers have been removed except the last ledger
Thread.sleep(1000);
Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 2);
Assert.assertEquals(ledger.getCurrentLedgerSize(), 0);
Assert.assertEquals(ledger.getLedgersInfoAsList().size(), 1);
Assert.assertEquals(ledger.getTotalSize(), 0);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,19 +18,17 @@
*/
package org.apache.pulsar.broker.service;

import java.util.concurrent.TimeUnit;
import lombok.Cleanup;
import org.apache.bookkeeper.mledger.ManagedLedgerConfig;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.util.Futures;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.Producer;
import org.junit.Test;
import org.testng.Assert;

import java.util.concurrent.TimeUnit;

public class CurrentLedgerRolloverIfFullTest extends BrokerTestBase {
@Override
protected void setup() throws Exception {
Expand Down Expand Up @@ -90,14 +88,10 @@ public void testCurrentLedgerRolloverIfFull() throws Exception {
Assert.assertNotEquals(managedLedger.getCurrentLedgerSize(), 0);

// trigger a ledger rollover
// and now we have two ledgers, one with expired data and one for empty
// the last ledger will be closed and removed and we have one ledger for empty
managedLedger.rollCurrentLedgerIfFull();
Thread.sleep(1000);
Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), 2);

// trigger a ledger trimming
// and now we only have the empty ledger
managedLedger.trimConsumedLedgersInBackground(Futures.NULL_PROMISE);
Assert.assertEquals(managedLedger.getCurrentLedgerSize(), 0);
Assert.assertEquals(managedLedger.getLedgersInfoAsList().size(), 1);
Assert.assertEquals(managedLedger.getTotalSize(), 0);
}
}

0 comments on commit 0e5c536

Please sign in to comment.