SongOf opened a new pull request, #4551:
URL: https://github.com/apache/bookkeeper/pull/4551
### Motivation
When the content of LedgerMetadata is as follows:
`Versioned(value=LedgerMetadata{formatVersion=3, ensembleSize=3,
writeQuorumSize=3, ackQuorumSize=2, state=CLOSED, length=42, lastEntryId=1,
digestType=CRC32C, password=base64:, ensembles={0=[10.167.101.44:3181,
10.145.144.76:3181, 10.145.136.51:3181], 1=[10.170.112.33:3181,
10.170.140.51:3181, 10.170.92.28:3181], 2=[10.171.7.2:3181, 10.172.180.82:3181,
10.172.149.89:3181]}, customMetadata={component=base64:bWFuYWdlZC1sZWRnZXI=,
pulsar/managed-ledger=base64:cHVibGljL2RlZmF1bHQvcGVyc2lzdGVudC9kaWNoYXRfcHJvZF9ldmVudF9sb2ctcGFydGl0aW9uLTI=,
pulsar/cursor=base64:Y2dfZGljaGF0X3Byb2RfZXZlbnRfbG9n,
application=base64:cHVsc2Fy}}, version=13)`
- this ledger is closed
- The firstEntryId of the last fragment of the ledger is 2. But the
lastEntryId of the ledger is 1.
- All bookies of the first fragment of this ledger have been offline. Such
as 10.167.101.44:3181,10.145.144.76:3181, 10.145.136.51:3181. Therefore,
entry(entryId=0) reading will fail.
Based on the above description, ReplicationWorker will replicate the first
and last fragment. The tryReadingFaultyEntries function will be called before
replicating.
`boolean tryReadingFaultyEntries(LedgerHandle lh, LedgerFragment
ledgerFragment)`
After the first fragment replica fails, the fragment will be skipped. At
this time, the value of unableToReadEntriesForReplication is <ledgerId=0,
entryIdsUnableToRead=<0>>.
When replicating the last fragment, tryReadingFaultyEntries will throw an
IllegalArgumentException("inconsistent range"). Which in turn causes the
ReplicationWorker process to exit.
The log is as follows:
`2025-02-05 20:16:16,041 [ DEBUG ] ReplicationWorker - Founds fragments
[Fragment(LedgerID: 449738, FirstEntryID: 0[0], LastKnownEntryID: 0[0], Host:
[10.145.136.51:3181, 10.167.101.44:3181, 10.145.144.76:3181], Closed: true),
Fragment(LedgerID: 449738, FirstEntryID: 1[1], LastKnownEntryID: 1[1], Host:
[10.170.92.28:3181, 10.170.112.33:3181, 10.170.140.51:3181], Closed: true),
Fragment(LedgerID: 449738, FirstEntryID: 2[-1], LastKnownEntryID: 1[-1], Host:
[10.171.7.2:3181, 10.172.149.89:3181, 10.172.180.82:3181], Closed: true)] for
replication from ledger: 449738`
From the log, we can see that the FirstEntryID of the last fragment is
greater than the LastKnownEntryID, which will cause the subSet function of the
ConcurrentSkipListSet class to throw an IllegalArgumentException.
The exception log is as follows:
`2025-02-05 16:51:23,786 [ ERROR ] BookieThread - Uncaught exception in
thread ReplicationWorker
java.lang.IllegalArgumentException: inconsistent range
at
java.util.concurrent.ConcurrentSkipListMap$SubMap.<init>(ConcurrentSkipListMap.java:2404)
~[?:?]
at
java.util.concurrent.ConcurrentSkipListMap.subMap(ConcurrentSkipListMap.java:1884)
~[?:?]
at
java.util.concurrent.ConcurrentSkipListSet.subSet(ConcurrentSkipListSet.java:416)
~[?:?]
at
org.apache.bookkeeper.replication.ReplicationWorker.tryReadingFaultyEntries(ReplicationWorker.java:316)`
### Changes
When the fragment's FirstEntryID is greater than LastKnownEntryID,
tryReadingFaultyEntries directly returns true.
--
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]