NoSuchElementException occured during ledger replication
BUG REPORT
Describe the bug Bookeeper version : 4.14.7
The following exception occurred in our production environment,and Bookie process shutdown.
I searched for the issue and did not find any similar errors
We added some logs and found that when replicate a certain closed ledger, this error is reported
the ledger metadata here below
we do some dig into codes but I'm not sure if there's a problem here
` org.apache.bookkeeper.client.LedgerFragmentReplicator#splitIntoSubFragments
static Set<LedgerFragment> splitIntoSubFragments(LedgerHandle lh,
LedgerFragment ledgerFragment, long rereplicationEntryBatchSize) {
Set<LedgerFragment> fragments = new HashSet<LedgerFragment>();
if (rereplicationEntryBatchSize <= 0) {
// rereplicationEntryBatchSize can not be 0 or less than 0,
// returning with the current fragment
fragments.add(ledgerFragment);
return fragments;
}
long firstEntryId = ledgerFragment.getFirstStoredEntryId();
long lastEntryId = ledgerFragment.getLastStoredEntryId();
/*
* if firstEntryId is INVALID_ENTRY_ID then lastEntryId should be
* INVALID_ENTRY_ID and viceversa.
*/
if (firstEntryId == INVALID_ENTRY_ID ^ lastEntryId == INVALID_ENTRY_ID) {
LOG.error("For LedgerFragment: {}, seeing inconsistent firstStoredEntryId: {} and lastStoredEntryId: {}",
ledgerFragment, firstEntryId, lastEntryId);
assert false;
}
long numberOfEntriesToReplicate = (lastEntryId - firstEntryId) + 1;
long splitsWithFullEntries = numberOfEntriesToReplicate
/ rereplicationEntryBatchSize;
if (splitsWithFullEntries == 0) {// only one fragment
fragments.add(ledgerFragment);
return fragments;
}
long fragmentSplitLastEntry = 0;
for (int i = 0; i < splitsWithFullEntries; i++) {
fragmentSplitLastEntry = (firstEntryId + rereplicationEntryBatchSize) - 1;
fragments.add(new LedgerFragment(lh, firstEntryId,
fragmentSplitLastEntry, ledgerFragment.getBookiesIndexes()));
firstEntryId = fragmentSplitLastEntry + 1;
}
long lastSplitWithPartialEntries = numberOfEntriesToReplicate
% rereplicationEntryBatchSize;
if (lastSplitWithPartialEntries > 0) {
fragments.add(new LedgerFragment(lh, firstEntryId, firstEntryId
+ lastSplitWithPartialEntries - 1, ledgerFragment
.getBookiesIndexes()));
}
return fragments;
}
`
the numberOfEntriesToReplicate variable can be wrong(assigned 1) when firstEntryId == INVALID_ENTRY_ID && lastEntryId == INVALID_ENTRY_ID, It should be assigned 0. Then it leads to the above error message when the control reaches to new LedgerFragment line.
Is it a bug? I’v reviewed the master branch and it has not been fixed yet.
To Reproduce
Is hard to reproduce in test env.
Expected behavior
the ledger replicate can be completed and replicate next.