Well it boils down to out-of-band or in-band communication. With out of
band it may not scale. 1 writer and 100 readers.
In your case it may work, and I dont see any recovery issue. But it is
possible that you don't read 7 first time and later on after recovery you
can read 7 in your example.



On Wed, Mar 29, 2017 at 8:03 AM, Enrico Olivelli <eolive...@gmail.com>
wrote:

> 2017-03-28 22:10 GMT+02:00 Venkateswara Rao Jujjuri <jujj...@gmail.com>:
>
> > Enrico what we do is, we do it at the application level.
> > The client does
> >
> >  // Sanity check before trying to read; Get the latest LAC.
> >         if (rquest_entryId > lh.getLastAddConfirmed()) {
> >             // Get the latest Lac.
> >             try {
> >                 lh.readExplicitLastConfirmed();
> >             } catch (BKException e) {
> >                 // We don't need to fail the read just because getting
> > explicit LAC failed.
> >                 LOG.warn("Read lac on extentId: {} failed: {} ",
> extentId,
> > e);
> >             }
> >         }
> > // Read now
> >         Enumeration<LedgerEntry> entries = lh.readEntries(rquest_entryId,
> > rquest_entryId);
> >
>
>
> I was using a very similar code in my prototype but IMHO this way we have a
> lot of overhead:
> - we need to send explicit LACs and save that info on the bookies
> - we have to read the LAC directly from bookies if the local LAC on readers
> is not up-to-date
>
> My idea is to try to read directly from the Bookie, bypassing the LAC
> protocol at all.
>
> My new prototype is working very well, but I fear I'm missing something for
> the recovery scenario.
>
> Example of good flow:
> - The writer writes entry 7, and waits for acknowledge from all the bookies
> - The writer passes the id 7 to the reader
> - The reader reads from a Bookie the entry 7
>
>
> Example of bad flow:
> - The writer writes entry 7, and waits for acknowledge from all the bookies
> - The writer crashes without neither closing nor sending other entries (so
> no LAC piggyback can be done)
> - Lets' assume that the LAC on bookies is currently 6
> - Some client performs a recovery of the ledger, and writes on the metadata
> that the last valid entry is 6
> - Entry 7 will no more be available
>
> I'm going to try to reproduce this scenario in unit tests,
> Maybe it is enough not to recover the ledger
>
>
>
>
>
>
> >
> >
> > On Tue, Mar 28, 2017 at 12:38 PM, Enrico Olivelli <eolive...@gmail.com>
> > wrote:
> >
> > > Il mar 28 mar 2017, 20:57 Sijie Guo <guosi...@gmail.com> ha scritto:
> > >
> > > > On Tue, Mar 28, 2017 at 11:21 AM, Enrico Olivelli <
> eolive...@gmail.com
> > >
> > > > wrote:
> > > >
> > > > > 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.
> > > > >
> > > >
> > > > Is reader running in the same process with the writer? what semantics
> > are
> > > > you going to achieve? read-my-write?
> > > >
> > >
> > > Separate processes.
> > > Yes read my write
> > >
> > > >
> > > >
> > > > > 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
> > > > >
> > > >
> > > --
> > >
> > >
> > > -- Enrico Olivelli
> > >
> >
> >
> >
> > --
> > Jvrao
> > ---
> > First they ignore you, then they laugh at you, then they fight you, then
> > you win. - Mahatma Gandhi
> >
>



-- 
Jvrao
---
First they ignore you, then they laugh at you, then they fight you, then
you win. - Mahatma Gandhi

Reply via email to