You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/08/09 14:41:44 UTC

[GitHub] ivankelly opened a new issue #1591: Deferred failure handling can cause data loss

ivankelly opened a new issue #1591: Deferred failure handling can cause data loss
URL: https://github.com/apache/bookkeeper/issues/1591
 
 
   The bookkeeper client has a feature where if you have a ledger with a write quorum(Qw) larger than an ack quorum(Qa), such as 3:3:2, if (Qw-Qa) bookies return an error, after the entry add has completed, the erroring bookie will be replaced in the ensemble in the background.
   
   This can cause data loss.
   
   Consider a 3:3:2 ledger. Assume zookeeper is not accepting writes.
   
   Start ensemble is b1,b2,b3
   - e0 is written to b1,b2,b3
   - b2 & b3 acknowledge, e0 request completes
   - b1 responds with and error and is added to the delayed ensemble change list
   - e1 is written, delayed error handling kicks off, b1 is replaced, so e1 is written to b4,b2,b3 while the client tries to update the metadata in zookeeper. zookeeper is stalled so fails to respond.
   - repeat this sequence for each of b2 and b3.
   
   As each bookie fails, it will be replaced and writes will be acknowledged. Eventually the ensembles will look something like
   ```
   0: b1,b2,b3
   1: b4,b2,b3
   3: b4,b5,b3
   4: b4,b5,b6
   ```
   How ever, this is only local, zookeeper still only has the initial ensemble. So, even though all entries from 4 onwards are acknowledged successfully to the client, if another client comes to read the ledger, they will not see them. If the other client recovers the ledger, the data is lost. TOAB violation.
   
   Here's a test case which triggers the issue:
   https://github.com/ivankelly/bookkeeper/blob/15bc251d46d5cd5fcceef130c0046eeacbe446cc/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestDeferredFailure.java
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services