Status

Current state: Rejected

Discussion thread: link

JIRA:   Unable to render Jira issues macro, execution error.

Released: N/A

Problem

We want to ensure that some "important" entries will be readable for which the writer has received the acknowledge from a quorum of bookies without being able to piggyback the LastAddConfirmed or close the LedgerHandle (a writer which crashes).

This is the simplest failing scenario:

  • A writer creates a ledger
  • The writer adds and entry and blocks for the acknowldege of the configured quorum of Bookies
  • The writer crashes
  • LAC has not been sent to Bookies and it has not been written to metadata
  • A recovery is performed, truncating the ledger to the maximum LastAddConfirmed entry ID
  • Now the entry is no more readable and there is no trace of it on metadata so it cannot be recovered

 

This problem is fatal in the case of using BookKeeper as a write-ahead log for a transactional database, this is an example:

  1. The writer is a SQL DBMS, which uses BookKeeper as a write-ahead log, so it writes to the log every 'change' to data and than applies that change in its own representation (memory, disk) of the table
  2. A client for the DBMS issues an "insert" and than a "commit", the DBMS writes to the log and assumes that the write will not be lost (the log is a "durable storage")
  3. The client receives an ACK for the commit, and even performs queries on the table an "sees" the new record
  4. The DBMS crashes and a recovery is to be performed from the log
  5. The entry with the commit (the last written one) is lost, so the transaction is not considered "committed" and now the record does not exists anymode
  6. From the client point of view the "durability" property of the transaction has not been respected

Design

We can add a new BookKeeper function, addConfirmedEntry  which acks like the addEntry function but sends on the protocol a new flag which tells to the Bookie to "advance" the LastAddConfirmed flag immediately as we already do with the ExplicitLAC.

TODO: report exact changes to the protocol

Notes:

  • This way we lose the property of BookKeeper which prevents the readers to 'see' unconfirmed entries from the writer. This is to be clearly documented and it is good to have a specific client-side function
  • This problem can be mitigated by sending Explicit LACs with a very little timeout, but anyway there is no guarantee of not losing the entry.

Actions:

  • Related JIRA Unable to render Jira issues macro, execution error.
  • TODO: create a PR

 

Related works:

  • The ExpliticLAC feature on 4.5.0 has enabled the Bookie side to support the update of the LastAddConfirmed out of the regular LAC protocol.
  • The BP-5 Allow reads outside the LAC Protocol feature is dual in repect to this feature, as it let the readers see entries which as been confirmed by the write out-of-bound from the BookKeeper protocol

 

  • No labels