Thank you Sijie for your explanations
My "new" use case is more similar to JV usage of BookKeeper at SF, I'm
trying to store data on BookKeeper as it could provide low-latency for
both reads and writes.
I think I will need the improvements from JV to force the advance of the LAC
I will dig into code to understand clearly how LAC is written into
entries and how many entries I should write to ensure that readers can
see the advance of the LAC.
If you have time to give me some pointers to code I will appreciate
that, but anyway I think I will be able to find by myself
Another question: is there any real limit in the max size of an entry ?
I would like to store entries up to 100 MB and I would like not to split
data into chunks (actually I'm going to store BLOBs)
As far as I will have a working proof-of-concept I will share by code on
GitHub, I think this will be another awesome example of usage of BookKeeper
Enrico
Il 05/12/2016 21:18, Sijie Guo ha scritto:
On Fri, Dec 2, 2016 at 8:19 AM, Enrico Olivelli <[email protected]
<mailto:[email protected]>> 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