bookkeeper-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Enrico Olivelli <eolive...@gmail.com>
Subject Understanding LAC
Date Fri, 02 Dec 2016 16:19:21 GMT
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.

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.

My questions:
1) How is the LAC propagated from writer -> bookies -> readers ?
2) Is there any way to force the flush of the LAC ?


Enrico

Mime
View raw message