You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by te...@apache.org on 2022/09/06 05:05:43 UTC

[pulsar] branch branch-2.10 updated: [Branch-2.10] [fix][ML] Fix offload read handle NPE. (#17478)

This is an automated email from the ASF dual-hosted git repository.

technoboy pushed a commit to branch branch-2.10
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.10 by this push:
     new 45c37f39902 [Branch-2.10] [fix][ML] Fix offload read handle NPE. (#17478)
45c37f39902 is described below

commit 45c37f39902219b7b0cea13aa0034635073e9d9d
Author: Yan Zhao <ho...@apache.org>
AuthorDate: Tue Sep 6 13:05:37 2022 +0800

    [Branch-2.10] [fix][ML] Fix offload read handle NPE. (#17478)
---
 .../bookkeeper/mledger/ManagedLedgerException.java |  7 +++
 .../bookkeeper/mledger/impl/ManagedLedgerImpl.java | 26 ++++++++--
 .../bookkeeper/mledger/impl/ManagedLedgerTest.java | 26 ++++++++++
 .../mledger/impl/NonDurableCursorTest.java         | 60 ++++++++++++++++++++++
 .../PersistentDispatcherMultipleConsumers.java     |  3 +-
 .../PersistentDispatcherSingleActiveConsumer.java  |  3 +-
 .../jcloud/impl/BlobStoreBackedReadHandleImpl.java | 12 +++--
 .../impl/BlobStoreBackedReadHandleImplV2.java      | 29 ++++++++---
 .../impl/BlobStoreManagedLedgerOffloaderTest.java  |  3 +-
 9 files changed, 150 insertions(+), 19 deletions(-)

diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java
index 7046ba48193..0280ceda725 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/ManagedLedgerException.java
@@ -194,4 +194,11 @@ public class ManagedLedgerException extends Exception {
         // Disable stack traces to be filled in
         return null;
     }
+
+    public static class OffloadReadHandleClosedException extends ManagedLedgerException {
+
+        public OffloadReadHandleClosedException() {
+            super("Offload read handle already closed");
+        }
+    }
 }
diff --git a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
index 9a92d35c057..6f933962d46 100644
--- a/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
+++ b/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java
@@ -2469,7 +2469,7 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
                     break;
                 }
                 // if truncate, all ledgers besides currentLedger are going to be deleted
-                if (isTruncate){
+                if (isTruncate) {
                     if (log.isDebugEnabled()) {
                         log.debug("[{}] Ledger {} will be truncated with ts {}",
                                 name, ls.getLedgerId(), ls.getTimestamp());
@@ -2497,11 +2497,14 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
                     }
                     ledgersToDelete.add(ls);
                 } else {
-                    // once retention constraint has been met, skip check
-                    if (log.isDebugEnabled()) {
-                        log.debug("[{}] Ledger {} not deleted. Neither expired nor over-quota", name, ls.getLedgerId());
+                    if (ls.getLedgerId() < getTheSlowestNonDurationReadPosition().getLedgerId()) {
+                        // once retention constraint has been met, skip check
+                        if (log.isDebugEnabled()) {
+                            log.debug("[{}] Ledger {} not deleted. Neither expired nor over-quota", name,
+                                    ls.getLedgerId());
+                        }
+                        invalidateReadHandle(ls.getLedgerId());
                     }
-                    invalidateReadHandle(ls.getLedgerId());
                 }
             }
 
@@ -4149,4 +4152,17 @@ public class ManagedLedgerImpl implements ManagedLedger, CreateCallback {
         }
     }
 
+    public Position getTheSlowestNonDurationReadPosition() {
+        PositionImpl theSlowestNonDurableReadPosition = PositionImpl.LATEST;
+        for (ManagedCursor cursor : cursors) {
+            if (cursor instanceof NonDurableCursorImpl) {
+                PositionImpl readPosition = (PositionImpl) cursor.getReadPosition();
+                if (readPosition.compareTo(theSlowestNonDurableReadPosition) < 0) {
+                    theSlowestNonDurableReadPosition = readPosition;
+                }
+            }
+        }
+        return theSlowestNonDurableReadPosition;
+    }
+
 }
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
index 908d381a838..753c34f38ef 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerTest.java
@@ -18,6 +18,7 @@
  */
 package org.apache.bookkeeper.mledger.impl;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.ArgumentMatchers.anyInt;
 import static org.mockito.ArgumentMatchers.anyString;
