bookkeeper-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Sijie Guo <si...@apache.org>
Subject Re: Understanding LAC
Date Mon, 05 Dec 2016 20:18:15 GMT
On Fri, Dec 2, 2016 at 8:19 AM, Enrico Olivelli <eolivelli@gmail.com> wrote:

> Hi,
> I'm doing some benchs in order to use BookKeeper as low-latency data
> storage, but I'm missing some piece of the LAC protocol.
>
> 1) From client1 I create a ledger and then perform an addEntry and
> wait for the ACK. My LedgerHandle is still open.
> 2) Client1 obtains the entryId and passes it to client2.
> 3) Client2 opens the ledger with 'norecovery' and tries to read the entry
> 4) The read fails, and on Client2 the LAC (readLastConfirmed) is still -1
>
> I know that the close or the openWithRecovery operations will clean up
> the ledger metadata and the LAC will be the number I expect.
>
> I see that the LAC sometimes "advances" even if the ledger is not
> closed or recovered, but I cannot find any way to force this advance.
>

The lac will be packed into entries and written with them. so when you
write next entries, it would advance the LAC.

In DistributedLog, we write a *control record* to advance LAC based on the
flush policy (immediate, time-based, size-based).


>
> I'm using this bookie side options:
>  conf.setFlushInterval(1000);
>  conf.setJournalFlushWhenQueueEmpty(true);
>  conf.setGcWaitTime(10);
>
> My need is to have a low-latency "storage" and so I need that readers
> can access stored data as soon as the write receives the 'ack' of the
> write
>
> I think that the 'piggy back' of the LAC in BK 4.5 will help my case.
>

yes. that would help here.


>
> My questions:
> 1) How is the LAC propagated from writer -> bookies -> readers ?
>

LAC is packed and written along with entries.

You can call #readLastAddConfirmed to get the new last added confirm in
readers. Once you get the new last add confirmed, you can read new entries.

with the long poll changes, you will be able to readLastConfirmedAndEntry
within one rpc call.

But since your use case is more like a 'streaming' use case, you can use
the distributedlog-core library to achieve this. As it has handle the
complexity of reading lac and entries in a streaming way, it would simply
your use case.


> 2) Is there any way to force the flush of the LAC ?
>

Right now, you can write entries to force flush the LAC. There is a change
from JV that adds explicit write/read lac rpcs. With that change, the
client can configure a flush policy to explicitly flush LAC.


>
>
> Enrico
>

Mime
View raw message