Skip to content

Conversation

@ArvinDevel
Copy link
Contributor

Descriptions of the changes in this PR:

Motivation

To guarantee high durability, BK write journal before flush data to persistent device which will cause two write of data.
At the presence of replicating and auto-recovery mechanism, the two-write is a bit waste of the persistent device bandwidth,
especially on the scenarios which prefer weak durability guarantee.
This proposal is aimed at providing bypass journal ledger, this feature includes these parts work:

  • add new write flag BYPASS_JOURNAL to existing protocol
  • impl the newly write flag at the client side and server side

Changes

(Describe: what changes you have made)

Master Issue: #1945


In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit
checks for pull requests. A pull request can only be merged when it passes precommit checks. However running all
the precommit checks can take a long time, some trivial changes don't need to run all the precommit checks. You
can check following list to skip the tests that don't need to run for your pull request. Leave them unchecked if
you are not sure, committers will help you:

  • [skip bookkeeper-server bookie tests]: skip testing org.apache.bookkeeper.bookie in bookkeeper-server module.
  • [skip bookkeeper-server client tests]: skip testing org.apache.bookkeeper.client in bookkeeper-server module.
  • [skip bookkeeper-server replication tests]: skip testing org.apache.bookkeeper.replication in bookkeeper-server module.
  • [skip bookkeeper-server tls tests]: skip testing org.apache.bookkeeper.tls in bookkeeper-server module.
  • [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module.
  • [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests.
  • [skip build java8]: skip build on java8. ONLY skip this when ONLY changing files under documentation under site.
  • [skip build java11]: skip build on java11. ONLY skip this when ONLY changing files under documentation under site.


Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

If this PR is a BookKeeper Proposal (BP):

  • Make sure the PR title is formatted like:
    <BP-#>: Description of bookkeeper proposal
    e.g. BP-1: 64 bits ledger is support
  • Attach the master issue link in the description of this PR.
  • Attach the google doc link if the BP is written in Google Doc.

Otherwise:

  • Make sure the PR title is formatted like:
    <Issue #>: Description of pull request
    e.g. Issue 123: Description ...
  • Make sure tests pass via mvn clean apache-rat:check install spotbugs:check.
  • Replace <Issue #> in the title with the actual Issue number.

@jvrao
Copy link
Contributor

jvrao commented Feb 12, 2019

Glad to see this is being worked on. Will review the proposal.


To guarantee high durability, BK write journal before flush data to persistent device which will cause two write of data.
At the presence of replicating and auto-recovery mechanism, the two-write is a bit waste of the persistent device bandwidth,
especially on the [scenarios](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-14+Relax+durability) which prefer weak durability guarantee.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't believe our auto-recovery and replication alone can address our persistence needs. I would request you to reword it saying that - "we may not need this level of persistence under scenarios like week durability....." something like that.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

done

At the presence of replicating and auto-recovery mechanism, the two-write is a bit waste of the persistent device bandwidth,
especially on the [scenarios](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-14+Relax+durability) which prefer weak durability guarantee.
This proposal is aimed at providing bypass journal ledger, this feature includes these parts work:
- add new write flag `BYPASS_JOURNAL` to existing protocol
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I assume BYPASS_JOURNAL is applicable only in the week durability case. We have that in the protocol, so can't BYPASS_JOURNAL be just a configuration parameter on bookie, which will used only (if enabled) on week durability case?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When user create LedgerHandle, he/she can construct it with WriteFlag.BYPASS_JOURNAL

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jvrao I think we cannot have a bookie configuration parameter because the client must be aware of relaxed durability and advance LAC accordingly (like we do with DEFERRED_SYNC)

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@jvrao If I remember this correctly, we were thinking of adding BYPASS_JOURNAL as a WriteFlag, no? Applications can decide whether to use journal or not.

especially on the [scenarios](https://cwiki.apache.org/confluence/display/BOOKKEEPER/BP-14+Relax+durability) which prefer weak durability guarantee.
This proposal is aimed at providing bypass journal ledger, this feature includes these parts work:
- add new write flag `BYPASS_JOURNAL` to existing protocol
- impl the newly write flag at the client side and server side
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do you mean client visible API change ? or where exactly is this flag?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. What I mean is extending WriteFlags, the user who wants using this feature only needs pass WriteFlags to the WriteHandle. looks like this:

newCreateLedgerOp()
                .withEnsembleSize(3)
                .withWriteQuorumSize(3)
                .withAckQuorumSize(3)
                .withPassword(PASSWORD)
                .withWriteFlags(WriteFlag.BYPASS_JOURNAL)
                .execute()

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@ArvinDevel it would be good if you can put the example code in the BP.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will do that


Modify server side code mostly and don't change legerHandle's LAC advance logic, if the write flag is `BYPASS_JOURNAL`, after write to `LegerStorage`(the data maybe in the memTable, or the buffer of File, or the os cache),
bookie return result to the client directly.
This impl is like [disable syncData](https://github.com/apache/bookkeeper/issues/753), once all the replica fails, the BK cluster can't recovery from it.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

With this the above will be obsolete?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The bookie level config syncData example is just used to compare. We still needs to modify server side to sense client's WriteFlag.BYPASS_JOURNAL option, and extend WriteFlag to give users this choice.


1. Relax LAC protocol

Modify server side code mostly and don't change legerHandle's LAC advance logic, if the write flag is `BYPASS_JOURNAL`, after write to `LegerStorage`(the data maybe in the memTable, or the buffer of File, or the os cache),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

don't change legerHandle's LAC advance logic

I think we should use the DEFERRED_SYNC way of handling LAC, that is that a regular write will not advance LAC, but you need a "force" (like you state below).

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The difficult is how to execute "force", leave to app or client lib, and how often to schedule force if WriteHandle is responsible for that. Do you have any ideas?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

force() RPC may carry a flag on the wire which tells that the clients wants to fait for a flush of the memtable for the given ledger.
force() with DEFERRED_SINC -> wait for/force a write to the journal
force() with BYPASS_JOURNAL -> wait for/force a flush on the EntryLogger

It makes sense only in conjunction with "multiple entry loggers"/"one entry logger per ledger" feature.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes, but I'm more worried about that is the synchronized "force" applicable for bypass-journal weak durability. On the "deferred_sync" scenario, the consumer/ReadHandle can tolerant reading after "force" or close. If we still restrict the reader can only read up to the point where writer "force", will this limit the applied scenarios of bypass-journal weak durability?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I agree with @eolivelli. BYPASS_JOURNAL should have same LAC semantics as DEFERRED_SYNC. We only advanced LAC when force happen. force will carry the write flag and decide to force a write to the journal, or force a flush on ledger storage.

For bypass journal use cases, I think applications are usually when a ledger is closed/forced. So the above assumptions would make implementation easier.


- Add persistent callback to LedgerStorage

Maintain non-persistent entry list and `maxPersistentEntryId` on `LedgerDescriptor`.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I tried to have a 'maxPersistentEntryId' while working on DEFERRED_SYNC but actually it was not possible because if you use "LedgerHandleAdv" entries won't get to the bookie with a specific order, so you will have to keep track of holes.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you're right, thanks


- Client side changes

Add `nonPersistentLAC` to WriteHandle. WriteHandle with 'bypass-journal' option updates the LAC using `maxPersistentEntryId`, and update `nonPersistentLAC` if receives enough ack.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nonPersistentLAC is more like 'pendingAddsSequenceHead' that we introduced for DEFERRED_SYNC

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, and I forgot to state something to complete nonPersistentLAC :

  • carry nonPersistentLAC to bookie, and LedgerDesciptor record it
  • extend readEntry of ReadHandle, so that it can read up to nonPersistentLAC

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why persisting an non-persistent value ?
The value makes sense only on the client (writer), you won't ever read that value from bookies

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What I want is relaxing the reader to read up to latest entry as early as possible.

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I very like this proposal and I think we should go for the 'WriteFlag' way.

thank you for working on this @ArvinDevel

@eolivelli
Copy link
Contributor

When you will go thru the implementation remember to create patches for the Server-side first and then we will be able to work on the client side, because client-side changes are "public API" and we cannot release an API which is missing the server side counterpart

@eolivelli
Copy link
Contributor

@ArvinDevel this doc does not deal with fencing.
We should state how with this new feature we are providing this core feature of BK.

With DEFERRED_SYNC for instance we are not changing how recovery/fencing works.
Should we store the fence flag on journal ? or does the fence operation need a flush of the FileInfo to disk ?

@ArvinDevel
Copy link
Contributor Author

We should state how with this new feature we are providing this core feature of BK.

To be honest, I'm not familiar with fencing stuff. To simplify the design, can we keep the simple design which not changing recovery/fencing? Since the fence info is small enough, store it to journal has little effect.

@ArvinDevel
Copy link
Contributor Author

@jvrao @eolivelli @sijie @merlimat
Most of you are agreed to implement BYPASS_JOURNAL using WRITE_FLAG, and use "force" api to advance LAC, then I'll focus on this method on the doc, ok?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants