Il mar 28 mar 2017, 17:45 Venkateswara Rao Jujjuri <jujj...@gmail.com> ha
scritto:

> We have Explicit LAC updates now, and I recall you have made it more
> granular (milli seconds)
> given that, I am just wondering the need of your out of band communication
> and taking an alternate route
> in the client code.
>

JV,
I am working to a new project that is essentialy a distributed large object
store, like you are doing at SF.
I am currently using the explicit LAC with I was waiting for long time.
After some benchs me and my colleagues start thinking if there was another
way to achieve the same goal and maybe using less resources.
In my project a client writes an object and only once the write has been
acknowledged the object is considered published on the store. So it must be
readable immediately by other clients.
As the coordination in this case is already done from other services
external to BookKeeper the LAC is not very useful IMHO and so maybe we can
just simply skip the check.

I wonder if this can have some issues

I am going to release the project as open-source as soon as the alpha
version will be usable, so that I can share the code and diaccuss better
with the community


> Thanks,
> JV
>
> On Tue, Mar 28, 2017 at 7:36 AM, Enrico Olivelli <eolive...@gmail.com>
> wrote:
>
> > Hi all,
> > I have just created this issue about a new small "feature"
> > https://issues.apache.org/jira/browse/BOOKKEEPER-1019.
> >
> > The patch is very simple and I wonder if I am missing something.
> >
> > I'm copying the description of the issue in this email in order to start
> > some discussion
> >
> > 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:
> >
> >         if (lastEntry > lastAddConfirmed) {
> >             LOG.error("ReadException on ledgerId:{} firstEntry:{}
> > lastEntry:{}",
> >                     new Object[] { ledgerId, firstEntry, lastEntry });
> >             cb.readComplete(BKException.Code.ReadException, this, null,
> > ctx);
> >             return;
> >         }
> >
> > this is my proposal:
> >
> >         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;
> >         }
> >
> >
> > Does this make sense to any of you ?
> >
> > Enrico
> >
>
>
>
> --
> Jvrao
> ---
> First they ignore you, then they laugh at you, then they fight you, then
> you win. - Mahatma Gandhi
>
-- 


-- Enrico Olivelli

Reply via email to