diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java index 24228d8d6d238..a5e3cfb3d69fd 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerConfig.java @@ -86,6 +86,8 @@ public class ManagedLedgerConfig { private int minimumBacklogEntriesForCaching = 1000; private int maxBacklogBetweenCursorsForCaching = 1000; + private int maxAckEntryNumForAutoSkipNonRecoverableData = 10000; + public boolean isCreateIfMissing() { return createIfMissing; } @@ -95,6 +97,15 @@ public ManagedLedgerConfig setCreateIfMissing(boolean createIfMissing) { return this; } + public int getMaxAckEntryNumForAutoSkipNonRecoverableData() { + return maxAckEntryNumForAutoSkipNonRecoverableData; + } + + public ManagedLedgerConfig setMaxAckEntryNumForAutoSkipNonRecoverableData(int maxAckEntryNum) { + this.maxAckEntryNumForAutoSkipNonRecoverableData = maxAckEntryNum; + return this; + } + /** * @return the lazyCursorRecovery */ diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java index c387aa970adb6..b840c36225060 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/EntryImpl.java @@ -42,6 +42,7 @@ protected EntryImpl newObject(Handle handle) { private long timestamp; private long ledgerId; private long entryId; + private boolean skipped; ByteBuf data; public static EntryImpl create(LedgerEntry ledgerEntry) { @@ -66,6 +67,17 @@ public static EntryImpl create(long ledgerId, long entryId, byte[] data) { return entry; } + public static EntryImpl createSkippedEntry(long ledgerId, long entryId) { + EntryImpl entry = RECYCLER.get(); + entry.timestamp = System.nanoTime(); + entry.ledgerId = ledgerId; + entry.entryId = entryId; + entry.skipped = true; + entry.data = Unpooled.EMPTY_BUFFER; + entry.setRefCnt(1); + return entry; + } + public static EntryImpl create(long ledgerId, long entryId, ByteBuf data) { EntryImpl entry = RECYCLER.get(); entry.timestamp = System.nanoTime(); @@ -146,6 +158,10 @@ public long getEntryId() { return entryId; } + public boolean skipped() { + return skipped; + } + @Override public int compareTo(EntryImpl other) { if (this.ledgerId != other.ledgerId) { @@ -172,6 +188,7 @@ protected void deallocate() { timestamp = -1; ledgerId = -1; entryId = -1; + skipped = false; recyclerHandle.recycle(this); } diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java index b90527834398a..5a040ce9a7db9 100644 --- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java +++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java @@ -116,6 +116,22 @@ public void readEntriesFailed(ManagedLedgerException exception, Object ctx) { recycle(); return; } + + int toAckEntryNum = 0; + List skippedEntries = new ArrayList<>(); + PositionImpl startPosition = readPosition; + PositionImpl endPosition = (PositionImpl) nexReadPosition; + while (startPosition.compareTo(endPosition) < 0) { + skippedEntries.add(EntryImpl.createSkippedEntry(startPosition.ledgerId, startPosition.entryId)); + startPosition = ledger.getNextValidPosition(startPosition); + toAckEntryNum++; + if (toAckEntryNum > cursor.getConfig().getMaxAckEntryNumForAutoSkipNonRecoverableData()) { + nexReadPosition = startPosition; + break; + } + } + List filteredEntries = cursor.filterReadEntries(skippedEntries); + entries.addAll(filteredEntries); updateReadPosition(nexReadPosition); checkReadCompletion(); } else { diff --git a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java index 6683d36c36e06..13fbcbb785dfd 100644 --- a/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java +++ b/pulsar-broker-common/src/main/java/org/apache/pulsar/broker/ServiceConfiguration.java @@ -2009,6 +2009,13 @@ The delayed message index bucket time step(in seconds) in per bucket snapshot se + " It helps when data-ledgers gets corrupted at bookkeeper and managed-cursor is stuck at that ledger." ) private boolean autoSkipNonRecoverableData = false; + @FieldContext( + dynamic = true, + category = CATEGORY_STORAGE_ML, + doc = "When autoSkipNonRecoverableData=true, " + + "the upper limit of the number of entries skipped by automatic ack." + ) + private int maxAckEntryNumForAutoSkipNonRecoverableData = 10000; @FieldContext( category = CATEGORY_STORAGE_ML, doc = "operation timeout while updating managed-ledger metadata." diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java index df02bbd85d470..8436f8503f773 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/AbstractBaseDispatcher.java @@ -31,6 +31,7 @@ import org.apache.bookkeeper.mledger.Entry; import org.apache.bookkeeper.mledger.ManagedCursor; import org.apache.bookkeeper.mledger.Position; +import org.apache.bookkeeper.mledger.impl.EntryImpl; import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.tuple.Pair; @@ -118,6 +119,18 @@ public int filterEntriesForConsumer(Optional optMetadataArray if (entry == null) { continue; } + if (entry instanceof EntryImpl) { + EntryImpl entryImpl = (EntryImpl) entry; + if (entryImpl.skipped()) { + if (entriesToFiltered == null) { + entriesToFiltered = new ArrayList<>(); + } + entriesToFiltered.add(entryImpl.getPosition()); + entries.set(i, null); + entry.release(); + continue; + } + } ByteBuf metadataAndPayload = entry.getDataBuffer(); final int metadataIndex = i + startOffset; final MessageMetadata msgMetadata = optMetadataArray.map(metadataArray -> metadataArray[metadataIndex]) diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java index 0ff8bfac45741..cd1f1dfa6fde3 100644 --- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java +++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/BrokerService.java @@ -1663,6 +1663,8 @@ public CompletableFuture getManagedLedgerConfig(TopicName t } } + managedLedgerConfig.setMaxAckEntryNumForAutoSkipNonRecoverableData( + serviceConfig.getMaxAckEntryNumForAutoSkipNonRecoverableData()); managedLedgerConfig.setThrottleMarkDelete(persistencePolicies.getManagedLedgerMaxMarkDeleteRate()); managedLedgerConfig.setDigestType(serviceConfig.getManagedLedgerDigestType()); managedLedgerConfig.setPassword(serviceConfig.getManagedLedgerPassword()); diff --git a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java index 8b0dbb76eecb7..fe43d18538629 100644 --- a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java +++ b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/BrokerBkEnsemblesTests.java @@ -20,6 +20,7 @@ import static org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest.retryStrategically; import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertTrue; import static org.testng.Assert.assertNotEquals; import static org.testng.Assert.fail; @@ -43,6 +44,7 @@ import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl; import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl; +import org.apache.bookkeeper.mledger.impl.PositionImpl; import org.apache.bookkeeper.mledger.proto.MLDataFormats.ManagedLedgerInfo.LedgerInfo; import org.apache.bookkeeper.util.StringUtils; import org.apache.pulsar.broker.BrokerTestUtil; @@ -166,6 +168,101 @@ public void testCrashBrokerWithoutCursorLedgerLeak() throws Exception { consumer.close(); } + @Test + public void testSkipCorruptDataLedgerAndCheckMarkdelete() throws Exception { + // Ensure intended state for autoSkipNonRecoverableData + admin.brokers().updateDynamicConfiguration("autoSkipNonRecoverableData", "true"); + @Cleanup + PulsarClient client = PulsarClient.builder() + .serviceUrl(pulsar.getWebServiceAddress()) + .statsInterval(0, TimeUnit.SECONDS) + .build(); + + final String ns1 = "prop/usc/crash-broker"; + final int totalMessages = 100; + final int totalDataLedgers = 5; + final int entriesPerLedger = totalMessages / totalDataLedgers; + + try { + admin.namespaces().createNamespace(ns1); + } catch (Exception e) { + } + + final String topic1 = "persistent://" + ns1 + "/my-topic-" + System.currentTimeMillis(); + // Create subscription + Consumer consumer = client.newConsumer().topic(topic1).subscriptionName("my-subscriber-name") + .receiverQueueSize(5).subscribe(); + PersistentTopic topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topic1).get(); + ManagedLedgerImpl ml = (ManagedLedgerImpl) topic.getManagedLedger(); + ManagedCursorImpl cursor = (ManagedCursorImpl) ml.getCursors().iterator().next(); + Field configField = ManagedCursorImpl.class.getDeclaredField("config"); + configField.setAccessible(true); + // Create multiple data-ledger + ManagedLedgerConfig config = (ManagedLedgerConfig) configField.get(cursor); + config.setMaxEntriesPerLedger(entriesPerLedger); + config.setMinimumRolloverTime(1, TimeUnit.MILLISECONDS); + // bookkeeper client + Field bookKeeperField = ManagedLedgerImpl.class.getDeclaredField("bookKeeper"); + bookKeeperField.setAccessible(true); + // Create multiple data-ledger + BookKeeper bookKeeper = (BookKeeper) bookKeeperField.get(ml); + + // (1) publish messages in 5 data-ledgers each with 20 entries under managed-ledger + Producer producer = client.newProducer().topic(topic1).create(); + for (int i = 0; i < totalMessages; i++) { + String message = "my-message-" + i; + producer.send(message.getBytes()); + } + producer.close(); + + Message msg = null; + // (2) consume 20 messages from first ledger + for (int i = 0; i < entriesPerLedger; i++) { + msg = consumer.receive(); + consumer.acknowledge(msg); + } + consumer.close(); + PositionImpl markDeletePosition1 = (PositionImpl) cursor.getMarkDeletedPosition(); + + // (3) delete first 4 data-ledgers and clear cache + NavigableMap ledgerInfo = ml.getLedgersInfo(); + Entry lastLedger = ledgerInfo.lastEntry(); + ledgerInfo.entrySet().forEach(entry -> { + if (!entry.equals(lastLedger)) { + assertEquals(entry.getValue().getEntries(), entriesPerLedger); + try { + bookKeeper.deleteLedger(entry.getKey()); + } catch (Exception e) { + log.warn("failed to delete ledger {}", entry.getKey(), e); + } + } + }); + pulsar.getBrokerService().removeTopicFromCache(topic1); + ManagedLedgerFactoryImpl factory = (ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory(); + Field field = ManagedLedgerFactoryImpl.class.getDeclaredField("ledgers"); + field.setAccessible(true); + @SuppressWarnings("unchecked") + ConcurrentHashMap> ledgers = (ConcurrentHashMap>) field + .get(factory); + ledgers.clear(); + + // (4) consumer will be able to consume 20 messages from last non-deleted ledger + consumer = client.newConsumer().topic(topic1).subscriptionName("my-subscriber-name") + .receiverQueueSize(5).subscribe(); + for (int i = 0; i < entriesPerLedger; i++) { + msg = consumer.receive(); + consumer.acknowledge(msg); + } + consumer.close(); + + topic = (PersistentTopic) pulsar.getBrokerService().getOrCreateTopic(topic1).get(); + ml = (ManagedLedgerImpl) topic.getManagedLedger(); + cursor = (ManagedCursorImpl) ml.getCursors().iterator().next(); + PositionImpl markDeletePosition2 = (PositionImpl) cursor.getMarkDeletedPosition(); + // markDeletePosition moves forward + assertTrue(markDeletePosition2.compareTo(markDeletePosition1) > 0); + } + /** * It verifies broker-configuration using which broker can skip non-recoverable data-ledgers. *