hangc0276 opened a new pull request, #3917:
URL: https://github.com/apache/bookkeeper/pull/3917

   ### Motivation
   When I decommission one bookie (bk3), one ledger replicate failed and 
blocked decommission process.
   
   This is the auto-recovery log:
   ```
   2023-03-29T06:29:22,642+0000 [ReplicationWorker] ERROR 
org.apache.bookkeeper.client.LedgerHandle - ReadEntries exception on 
ledgerId:904368 firstEntry:14 lastEntry:14 lastAddConfirmed:13
   2023-03-29T06:29:22,642+0000 [ReplicationWorker] ERROR 
org.apache.bookkeeper.replication.ReplicationWorker - Received error: -1 while 
trying to read entry: 14 of ledger: 904368 in ReplicationWorker
   2023-03-29T06:29:22,642+0000 [ReplicationWorker] ERROR 
org.apache.bookkeeper.replication.ReplicationWorker - Failed to read faulty 
entries, so giving up replicating ledgerFragment Fragment(LedgerID: 904368, 
FirstEntryID: 0[0], LastKnownEntryID: 14[14], Host: 
[betausc1-bk-10.betausc1-bk-headless.o-vaxkx.svc.cluster.local:3181], Closed: 
true)
   2023-03-29T06:29:22,644+0000 [ReplicationWorker] ERROR 
org.apache.bookkeeper.replication.ReplicationWorker - ReplicationWorker failed 
to replicate Ledger : 904368 for 6 number of times, so deferring the ledger 
lock release by 300000 msecs
   ```
   The ledger's metadata:
   ```
   ledgerID: 904368
   2023-03-29T06:47:56,511+0000 [main] INFO  
org.apache.bookkeeper.tools.cli.commands.
   client.LedgerMetaDataCommand - LedgerMetadata{formatVersion=3, 
ensembleSize=3, writeQuorumSize=3, 
   ackQuorumSize=2, state=OPEN, digestType=CRC32C, password=base64:, 
   ensembles={0=[bk1:3181, bk2:3181, bk3:3181], 15=[bk1:3181, bk2:3181, 
bk4:3181]},...}
   ```
   
   The ledger (904368) has two ensembles, `ensembles={0=[bk1:3181, bk2:3181, 
bk3:3181], 15=[bk1:3181, bk2:3181, bk4:3181]}`. However, the replication worker 
got the ledger's LAC is 13, but it got the replication fragment entry range is 
[0, 14]. When reading entry 14, it failed.
   
   In checking the ledger to find the missing replicas fragment, we use the 
next ensemble's `key - 1` as the current fragment's lastKnownEntryId, and it 
doesn't consider the ledger's lastAddConfirm.
   
https://github.com/apache/bookkeeper/blob/912896deb2e748389e15e74c37539b2ff36302c7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerChecker.java#L384-L396
   
   **One question is why the ensembles created a new ensemble starting with 
entryId = 15, but the ledger's lastAddConfirm is 13.**
   
   This question is related to two parts, one is how the new ensemble was 
created and the other is how the lastAddConfirm was generated.
   
   #### How the new ensemble was created
   The ensemble change is controlled on the bookie client side. 
   
   When one entry is ready to send to the bookie server, the bookie client will 
check whether need to do the ensemble change.
   
https://github.com/apache/bookkeeper/blob/912896deb2e748389e15e74c37539b2ff36302c7/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L254
   
   For the above case, when writing entry 15, one bookie is lost, it will 
trigger the ensemble change and generate the new ensemble: 15=[bk1:3181, 
bk2:3181, bk4:3181]. However, entry 15 write failed, such as timeout or bookie 
rejected the write.
   
   For now, entry 14 is written succeed.
   
   #### How the lastAddConfirm was generated
   Due to the ledger being in the `OPEN` state, the ledger handle will send a 
readLAC request to get the ledger's lastAddConfirm. 
   
   For the above case, if bk1 holds the max entry 14, bk2 holds the max entry 
13 and bk3 is lost, the LedgerHandle get lastAddConfirm will be 13, not 14.
   
   ### Changes
   When generating missing replica fragments' entry range, take the ledger's 
lastAddConfirm into consideration.
   


-- 
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]

Reply via email to