@@ -3580,6 +3581,31 @@ public class ManagedLedgerTest extends MockedBookKeeperTestCase {
         });
     }
 
+    @Test
+    public void testGetTheSlowestNonDurationReadPosition() throws Exception {
+        ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("test_",
+                new ManagedLedgerConfig().setMaxEntriesPerLedger(1).setRetentionTime(-1, TimeUnit.SECONDS)
+                        .setRetentionSizeInMB(-1));
+        ledger.openCursor("c1");
+
+        List<Position> positions = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            positions.add(ledger.addEntry(("entry-" + i).getBytes(UTF_8)));
+        }
+
+        Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), PositionImpl.LATEST);
+
+        ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST);
+
+        Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), positions.get(0));
+
+        ledger.deleteCursor(nonDurableCursor.getName());
+
+        Assert.assertEquals(ledger.getTheSlowestNonDurationReadPosition(), PositionImpl.LATEST);
+
+        ledger.close();
+    }
+
     @Test
     public void testGetLedgerMetadata() throws Exception {
         ManagedLedgerImpl managedLedger = (ManagedLedgerImpl) factory.open("testGetLedgerMetadata");
diff --git a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java
index 697cf28c172..437ca9937af 100644
--- a/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java
+++ b/managed-ledger/src/test/java/org/apache/bookkeeper/mledger/impl/NonDurableCursorTest.java
@@ -31,6 +31,7 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
 import java.nio.charset.Charset;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CountDownLatch;
@@ -54,6 +55,7 @@ import org.apache.bookkeeper.mledger.Position;
 import org.apache.bookkeeper.test.MockedBookKeeperTestCase;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.testng.Assert;
 import org.testng.annotations.Test;
 
 public class NonDurableCursorTest extends MockedBookKeeperTestCase {
@@ -735,6 +737,64 @@ public class NonDurableCursorTest extends MockedBookKeeperTestCase {
         ledger.close();
     }
 
+    @Test
+    public void testInvalidateReadHandleWithSlowNonDurableCursor() throws Exception {
+        ManagedLedgerImpl ledger = (ManagedLedgerImpl) factory.open("testInvalidateReadHandleWithSlowNonDurableCursor",
+                new ManagedLedgerConfig().setMaxEntriesPerLedger(1).setRetentionTime(-1, TimeUnit.SECONDS)
+                        .setRetentionSizeInMB(-1));
+        ManagedCursor c1 = ledger.openCursor("c1");
+        ManagedCursor nonDurableCursor = ledger.newNonDurableCursor(PositionImpl.EARLIEST);
+
+        List<Position> positions = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            positions.add(ledger.addEntry(("entry-" + i).getBytes(UTF_8)));
+        }
+
+        CountDownLatch latch = new CountDownLatch(10);
+        for (int i = 0; i < 10; i++) {
+            ledger.asyncReadEntry((PositionImpl) positions.get(i), new AsyncCallbacks.ReadEntryCallback() {
+                @Override
+                public void readEntryComplete(Entry entry, Object ctx) {
+                    latch.countDown();
+                }
+
+                @Override
+                public void readEntryFailed(ManagedLedgerException exception, Object ctx) {
+                    latch.countDown();
+                }
+            }, null);
+        }
+
+        latch.await();
+
+        c1.markDelete(positions.get(4));
+
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        ledger.internalTrimConsumedLedgers(promise);
+        promise.join();
+
+        Assert.assertTrue(ledger.ledgerCache.containsKey(positions.get(0).getLedgerId()));
+        Assert.assertTrue(ledger.ledgerCache.containsKey(positions.get(1).getLedgerId()));
+        Assert.assertTrue(ledger.ledgerCache.containsKey(positions.get(2).getLedgerId()));
+        Assert.assertTrue(ledger.ledgerCache.containsKey(positions.get(3).getLedgerId()));
+        Assert.assertTrue(ledger.ledgerCache.containsKey(positions.get(4).getLedgerId()));
+
+        promise = new CompletableFuture<>();
+
+        nonDurableCursor.markDelete(positions.get(3));
+
+        ledger.internalTrimConsumedLedgers(promise);
+        promise.join();
+
+        Assert.assertFalse(ledger.ledgerCache.containsKey(positions.get(0).getLedgerId()));
+        Assert.assertFalse(ledger.ledgerCache.containsKey(positions.get(1).getLedgerId()));
+        Assert.assertFalse(ledger.ledgerCache.containsKey(positions.get(2).getLedgerId()));
+        Assert.assertFalse(ledger.ledgerCache.containsKey(positions.get(3).getLedgerId()));
+        Assert.assertTrue(ledger.ledgerCache.containsKey(positions.get(4).getLedgerId()));
+
+        ledger.close();
+    }
+
     @Test(expectedExceptions = NullPointerException.class)
     void testCursorWithNameIsNotNull() throws Exception {
         final String p1CursorName = "entry-1";
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
index ee11d8961b8..bfff0c67c73 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherMultipleConsumers.java
@@ -657,7 +657,8 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMul
                 // Notify the consumer only if all the messages were already acknowledged
                 consumerList.forEach(Consumer::reachedEndOfTopic);
             }
-        } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException) {
+        } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException
+                || exception.getCause() instanceof ManagedLedgerException.OffloadReadHandleClosedException) {
             waitTimeMillis = 1;
             if (log.isDebugEnabled()) {
                 log.debug("[{}] Error reading transaction entries : {}, Read Type {} - Retrying to read in {} seconds",
diff --git a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java
index 79f397af883..1f2636fbe4c 100644
--- a/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java
+++ b/pulsar-broker/src/main/java/org/apache/pulsar/broker/service/persistent/PersistentDispatcherSingleActiveConsumer.java
@@ -497,7 +497,8 @@ public class PersistentDispatcherSingleActiveConsumer extends AbstractDispatcher
                 // Notify the consumer only if all the messages were already acknowledged
                 consumers.forEach(Consumer::reachedEndOfTopic);
             }
-        } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException) {
+        } else if (exception.getCause() instanceof TransactionBufferException.TransactionNotSealedException
+                || exception.getCause() instanceof ManagedLedgerException.OffloadReadHandleClosedException) {
             waitTimeMillis = 1;
             if (log.isDebugEnabled()) {
                 log.debug("[{}] Error reading transaction entries : {}, - Retrying to read in {} seconds", name,
diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java
index fae534454e0..c5ca5e1aaf3 100644
--- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java
+++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java
@@ -35,6 +35,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
 import org.apache.bookkeeper.mledger.offload.jcloud.BackedInputStream;
 import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlock;
 import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockBuilder;
@@ -103,14 +104,15 @@ public class BlobStoreBackedReadHandleImpl implements ReadHandle {
         log.debug("Ledger {}: reading {} - {}", getId(), firstEntry, lastEntry);
         CompletableFuture<LedgerEntries> promise = new CompletableFuture<>();
         executor.submit(() -> {
+            if (state == State.Closed) {
+                log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}",
+                        ledgerId, firstEntry, lastEntry);
+                promise.completeExceptionally(new ManagedLedgerException.OffloadReadHandleClosedException());
+                return;
+            }
             List<LedgerEntry> entries = new ArrayList<LedgerEntry>();
             boolean seeked = false;
             try {
-                if (state == State.Closed) {
-                    log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}",
-                        ledgerId, firstEntry, lastEntry);
-                    throw new BKException.BKUnexpectedConditionException();
-                }
                 if (firstEntry > lastEntry
                     || firstEntry < 0
                     || lastEntry > getLastAddConfirmed()) {
diff --git a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java
index a6f66132c4a..a99f0abd248 100644
--- a/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java
+++ b/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImplV2.java
@@ -38,6 +38,7 @@ import org.apache.bookkeeper.client.api.LedgerMetadata;
 import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.client.impl.LedgerEntriesImpl;
 import org.apache.bookkeeper.client.impl.LedgerEntryImpl;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
 import org.apache.bookkeeper.mledger.offload.jcloud.BackedInputStream;
 import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2;
 import org.apache.bookkeeper.mledger.offload.jcloud.OffloadIndexBlockV2Builder;
@@ -57,6 +58,13 @@ public class BlobStoreBackedReadHandleImplV2 implements ReadHandle {
     private final List<DataInputStream> dataStreams;
     private final ExecutorService executor;
 
+    private State state = null;
+
+    enum State {
+        Opened,
+        Closed
+    }
+
     static class GroupedReader {
         @Override
         public String toString() {
@@ -97,6 +105,7 @@ public class BlobStoreBackedReadHandleImplV2 implements ReadHandle {
             dataStreams.add(new DataInputStream(inputStream));
         }
         this.executor = executor;
+        this.state = State.Opened;
     }
 
     @Override
@@ -121,6 +130,7 @@ public class BlobStoreBackedReadHandleImplV2 implements ReadHandle {
                 for (DataInputStream dataStream : dataStreams) {
                     dataStream.close();
                 }
+                state = State.Closed;
                 promise.complete(null);
             } catch (IOException t) {
                 promise.completeExceptionally(t);
@@ -133,13 +143,20 @@ public class BlobStoreBackedReadHandleImplV2 implements ReadHandle {
     public CompletableFuture<LedgerEntries> readAsync(long firstEntry, long lastEntry) {
         log.debug("Ledger {}: reading {} - {}", getId(), firstEntry, lastEntry);
         CompletableFuture<LedgerEntries> promise = new CompletableFuture<>();
-        if (firstEntry > lastEntry
-                || firstEntry < 0
-                || lastEntry > getLastAddConfirmed()) {
-            promise.completeExceptionally(new IllegalArgumentException());
-            return promise;
-        }
         executor.submit(() -> {
+            if (state == State.Closed) {
+                log.warn("Reading a closed read handler. Ledger ID: {}, Read range: {}-{}",
+                        ledgerId, firstEntry, lastEntry);
+                promise.completeExceptionally(new ManagedLedgerException.OffloadReadHandleClosedException());
+                return;
+            }
+
+            if (firstEntry > lastEntry
+                    || firstEntry < 0
+                    || lastEntry > getLastAddConfirmed()) {
+                promise.completeExceptionally(new BKException.BKIncorrectParameterException());
+                return;
+            }
             List<LedgerEntry> entries = new ArrayList<LedgerEntry>();
             List<GroupedReader> groupedReaders = null;
             try {
diff --git a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java
index ab979f8a5a1..b78666ad645 100644
--- a/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java
+++ b/tiered-storage/jcloud/src/test/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreManagedLedgerOffloaderTest.java
@@ -41,6 +41,7 @@ import org.apache.bookkeeper.client.api.LedgerEntries;
 import org.apache.bookkeeper.client.api.LedgerEntry;
 import org.apache.bookkeeper.client.api.ReadHandle;
 import org.apache.bookkeeper.mledger.LedgerOffloader;
+import org.apache.bookkeeper.mledger.ManagedLedgerException;
 import org.apache.bookkeeper.mledger.offload.jcloud.provider.JCloudBlobStoreProvider;
 import org.apache.bookkeeper.mledger.offload.jcloud.provider.TieredStorageConfiguration;
 import org.jclouds.blobstore.BlobStore;
@@ -512,7 +513,7 @@ public class BlobStoreManagedLedgerOffloaderTest extends BlobStoreManagedLedgerO
         try {
             toTest.readAsync(0, lac).get();
         } catch (Exception e) {
-            if (e.getCause() instanceof BKException.BKUnexpectedConditionException) {
+            if (e.getCause() instanceof ManagedLedgerException.OffloadReadHandleClosedException) {
                 // expected exception
                 return;
             }