Skip to content

Conversation

@poorbarcode
Copy link
Contributor

Motivation

The steps of reproducing the issue:

  • We have a ledger, which contains 5 entries, the last confirmed is 4
  • Enable offloading for a topic
  • Read the first round: [0 ~ 3]
  • Read the second round [0 ~ 4], the error will occur.
2025-09-11T11:22:09,210+0000 [offloader-OrderedScheduler-1-0] ERROR org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedReadHandleImpl - Failed to read entries 0 - 10 from the offloader in ledger 1670362
java.io.EOFException: null
	at java.base/java.io.DataInputStream.readFully(Unknown Source) ~[?:?]
	at java.base/java.io.DataInputStream.readInt(Unknown Source) ~[?:?]
	at org.apache.bookkeeper.mledger.offload.jcloud.impl.BlobStoreBackedReadHandleImpl.lambda$readAsync$2(BlobStoreBackedReadHandleImpl.java:171) ~[?:?]
	at org.apache.bookkeeper.common.util.SingleThreadSafeScheduledExecutorService$SafeRunnable.run(SingleThreadSafeScheduledExecutorService.java:45) ~[io.streamnative-bookkeeper-common-4.17.2.2.jar:4.17.2.2]
	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) ~[?:?]
	at java.base/java.util.concurrent.FutureTask.run(Unknown Source) ~[?:?]
	at java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown Source) ~[?:?]
	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) ~[?:?]
	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) ~[?:?]
	at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) ~[io.netty-netty-common-4.1.124.Final.jar:4.1.124.Final]
	at java.base/java.lang.Thread.run(Unknown Source) [?:?]

Root cause

After the previous round of reading, the offset of the stream is the start position of the latest entry. If Pulsar tries to read [0 ~ LAC], it will encounter the bottom logic branch: skip the latest entry, then the next round look will receive an EOFException because the stream is empty.

https://github.com/apache/pulsar/blob/v4.1.0/tiered-storage/jcloud/src/main/java/org/apache/bookkeeper/mledger/offload/jcloud/impl/BlobStoreBackedReadHandleImpl.java#L210

if (entryId == nextExpectedId) {
    entries.add(LedgerEntryImpl.create(ledgerId, entryId, length, buf));
} else if (entryId > nextExpectedId && entryId < lastEntry) {
    seekToEntry(nextExpectedId);
} else if (entryId < nextExpectedId
    && !index.getIndexEntryForEntry(nextExpectedId).equals(index.getIndexEntryForEntry(entryId))) {
    seekToEntry(nextExpectedId);
} else if (entryId > lastEntry) {
    seekToEntry(nextExpectedId);
} else {
    long ignore = inputStream.skip(length);
}

Modifications

 public final int readInt() throws IOException {
    int ch1 = in.read();
    int ch2 = in.read();
    int ch3 = in.read();
    int ch4 = in.read();
    if ((ch1 | ch2 | ch3 | ch4) < 0)
        throw new EOFException(); // if the value is not an int, it throws an error here.
    return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0));
}

Documentation

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

Matching PR in forked repository

PR in forked repository: x

@poorbarcode poorbarcode added this to the 4.2.0 milestone Sep 17, 2025
@poorbarcode poorbarcode self-assigned this Sep 17, 2025
@github-actions github-actions bot added the doc-not-needed Your PR changes do not impact docs label Sep 17, 2025
@codecov-commenter
Copy link

codecov-commenter commented Sep 22, 2025

Codecov Report

❌ Patch coverage is 54.38596% with 52 lines in your changes missing coverage. Please review.
✅ Project coverage is 74.18%. Comparing base (8e35e34) to head (4baa0de).
⚠️ Report is 15 commits behind head on master.

Files with missing lines Patch % Lines
...oad/jcloud/impl/BlobStoreBackedReadHandleImpl.java 56.36% 38 Missing and 10 partials ⚠️
...ger/offload/jcloud/impl/OffloadIndexEntryImpl.java 0.00% 2 Missing and 2 partials ⚠️
Additional details and impacted files

Impacted file tree graph

@@             Coverage Diff              @@
##             master   #24753      +/-   ##
============================================
- Coverage     74.26%   74.18%   -0.09%     
+ Complexity    33635    33224     -411     
============================================
  Files          1900     1901       +1     
  Lines        148425   148474      +49     
  Branches      17209    17216       +7     
============================================
- Hits         110225   110139      -86     
- Misses        29426    29534     +108     
- Partials       8774     8801      +27     
Flag Coverage Δ
inttests 26.27% <ø> (-0.78%) ⬇️
systests 22.75% <ø> (-0.05%) ⬇️
unittests 73.71% <54.38%> (-0.07%) ⬇️

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

Files with missing lines Coverage Δ
...ger/offload/jcloud/impl/OffloadIndexEntryImpl.java 76.47% <0.00%> (-8.15%) ⬇️
...oad/jcloud/impl/BlobStoreBackedReadHandleImpl.java 63.18% <56.36%> (-14.60%) ⬇️

... and 95 files with indirect coverage changes

🚀 New features to boost your workflow:
  • ❄️ Test Analytics: Detect flaky tests, report on failures, and find test suite problems.
  • 📦 JS Bundle Analysis: Save yourself from yourself by tracking and limiting bundle sizes in JS merges.

@poorbarcode poorbarcode merged commit 3a98763 into apache:master Sep 22, 2025
96 of 98 checks passed
lhotari pushed a commit that referenced this pull request Sep 22, 2025
KannarFr pushed a commit to CleverCloud/pulsar that referenced this pull request Sep 22, 2025
@poorbarcode poorbarcode deleted the fix/BlobStoreBackedReadHandleImpl branch September 23, 2025 04:02
poorbarcode added a commit that referenced this pull request Sep 23, 2025
poorbarcode added a commit that referenced this pull request Sep 23, 2025
manas-ctds pushed a commit to datastax/pulsar that referenced this pull request Sep 26, 2025
srinath-ctds pushed a commit to datastax/pulsar that referenced this pull request Sep 26, 2025
manas-ctds pushed a commit to datastax/pulsar that referenced this pull request Sep 29, 2025
srinath-ctds pushed a commit to datastax/pulsar that referenced this pull request Sep 29, 2025
walkinggo pushed a commit to walkinggo/pulsar that referenced this pull request Oct 8, 2025
nodece pushed a commit to ascentstream/pulsar that referenced this pull request Oct 30, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants