Fix ledger replicated failed blocks bookie decommission process#3917
Conversation
eolivelli
left a comment
There was a problem hiding this comment.
I wonder of we risk to lose one entry in case of AQ < WQ.
I have to think more
bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
Show resolved
Hide resolved
I have a question about this. There are two places to invoke LedgerChecker#checkLedger. 2. Both places use the openLedgerNoRecovery to open the LedgerHandle. It will use lh.asyncReadLastConfirmed() to get the LAC. And we use the V2 protocol, it invokes In asyncReadPiggybackLastConfirmed, it invokes ReadLastConfirmedOp#initiate() to get the LAC. In the ReadLastConfirmedOp, it will read all bookies in the currentEnsemble. The currentEnsemble is So if the bk1 last entry is 14, the lac is 14-1 = 13 line_l08-line_112, it will pick the max lac to override maxRecoveredData. line_137-line_148, use the maxRecoveredData to invoke callback. |
@horizonzy Yes, you are right. I updated the description, please help take a look, thanks. |
@eolivelli Sorry, I misunderstood the lastAddConfirm part. It doesn't have the risk of losing one entry. I updated the description, please help take a look, thanks. |
| this.isLedgerClosed = lh.getLedgerMetadata().isClosed() | ||
| || !ensemble.equals(ensembles.get(ensembles.lastKey())); | ||
| || (!ensemble.equals(ensembles.get(ensembles.lastKey())) | ||
| && lh.getLastAddConfirmed() >= ensembles.lastKey() - 1); |
There was a problem hiding this comment.
Here should be `lh.getLastAddConfirmed() == ensembles.lastKey() - 2.
The lac maybe 100. The last ensemble key maybe 10.
There was a problem hiding this comment.
Maybe not. Take the following case for example:
0: [bk1, bk2, bk3]
10: [bk1, bk2, bk4]
LAC = 100
Ledger is OPEN
For the first ensemble: isLedgerClosed = (false || (true && 100 >= 9)) => true
For the last ensemble: isLedgerClosed = (false || (false && 100 >= 9)) => false
The result is expected.
| LedgerHandle lh = bkc.createLedger(3, 3, 2, BookKeeper.DigestType.CRC32, TESTPASSWD); | ||
| assertFalse(lh.getLedgerMetadata().isClosed()); | ||
|
|
||
| List<BookieId> firstEnsemble = lh.getLedgerMetadata().getAllEnsembles().firstEntry().getValue(); |
| * segment/ensemble has missing bookies. | ||
| */ | ||
| private boolean isLastSegmentOpenAndMissingBookies(LedgerHandle lh) throws BKException { | ||
| LedgerMetadata md = admin.getLedgerMetadata(lh); |
| SortedMap<Long, ? extends List<BookieId>> ensembles = admin.getLedgerMetadata(lh).getAllEnsembles(); | ||
| List<BookieId> finalEnsemble = ensembles.get(ensembles.lastKey()); | ||
| if (ensembles.size() > 1 && lh.getLastAddConfirmed() < ensembles.lastKey() - 1) { | ||
| finalEnsemble = new ArrayList<>(finalEnsemble); |
There was a problem hiding this comment.
finalEnsemble = new ArrayList<>(finalEnsemble); may be meaningless.
There was a problem hiding this comment.
The finalEnsemble is immutable in the ledger metadata.
|
@merlimat @eolivelli @dlg99 Please help take a look, thanks. |
### 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.
### One question
**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.
#### 1. 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 server rejected the write.
For now, entry 14 is written succeed.
#### 2. How the lastAddConfirm was generated
Due to the ledger being in the `OPEN` state, the ledger handle will send a readLAC request according to the last ensemble to get the ledger's lastAddConfirm.
For the above case, the readLAC request will send to bk1, bk2, and bk4.
For the `V2` protocol (Pulsar uses the V2 protocol to interact with the BookKeeper cluster), the bookie client put the lastAddConfirm EntryId in the next Entry's metadata.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java#L134
When we use the `V2` protocol to open an `OPEN` state ledger to read, it will send a readLastAddConfirm request to the bookie server, and the bookie server gets the last entry of this ledger and return to the client.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java#L108
However, the bookie client will parse the response entry and get the lastAddConfirm from the entry's metadata. Due to the entry just recording the previous EntryId as the lastAddConfirm, the LedgerHandle got the lastAddConfirm will be the penultimate EntryId of the ledger.
For the above case, the bk1 holds the max entry 14, bk2 holds the max entry 14, and bk4 returns NoSuchEntryException, LedgerHandle gets lastAddConfirm will be `14 - 1 = 13`, not 14.
When the replicator tries to recover the first ensemble 0=[bk1:3181, bk2:3181, bk3:3181] with entry range [0, 14], reading entry 14 will throw a ReadEntryException due to the lastAddConfirm is 13.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L685-L690
### Solution
When encountered that case that
- The ledger is `OPEN`
- The ledger has multiple ensembles
- The ledger's last ensemble doesn't have any entries, which means `lastAddConfirm < last ensemble key - 1`
We should treat the penultimate segment/ensemble of the ledger as an `OPEN` state instead of a closed state.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java#L56-L57
After we treat the segment/ensemble as `OPEN` state, the replicator will close the ledger first and replicate it.
(cherry picked from commit eff38e4)
…he#3917) 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. **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. 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 server rejected the write. For now, entry 14 is written succeed. Due to the ledger being in the `OPEN` state, the ledger handle will send a readLAC request according to the last ensemble to get the ledger's lastAddConfirm. For the above case, the readLAC request will send to bk1, bk2, and bk4. For the `V2` protocol (Pulsar uses the V2 protocol to interact with the BookKeeper cluster), the bookie client put the lastAddConfirm EntryId in the next Entry's metadata. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java#L134 When we use the `V2` protocol to open an `OPEN` state ledger to read, it will send a readLastAddConfirm request to the bookie server, and the bookie server gets the last entry of this ledger and return to the client. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java#L108 However, the bookie client will parse the response entry and get the lastAddConfirm from the entry's metadata. Due to the entry just recording the previous EntryId as the lastAddConfirm, the LedgerHandle got the lastAddConfirm will be the penultimate EntryId of the ledger. For the above case, the bk1 holds the max entry 14, bk2 holds the max entry 14, and bk4 returns NoSuchEntryException, LedgerHandle gets lastAddConfirm will be `14 - 1 = 13`, not 14. When the replicator tries to recover the first ensemble 0=[bk1:3181, bk2:3181, bk3:3181] with entry range [0, 14], reading entry 14 will throw a ReadEntryException due to the lastAddConfirm is 13. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L685-L690 When encountered that case that - The ledger is `OPEN` - The ledger has multiple ensembles - The ledger's last ensemble doesn't have any entries, which means `lastAddConfirm < last ensemble key - 1` We should treat the penultimate segment/ensemble of the ledger as an `OPEN` state instead of a closed state. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java#L56-L57 After we treat the segment/ensemble as `OPEN` state, the replicator will close the ledger first and replicate it. (cherry picked from commit eff38e4)
### 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.
### One question
**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.
#### 1. 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 server rejected the write.
For now, entry 14 is written succeed.
#### 2. How the lastAddConfirm was generated
Due to the ledger being in the `OPEN` state, the ledger handle will send a readLAC request according to the last ensemble to get the ledger's lastAddConfirm.
For the above case, the readLAC request will send to bk1, bk2, and bk4.
For the `V2` protocol (Pulsar uses the V2 protocol to interact with the BookKeeper cluster), the bookie client put the lastAddConfirm EntryId in the next Entry's metadata.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java#L134
When we use the `V2` protocol to open an `OPEN` state ledger to read, it will send a readLastAddConfirm request to the bookie server, and the bookie server gets the last entry of this ledger and return to the client.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java#L108
However, the bookie client will parse the response entry and get the lastAddConfirm from the entry's metadata. Due to the entry just recording the previous EntryId as the lastAddConfirm, the LedgerHandle got the lastAddConfirm will be the penultimate EntryId of the ledger.
For the above case, the bk1 holds the max entry 14, bk2 holds the max entry 14, and bk4 returns NoSuchEntryException, LedgerHandle gets lastAddConfirm will be `14 - 1 = 13`, not 14.
When the replicator tries to recover the first ensemble 0=[bk1:3181, bk2:3181, bk3:3181] with entry range [0, 14], reading entry 14 will throw a ReadEntryException due to the lastAddConfirm is 13.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L685-L690
### Solution
When encountered that case that
- The ledger is `OPEN`
- The ledger has multiple ensembles
- The ledger's last ensemble doesn't have any entries, which means `lastAddConfirm < last ensemble key - 1`
We should treat the penultimate segment/ensemble of the ledger as an `OPEN` state instead of a closed state.
https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java#L56-L57
After we treat the segment/ensemble as `OPEN` state, the replicator will close the ledger first and replicate it.
(cherry picked from commit eff38e4)
…he#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. ### One question **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. #### 1. 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 server rejected the write. For now, entry 14 is written succeed. #### 2. How the lastAddConfirm was generated Due to the ledger being in the `OPEN` state, the ledger handle will send a readLAC request according to the last ensemble to get the ledger's lastAddConfirm. For the above case, the readLAC request will send to bk1, bk2, and bk4. For the `V2` protocol (Pulsar uses the V2 protocol to interact with the BookKeeper cluster), the bookie client put the lastAddConfirm EntryId in the next Entry's metadata. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java#L134 When we use the `V2` protocol to open an `OPEN` state ledger to read, it will send a readLastAddConfirm request to the bookie server, and the bookie server gets the last entry of this ledger and return to the client. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java#L108 However, the bookie client will parse the response entry and get the lastAddConfirm from the entry's metadata. Due to the entry just recording the previous EntryId as the lastAddConfirm, the LedgerHandle got the lastAddConfirm will be the penultimate EntryId of the ledger. For the above case, the bk1 holds the max entry 14, bk2 holds the max entry 14, and bk4 returns NoSuchEntryException, LedgerHandle gets lastAddConfirm will be `14 - 1 = 13`, not 14. When the replicator tries to recover the first ensemble 0=[bk1:3181, bk2:3181, bk3:3181] with entry range [0, 14], reading entry 14 will throw a ReadEntryException due to the lastAddConfirm is 13. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L685-L690 ### Solution When encountered that case that - The ledger is `OPEN` - The ledger has multiple ensembles - The ledger's last ensemble doesn't have any entries, which means `lastAddConfirm < last ensemble key - 1` We should treat the penultimate segment/ensemble of the ledger as an `OPEN` state instead of a closed state. https://github.com/apache/bookkeeper/blob/df4492012cc03682534cbc8dd68dd81163b0c947/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java#L56-L57 After we treat the segment/ensemble as `OPEN` state, the replicator will close the ledger first and replicate it.
Motivation
When I decommission one bookie (bk3), one ledger replicate failed and blocked decommission process.
This is the auto-recovery log:
The ledger's metadata:
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.One question
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.
1. 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.
bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
Line 254 in 912896d
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 server rejected the write.
For now, entry 14 is written succeed.
2. How the lastAddConfirm was generated
Due to the ledger being in the
OPENstate, the ledger handle will send a readLAC request according to the last ensemble to get the ledger's lastAddConfirm.For the above case, the readLAC request will send to bk1, bk2, and bk4.
For the
V2protocol (Pulsar uses the V2 protocol to interact with the BookKeeper cluster), the bookie client put the lastAddConfirm EntryId in the next Entry's metadata.bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/checksum/DigestManager.java
Line 134 in df44920
When we use the
V2protocol to open anOPENstate ledger to read, it will send a readLastAddConfirm request to the bookie server, and the bookie server gets the last entry of this ledger and return to the client.bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadLastConfirmedOp.java
Line 108 in df44920
However, the bookie client will parse the response entry and get the lastAddConfirm from the entry's metadata. Due to the entry just recording the previous EntryId as the lastAddConfirm, the LedgerHandle got the lastAddConfirm will be the penultimate EntryId of the ledger.
For the above case, the bk1 holds the max entry 14, bk2 holds the max entry 14, and bk4 returns NoSuchEntryException, LedgerHandle gets lastAddConfirm will be
14 - 1 = 13, not 14.When the replicator tries to recover the first ensemble 0=[bk1:3181, bk2:3181, bk3:3181] with entry range [0, 14], reading entry 14 will throw a ReadEntryException due to the lastAddConfirm is 13.
bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
Lines 685 to 690 in df44920
Solution
When encountered that case that
OPENlastAddConfirm < last ensemble key - 1We should treat the penultimate segment/ensemble of the ledger as an
OPENstate instead of a closed state.bookkeeper/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerFragment.java
Lines 56 to 57 in df44920
After we treat the segment/ensemble as
OPENstate, the replicator will close the ledger first and replicate it.