Hello Whitney,

On Mon, Feb 4, 2013 at 10:44 AM, Whitney Sorenson <wsoren...@hubspot.com>wrote:

> Thank you for responding.
>
> Forgive me if I'm missing something, but if I have a writer and separate
> readers, why would I want to have to communicate ledger ids between them?
> More specifically, we have a series of writers writing to a write-ahead log
> and a separate set of readers that are consuming these ledgers to move them
> into long term storage and send them to queues / workflows to be processed.
>

I am just curious about the case you mentioned that you have a series of
writers writing to a write-ahead log. If the write-ahead log means a
ledger, I couldn't image how you implemented a series of writers writing to
it, since bookkeeper just allow one writer writing to a ledger.

if the write-head log is formed by several ledgers, it means that you might
already have a mechanism to map the writer to the ledger, so when a writer
#openLedger, it means that the ledger would be closed. basically, the close
state could be distinguished by different calls : the ledger handle
returned by #createLedger is an opened ledger while the ledger handle
returned by #openLedger is a closed ledger. If you want to write entries,
you had to create a new ledger. Either the ledger is closed or the writer
is crashed, the ledger could not be written again. So I don't think you
need to keep the state is your end. If I don't understand your case, please
let me know.


> This means I have to keep the state about which ledgers are available, and
> which are closed, which seems to be a complete duplication of the state
> that is already in BK.
>
> I'm not sure named ledgers are helpful in this situation, except that we
> could keep less state (perhaps a sequential id.)
>
> On Mon, Feb 4, 2013 at 1:27 PM, Sijie Guo <guosi...@gmail.com> wrote:
>
>>
>> Hello, Whitney:
>>
>> please check the replies inline.
>>
>> On Mon, Feb 4, 2013 at 8:47 AM, Whitney Sorenson 
>> <wsoren...@hubspot.com>wrote:
>>
>>> Hey all,
>>>
>>> A couple questions about running BK stand-alone:
>>>
>>> 1) If I call openLedgerNoRecovery am I blocking writes or not? What are
>>> the guarantees I lose - just ordering? Can I use this to essentially read /
>>> tail an active ledger?
>>>
>>
>> open a ledger using openLedgerNoRecovery doesn't block any writes to it.
>> And you don't lose the ordering guarantee. You could use it to read/tail an
>> active ledger, but please keep in mind that you need to call
>> #readLastConfirmed to catch up to the latest confirmed entries added by the
>> writer. And the entries you could read from an openLedgerNoRecovery ledger,
>> is just between 0 and last confirmed.
>>
>> you could check:
>> http://zookeeper.apache.org/bookkeeper/docs/r4.2.0/apidocs/org/apache/bookkeeper/client/BookKeeper.html#asyncOpenLedgerNoRecovery(long,
>> org.apache.bookkeeper.client.BookKeeper.DigestType, byte[],
>> org.apache.bookkeeper.client.AsyncCallback.OpenCallback, java.lang.Object)
>>
>>
>>>
>>> 2) How can I access BK's metadata so that I can determine a list of
>>> ledgers, and which ledgers are closed/open? It doesn't appear in the client
>>> documentation (
>>> http://zookeeper.apache.org/bookkeeper/docs/r4.2.0/apidocs/org/apache/bookkeeper/client/)
>>> Is this not an intended operation? Are clients supposed to track ledger ids
>>> on their own (we are currently doing this but it seems suboptimal)
>>>
>>>
>> currently we don't expose the API for client. Is there any special case
>> you are considering? We'd happy to expose it if necessary.
>>
>>  Since most of the cases are working in following styles: a *standby*
>> writer observes the *active* writer state, if the *active* writer failed,
>> the *standby* writer would take over the responsibility, closed the ledger
>> written by *active* writer, replayed the ledger and created a new ledger to
>> write new entries. For now, clients needs to track ledger ids on their end.
>>
>> There is one proposal working on providing *named* ledgers on top of
>> bookkeeper to ease user's experience tracking ledger ids. You could check :
>> https://issues.apache.org/jira/browse/BOOKKEEPER-220 . And we are under
>> discussion on whether to provide ledger name internally in bookkeeper for
>> metadata access concerns. We'd like to hear your feedback on the usage of
>> API and make it better.
>>
>>
>>
>>> Thank you;
>>>
>>> -Whitney Sorenson
>>> HubSpot
>>>
>>>
>>
>

Reply via email to