horizonzy opened a new pull request, #17056:
URL: https://github.com/apache/pulsar/pull/17056

   ### Motivation
   Now, when we get ReadHanle and then use it to read data (Non-durable cursor 
read). Maybe the ReadHandle we get already is invalidated.
   
   There is a race condition, maybe after we get the ReadHandle, before use it 
to read. 
   The method `ManagedLedgerImpl#internalTrimLedgers` invalidate this 
ReadHandle (Cause in this method, it just calculate by durable-cursor, but 
there maybe a non-durable cursor read data) so we use the invalidated offload 
read handle to read data, casue NPE.
   
   
   
https://github.com/apache/pulsar/blob/96930fda8a7776aaf6e61235a1d77ffe6e564052/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/ManagedLedgerImpl.java#L2542
   
   ```
   07:11:40.056 [offloader-OrderedScheduler-1-0] WARN  
org.apache.pulsar.broker.admin.impl.PersistentTopicsBase - 
[admin][persistent://cme_dev/market_data_mbo_v1/345_0-partition-0] Failed to 
reset cursor on subscription security_data_normalizer_subscriber_v1_dlabak_m01 
to position 10218543:44528:0
   
org.apache.pulsar.broker.service.BrokerServiceException$SubscriptionBusyException:
 Failed to fence subscription
        at 
org.apache.pulsar.broker.service.persistent.PersistentSubscription.resetCursor(PersistentSubscription.java:630)
 ~[io.streamnative-pulsar-broker-2.9.2.23.jar:2.9.2.23]
        at 
org.apache.pulsar.broker.service.persistent.PersistentSubscription.resetCursor(PersistentSubscription.java:624)
 ~[io.streamnative-pulsar-broker-2.9.2.23.jar:2.9.2.23]
        at 
org.apache.pulsar.broker.admin.impl.PersistentTopicsBase.lambda$internalResetCursorOnPosition$120(PersistentTopicsBase.java:2234)
 ~[io.streamnative-pulsar-broker-2.9.2.23.jar:2.9.2.23]
        at 
java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:714)
 ~[?:?]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) 
~[?:?]
        at 
java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:2073) 
~[?:?]
        at 
org.apache.pulsar.broker.admin.impl.PersistentTopicsBase$2.readEntryFailed(PersistentTopicsBase.java:2277)
 ~[io.streamnative-pulsar-broker-2.9.2.23.jar:2.9.2.23]
        at 
org.apache.bookkeeper.mledger.impl.EntryCacheImpl.lambda$asyncReadEntry0$1(EntryCacheImpl.java:233)
 ~[io.streamnative-managed-ledger-2.9.2.23.jar:2.9.2.23]
        at 
java.util.concurrent.CompletableFuture.uniExceptionally(CompletableFuture.java:986)
 ~[?:?]
        at 
java.util.concurrent.CompletableFuture$UniExceptionally.tryFire(CompletableFuture.java:970)
 ~[?:?]
        at 
java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:506) 
~[?:?]
        at 
java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:2088)
 ~[?:?]
        at 
org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedReadHandleImpl.lambda$readAsync$1(BlobStoreBackedReadHandleImpl.java:176)
 ~[?:?]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
        at 
com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
 [com.google.guava-guava-30.1-jre.jar:?]
        at 
com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
 [com.google.guava-guava-30.1-jre.jar:?]
        at 
com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
 [com.google.guava-guava-30.1-jre.jar:?]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
        at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
 [?:?]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) 
[?:?]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) 
[?:?]
        at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 [io.netty-netty-common-4.1.77.Final.jar:4.1.77.Final]
        at java.lang.Thread.run(Thread.java:829) [?:?]
   ```
   
   ### Modifications
   1.Fix the NPE.
   2.Check the invalidate ledgerId is less than the slowest non-durable, reduce 
the npe probability.
   
   
   ### Documentation
   
   Check the box below or label this PR directly.
   
   Need to update docs? 
   
   - [ ] `doc-required` 
   (Your PR needs to update docs and you will update later)
     
   - [ ] `doc-not-needed` 
   (Please explain why)
     
   - [ ] `doc` 
   (Your PR contains doc changes)
   
   - [ ] `doc-complete`
   (Docs have been already added)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to