shibd opened a new pull request, #22531:
URL: https://github.com/apache/pulsar/pull/22531
### Motivation
In the implementation of tiered storage, if a `blob` in blob storage is lost
unexpectedly, reading it will cause a `NullPointerException`, causing a
subscription block.
```
116 Caused by: java.lang.NullPointerException
117 2023-08-28T02:35:23,544+0000 [offloader-OrderedScheduler-0-0] WARN
org.apache.bookkeeper.mledger.impl.OpReadEntry -
[test/test/persistent/test-partition-1][test-consume] read failed from ledger
at position:1358058:0
118 org.apache.bookkeeper.mledger.ManagedLedgerException: Other exception
119 Caused by: java.io.IOException: Error reading from BlobStore
120 at
org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedInputStreamImpl.refillBufferIfNeeded(BlobStoreBackedInputStreamImpl.java:91)
~[?:?]
121 at
org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedInputStreamImpl.read(BlobStoreBackedInputStreamImpl.java:99)
~[?:?]
122 at java.io.DataInputStream.readInt(DataInputStream.java:392)
~[?:?]
123 at
org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedReadHandleImpl.lambda$readAsync$1(BlobStoreBackedReadHandleImpl.java:136)
~[?:?]
124 at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) ~[?:?]
125 at java.util.concurrent.FutureTask.run(FutureTask.java:264)
~[?:?]
126 at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
~[?:?]
127 at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
~[?:?]
128 at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
~[?:?]
129 at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
~[io.netty-netty-common-4.1.87.Final.jar:4.1.87.Final]
130 at java.lang.Thread.run(Thread.java:829) ~[?:?]
131 Caused by: java.lang.NullPointerException
132 2023-08-28T02:35:23,545+0000
[broker-topic-workers-OrderedExecutor-5-0] ERROR
org.apache.pulsar.broker.service.persistent.PersistentDispatcherSingleActiveConsumer
- [persistent://test/test/test-partition-1 / test-consume
-Consumer{subscription=PersistentSubscription{topic=persistent://test/test/test-partition-1,
name=test-consume}, consumerId=1, consumerName=07da3,
address=/127.0.0.1:39850}] Error reading entries at 1358058:0 : Other exception
- Retrying to read in 27.426 seconds
133 2023-08-28T02:35:25,999+0000 [offloader-OrderedScheduler-0-0] ERROR
org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedReadHandleImpl
- Failed to read entries 0 - 0 fro m the offloader in ledger 1358058
134 java.io.IOException: Error reading from BlobStore
```
#1046 introduced the `autoSkipNonRecoverableData` configuration to skip
BookKeeper ledgers lost unexpectedly. This configuration can be utilized to
address the same issue in tiered storage..
BTW: This PR is built upon the enhancements made by #21269. Special thanks
to @liangyepianzhou for the initial research. However, I am unable to retain
your commit information.
### Modifications
- In `BackedInputStream`, When
[blobStore.getBlob](https://github.com/apache/jclouds/blob/master/blobstore/src/main/java/org/jclouds/blobstore/BlobStore.java#L280C1-L280C77)
return null, means a blob not found in container, throw a
`KeyNotFoundException` to the
caller(**BlobStoreBackedReadHandleImpl.readAsync**).
- In `BlobStoreBackedReadHandleImpl.readAsync` method of the `ReadHandle`
interface, if receive a `KeyNotFoundException` exception, will throw a
`BKNoSuchLedgerExistsException` that will transfer to
`NonRecoverableLedgerException`, which will eventually be handled by the
following code.
https://github.com/apache/pulsar/blob/66271e3bf3cf0699789c759d852c24e6f00f90cd/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpReadEntry.java#L114-L140
### Verifying this change
- Add testNotFoundOnRead unit test to cover bucket not found.
- Add testReadNotExistLedger unit test to cover ledger not found.
### Documentation
<!-- DO NOT REMOVE THIS SECTION. CHECK THE PROPER BOX ONLY. -->
- [ ] `doc` <!-- Your PR contains doc changes. -->
- [ ] `doc-required` <!-- Your PR changes impact docs and you will update
later -->
- [x] `doc-not-needed` <!-- Your PR changes do not impact docs -->
- [ ] `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]