Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[improve][offload] Apply autoSkipNonRecoverableData configuration to tiered storage #22531

Merged
merged 2 commits into from
Apr 19, 2024

Conversation

shibd
Copy link
Member

@shibd shibd commented Apr 18, 2024

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.

Modifications

  • In BackedInputStream, When blobStore.getBlob 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.

} else if (cursor.config.isAutoSkipNonRecoverableData() && exception instanceof NonRecoverableLedgerException) {
log.warn("[{}][{}] read failed from ledger at position:{} : {}", cursor.ledger.getName(), cursor.getName(),
readPosition, exception.getMessage());
final ManagedLedgerImpl ledger = (ManagedLedgerImpl) cursor.getManagedLedger();
Position nexReadPosition;
Long lostLedger = null;
if (exception instanceof ManagedLedgerException.LedgerNotExistException) {
// try to find and move to next valid ledger
nexReadPosition = cursor.getNextLedgerPosition(readPosition.getLedgerId());
lostLedger = readPosition.ledgerId;
} else {
// Skip this read operation
nexReadPosition = ledger.getValidPositionAfterSkippedEntries(readPosition, count);
}
// fail callback if it couldn't find next valid ledger
if (nexReadPosition == null) {
callback.readEntriesFailed(exception, ctx);
cursor.ledger.mbean.recordReadEntriesError();
recycle();
return;
}
updateReadPosition(nexReadPosition);
if (lostLedger != null) {
cursor.getManagedLedger().skipNonRecoverableLedger(lostLedger);
}
checkReadCompletion();
} else {

Verifying this change

  • Add testNotFoundOnRead unit test to cover bucket not found.
  • Add testReadNotExistLedger unit test to cover ledger not found.

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

@shibd shibd added type/enhancement The enhancements for the existing features or docs. e.g. reduce memory usage of the delayed messages area/tieredstorage ready-to-test labels Apr 18, 2024
@shibd shibd self-assigned this Apr 18, 2024
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Apr 18, 2024
@codecov-commenter
Copy link

Codecov Report

Attention: Patch coverage is 50.00000% with 8 lines in your changes are missing coverage. Please review.

Project coverage is 73.86%. Comparing base (bbc6224) to head (4aacca4).
Report is 164 commits behind head on master.

Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #22531      +/-   ##
============================================
+ Coverage     73.57%   73.86%   +0.29%     
+ Complexity    32624    32505     -119     
============================================
  Files          1877     1885       +8     
  Lines        139502   140151     +649     
  Branches      15299    15372      +73     
============================================
+ Hits         102638   103526     +888     
+ Misses        28908    28622     -286     
- Partials       7956     8003      +47     
Flag Coverage Δ
inttests 27.04% <ø> (+2.45%) ⬆️
systests 24.58% <ø> (+0.25%) ⬆️
unittests 73.13% <50.00%> (+0.29%) ⬆️

Flags with carried forward coverage won't be shown. Click here to find out more.

Files Coverage Δ
...ad/jcloud/impl/BlobStoreBackedInputStreamImpl.java 97.53% <100.00%> (+0.12%) ⬆️
...oad/jcloud/impl/BlobStoreBackedReadHandleImpl.java 77.85% <50.00%> (-2.29%) ⬇️
...d/jcloud/impl/BlobStoreBackedReadHandleImplV2.java 60.24% <16.66%> (-0.91%) ⬇️

... and 240 files with indirect coverage changes

@shibd shibd requested a review from dao-jun April 18, 2024 13:31
@Technoboy- Technoboy- added this to the 3.3.0 milestone Apr 19, 2024
@shibd shibd merged commit fbf4cb7 into apache:master Apr 19, 2024
49 of 52 checks passed
shibd added a commit that referenced this pull request Apr 19, 2024
shibd added a commit that referenced this pull request Apr 19, 2024
shibd added a commit that referenced this pull request Apr 19, 2024
mukesh-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 19, 2024
…tiered storage (apache#22531)

(cherry picked from commit fbf4cb7)
(cherry picked from commit ff8d3b7)
srinath-ctds pushed a commit to datastax/pulsar that referenced this pull request Apr 23, 2024
…tiered storage (apache#22531)

(cherry picked from commit fbf4cb7)
(cherry picked from commit ff8d3b7)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants