[
https://issues.apache.org/jira/browse/BOOKKEEPER-1019?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15950851#comment-15950851
]
ASF GitHub Bot commented on BOOKKEEPER-1019:
--------------------------------------------
GitHub user eolivelli opened a pull request:
https://github.com/apache/bookkeeper/pull/121
BOOKKEEPER-1019 Support for reading entries after LAC
This patch introduces a new client-side configuration option to allow reads
outside the boundary of the local LastAddConfirmed value.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/eolivelli/bookkeeper
BOOKKEEPER-1019-read-after-lac
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/bookkeeper/pull/121.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #121
----
commit 1edc56a21a9a59946c454072e067bb8e46097def
Author: eolivelli <[email protected]>
Date: 2017-03-31T13:19:24Z
BOOKKEEPER-1019 Support for reading entries after LAC
----
> Support for reading entries after LAC (causal consistency driven by
> out-of-band communications)
> -----------------------------------------------------------------------------------------------
>
> Key: BOOKKEEPER-1019
> URL: https://issues.apache.org/jira/browse/BOOKKEEPER-1019
> Project: Bookkeeper
> Issue Type: New Feature
> Components: bookkeeper-client
> Affects Versions: 4.4.0
> Reporter: Enrico Olivelli
> Assignee: Enrico Olivelli
> Fix For: 4.5.0
>
>
> Currently we check in asyncReadEntries that the range of entries is within
> the range 0....LastAddConfirmed.
> This is because the LAC guarantees that the client can read only entries that
> have been acked from the writer.
> The LAC protocol is very useful when there is not direct communication
> between "writers" and "readers".
> I have an use case in which the "writer" blocks until the write is acked
> (like addEntry) and then it takes the returned id (ledgerId + entryid) and
> passes it to a "reader" which in turn tries to read the entry.
> This communication is done out-of-band in respect to BookKeeper and we can
> assume that the entries has been stored in a durable way (the write as been
> acked by a quorum of bookies).
> As the 'reader' as received a confirmation the the writer as successifully
> written the entry it can read it without waiting for the piggyback of the LAC
> of the standard bookkeeper protocol.
> This is the normal way of working with transactional databases or with
> filesystems.
> This is kind of "causal consistency".
> The idea is to add a configuration option to relax the check in
> asyncreadEntries
> this is 4.4 version:
> {code}
> if (lastEntry > lastAddConfirmed) {
> LOG.error("ReadException on ledgerId:{} firstEntry:{}
> lastEntry:{}",
> new Object[] { ledgerId, firstEntry, lastEntry });
> cb.readComplete(BKException.Code.ReadException, this, null, ctx);
> return;
> }
> {code}
> this is my proposal:
> {code}
> if (lastEntry > lastAddConfirmed &&
> !allowReadingAfterLastAddConfirmed) {
> LOG.error("ReadException on ledgerId:{} firstEntry:{}
> lastEntry:{}",
> new Object[] { ledgerId, firstEntry, lastEntry });
> cb.readComplete(BKException.Code.ReadException, this, null, ctx);
> return;
> }
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)