Re: Improve the process of removing bookies from a cluster

2021-09-07 Thread Venkateswara Rao Jujjuri
Glad to see this thread. This is one of the biggest limitations to
autoscaling.

On Tue, Sep 7, 2021 at 6:11 AM Jonathan Ellis  wrote:

> On Tue, Sep 7, 2021 at 8:05 AM Ivan Kelly  wrote:
>
> > Hi Yang,
> >
> > > Autoscaling is exactly one motivation for me to bring this topic up. I
> > > understand that the auto-recovery is not perfect at the moment, but
> it's
> > an
> > > important component that maintains the core invariants of a bookkeeper
> > > cluster, so I think we may keep improving it until we find a better
> > > replacement.
> >
> > Internally we have replaced auto recovery with another mechanism that
> > checks that the bookie
> > has all the data it says it has. We have plans to push this upstream
> > in the next month or two. A side
> > effect of the change is that it allows you to run without journal safely.
> > However, it doesn't cover the decommission usecase. For decommission,
> > our thinking is that once we
> > have tiered storage at the bookie level, the decommission story
> > becomes a lot easier. Basically, you
> > switch to read-only and wait for tiered storage to clear it out, even
> > bumping the bookies ledgers in priority
> > for offloading to tiered storage. We're still early in this process
> > (utilization metrics have to come first).
> >
>
> This sounds very useful, I'm excited to see more details.
>


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


Re: Running without journal

2021-05-04 Thread Venkateswara Rao Jujjuri
> how you justify removing the ledger as opposed to removing the ledger
storage and preserving the journal

I will be waiting for the BP on this point too. :) But glad to see that we
are working to avoid double writes. :)
With the EntryLogPerLedger (ELPL) feature, it is a more-or-less journal per
ledger.
With the entrylogs, we need to  maintain index files and journals, hence
two writes if we want to persist
data in-lieu of Journal.

Another way to think about this is, having a ledger durability mode. Does
it need fragment level durability, or durability at close.
Based on that we can completely avoid journal writes with ELPL + flush on
close.

Thanks,
JV

On Mon, May 3, 2021 at 8:34 AM Anup Ghatage  wrote:

> HI!
>
> I know we are interested in this for sure. (cc @Venkateswara Rao Jujjuri
> )
> Is this similar to Matteo Merli's PR which I found was simple and still
> got the job done: https://github.com/apache/bookkeeper/pull/2401/files
>
> Regards,
> Anup
>
> On Mon, May 3, 2021 at 8:04 AM Flavio Junqueira  wrote:
>
>> +1, it makes sense to enable bookies to run without duplicating IOs for
>> entry data. I'm curious to see how you justify removing the ledger as
>> opposed to removing the ledger storage and preserving the journal. I
>> suspect that the random reads against the ledger storage matter more to you
>> than the sequential writes, and you're possibly able to make it perform
>> well enough with SSD and even NVMe drives.
>>
>> I should wait for your write up rather than speculate. Looking forward to
>> seeing the BP.
>>
>> -Flavio
>>
>> > On 3 May 2021, at 16:52, Enrico Olivelli  wrote:
>> >
>> > Il giorno lun 3 mag 2021 alle ore 16:30 Jack Vanlightly
>> > mailto:jvanligh...@splunk.com.invalid>>
>> ha scritto:
>> >>
>> >> Hi all,
>> >>
>> >> At Splunk we have defined and implemented changes to BookKeeper to
>> allow
>> >> bookies to run without the journal. The motivation for this work is to
>> >> allow BookKeeper to be run with lower operating costs while still
>> offering
>> >> decent data safety guarantees.
>> >>
>> >> Before submitting the work as a PR we'd like to formalise the proposed
>> >> changes in a BP where we state our motivation, explain the protocol
>> >> changes, the work on formally verifying the proposal and be open to
>> comment.
>> >>
>> >> We'll create a BP this week if that sounds good to you all.
>> >
>> > Great to hear that !
>> >
>> > Thanks
>> > Enrico
>> >
>> >>
>> >> Thanks
>> >> Jack
>> >>
>> >> --
>> >> *Jack Vanlightly*
>> >> Principal Software Engineer
>> >> Splunk Inc.
>> >> jvanligh...@splunk.com <mailto:jvanligh...@splunk.com> <
>> kramas...@splunk.com <mailto:kramas...@splunk.com>>
>> >> Barcelona
>>
>>
>
> --
> Anup Ghatage
> www.ghatage.com
>


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


Re: NPE in pendingAddOp

2021-02-16 Thread Venkateswara Rao Jujjuri
But post recycle should not result in this scenario as I mentioned in my
comment.

On Tue, Feb 16, 2021 at 2:49 PM Andrey Yegorov 
wrote:

> Simple check for null won't prevent timing out wrong (post recycle/reuse)
> handle.
>
>
> On Tue, Feb 16, 2021 at 14:38 Venkateswara Rao Jujjuri 
> wrote:
>
> > Yes; That is exactly how my theory is and the fix is also the same. But
> > still looking into how we could get into it.
> >
> > // If the object is recycled while this task is on the queue lh could get
> > null.
> > // monitorPendingAddOps() can't schedule after recycling because
> > // time timed out pendingAddOp won't be available on the
> > // pendingAddOps concurrent queue after a successful recycle.
> > //if (completed) {
> > if (lh == null || completed) {
> >
> >
> >
> > On Tue, Feb 16, 2021 at 2:30 PM Andrey Yegorov  >
> > wrote:
> >
> > > This could happen if PendingAddOp got recycled (and set lh to null)
> > before
> > > the runnable submitted by timeoutQuorumWait had a chance to run.
> > > I don't know how it got into this state but if is easy to miss if
> > recycled
> > > object gets reused quickly - timeout will fail wrong ledger.
> > >
> > > The right thing to do is to figure bout how recycling could happen
> before
> > > timeout runnable had a chance to run.
> > >
> > > As workaround you can simply cache ledger handle before creating
> runnable
> > > and use the cached value in the runnable.
> > > Hope this makes sense.
> > >
> > > On Tue, Feb 16, 2021 at 13:12 Enrico Olivelli 
> > wrote:
> > >
> > > > Sorry
> > > > I never seen that error
> > > >
> > > > Enrico
> > > >
> > > > Il giorno mar 16 feb 2021 alle ore 19:50 Venkateswara Rao Jujjuri <
> > > > jujj...@gmail.com> ha scritto:
> > > >
> > > > > Ignore my comment about different thread, I see that
> > > timeoutQuorumWait()
> > > > is
> > > > > run through OSE.
> > > > > But we did NPE in this line
> > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173
> > > > > >.
> > > > > In this run, we have disabled ensemble changes.
> > > > >
> > > > > On Tue, Feb 16, 2021 at 10:28 AM Venkateswara Rao Jujjuri <
> > > > > jujj...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > We have observed NPE in pendingAddOp in this line
> > > > > > <
> > > > >
> > > >
> > >
> >
> https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173
> > > > > >.
> > > > > >
> > > > > > This looks like a race between execution of
> > recyclePendAddOpObject()
> > > in
> > > > > > OSE context and maybeTimeout() in monitorPendingAddOps()
> executing
> > in
> > > > > > bk.scheduler context.
> > > > > >
> > > > > > This looks like an age-old code and wondering why we haven't seen
> > > this
> > > > so
> > > > > > far.
> > > > > > Has Anyone in the community observed this?
> > > > > >
> > > > > >
> > > > > > --
> > > > > > 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
> > > > >
> > > >
> > > --
> > > --
> > > Andrey Yegorov
> > >
> >
> >
> > --
> > Jvrao
> > ---
> > First they ignore you, then they laugh at you, then they fight you, then
> > you win. - Mahatma Gandhi
> >
> --
> --
> Andrey Yegorov
>


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


Re: NPE in pendingAddOp

2021-02-16 Thread Venkateswara Rao Jujjuri
Yes; That is exactly how my theory is and the fix is also the same. But
still looking into how we could get into it.

// If the object is recycled while this task is on the queue lh could get null.
// monitorPendingAddOps() can't schedule after recycling because
// time timed out pendingAddOp won't be available on the
// pendingAddOps concurrent queue after a successful recycle.
//if (completed) {
if (lh == null || completed) {



On Tue, Feb 16, 2021 at 2:30 PM Andrey Yegorov 
wrote:

> This could happen if PendingAddOp got recycled (and set lh to null) before
> the runnable submitted by timeoutQuorumWait had a chance to run.
> I don't know how it got into this state but if is easy to miss if recycled
> object gets reused quickly - timeout will fail wrong ledger.
>
> The right thing to do is to figure bout how recycling could happen before
> timeout runnable had a chance to run.
>
> As workaround you can simply cache ledger handle before creating runnable
> and use the cached value in the runnable.
> Hope this makes sense.
>
> On Tue, Feb 16, 2021 at 13:12 Enrico Olivelli  wrote:
>
> > Sorry
> > I never seen that error
> >
> > Enrico
> >
> > Il giorno mar 16 feb 2021 alle ore 19:50 Venkateswara Rao Jujjuri <
> > jujj...@gmail.com> ha scritto:
> >
> > > Ignore my comment about different thread, I see that
> timeoutQuorumWait()
> > is
> > > run through OSE.
> > > But we did NPE in this line
> > > <
> > >
> >
> https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173
> > > >.
> > > In this run, we have disabled ensemble changes.
> > >
> > > On Tue, Feb 16, 2021 at 10:28 AM Venkateswara Rao Jujjuri <
> > > jujj...@gmail.com>
> > > wrote:
> > >
> > > > We have observed NPE in pendingAddOp in this line
> > > > <
> > >
> >
> https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173
> > > >.
> > > >
> > > > This looks like a race between execution of recyclePendAddOpObject()
> in
> > > > OSE context and maybeTimeout() in monitorPendingAddOps() executing in
> > > > bk.scheduler context.
> > > >
> > > > This looks like an age-old code and wondering why we haven't seen
> this
> > so
> > > > far.
> > > > Has Anyone in the community observed this?
> > > >
> > > >
> > > > --
> > > > 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
> > >
> >
> --
> --
> Andrey Yegorov
>


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


Re: NPE in pendingAddOp

2021-02-16 Thread Venkateswara Rao Jujjuri
Ignore my comment about different thread, I see that timeoutQuorumWait() is
run through OSE.
But we did NPE in this line
<https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173>.
In this run, we have disabled ensemble changes.

On Tue, Feb 16, 2021 at 10:28 AM Venkateswara Rao Jujjuri 
wrote:

> We have observed NPE in pendingAddOp in this line
> <https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java#L173>.
>
> This looks like a race between execution of recyclePendAddOpObject() in
> OSE context and maybeTimeout() in monitorPendingAddOps() executing in
> bk.scheduler context.
>
> This looks like an age-old code and wondering why we haven't seen this so
> far.
> Has Anyone in the community observed this?
>
>
> --
> 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


NPE in pendingAddOp

2021-02-16 Thread Venkateswara Rao Jujjuri
We have observed NPE in pendingAddOp in this line
.

This looks like a race between execution of recyclePendAddOpObject() in OSE
context and maybeTimeout() in monitorPendingAddOps() executing in
bk.scheduler context.

This looks like an age-old code and wondering why we haven't seen this so
far.
Has Anyone in the community observed this?


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


Re: Unbounded memory usage for WQ > AQ ?

2021-01-18 Thread Venkateswara Rao Jujjuri
On Mon, Jan 18, 2021 at 10:53 AM Sijie Guo  wrote:

> > One concern for me in this thread is case (3). I'd expect a client that
> doesn't crash to not give up, and eventually replace the bookie if it is
> unresponsive.
>
> The current implementation doesn't retry replacing a bookie if an entry is
> already acknowledged (receiving AQ responses). It relies on inspection to
> repair the hole.
>

Exactly. It is not even practical to do this as with the current code.
Once the Qa meets we move the LAC. So

Ensemble  B0  B1 B2 LAC
Entry:0   W  W   W  -1
1W  WNR0   (NR: No Response)
2W  WNR1
Now B1 failed with network error where write fails immediately
3  when attempted to write it gets error immediately and
attempts ensemble change.
I think this is wrong. Why we treat errors after Qa is
different from before reaching Qa.
   What is stopping the code from waiting to see if Qa is
met or not before attempting ensemble change.? @Sijie Guo
 ?
Ensemble B0  B10  B2LAC
3   WWNR   2

Since we changed ensemble if entry 1 and 2 fails with timeout we can't go
back and retroactively change the ensemble



> In case (1), the client crashed and the ledger will be recovered by some
reader. For all entries that have been acknowledged, including e, I'd
expect them to be readable from the closed ledger. Each one of these
entries that haven't been written to bookie b should be written there as
part of the recovery process.

I don't think this can ever happen because we have OSE hashed by ledgerId.
We can't receive and process any responses before we send out to all Qw
bookies.

Not sure what is the consensus reached on Issue#1063
.
If it appears to be a problem let's have a quick call, maybe that is easy
to resolve.

Thanks,
JV


> So the memory pressure is not coming from retrying. It is straight that the
> bookkeeper client references the sendBuffers until it receives any
> responses from the slow bookie. The bookkeeper client allows enqueuing
> addEntry operations because the operations meet the AQ requirements. Pulsar
> does add `maxPendingPublishdRequestsPerConnection` mechanism to throttle
> the add operations. But this won't work as bookkeeper will notify the
> callbacks once the operations meet the AQ requirements. But there is a huge
> amount of memory (throughput * timeout period) referenced by a slow bookie.
> Hence we have to add a memory-based throttling mechanism as Matteo
> suggested.
>
> If we want to add the retry logic to replace a bookie, this will add more
> pressure to the memory. But it can still be solved by a memory-based
> back-pressure mechansim.
>
> Thanks,
> Sijie
>
> On Mon, Jan 18, 2021 at 8:10 AM Flavio Junqueira  wrote:
>
> > In the scenario that WQ > AQ, a client acknowledges the add of an entry e
> > to the application once it receives AQ bookie acks. Say now that the
> client
> > is not able to write a copy of e to at least one bookie b, it could be
> > because:
> >
> > 1- The client crashed before it is able to do it
> > 2- Bookie b crashed
> > 3- The client gave up trying
> >
> > In case (1), the client crashed and the ledger will be recovered by some
> > reader. For all entries that have been acknowledged, including e, I'd
> > expect them to be readable from the closed ledger. Each one of these
> > entries that haven't been written to bookie b should be written there as
> > part of the recovery process.
> >
> > In case (2), the client is not able to write entry e to the crashed
> bookie
> > b, so it will replace the bookie and write e to the new bookie. I see in
> > this discussion that there is an option to disable bookie replacement,
> I'm
> > ignoring that for this discussion.
> >
> > In case (3), the client say discards the entry after adding successfully
> > to AQ bookies, and gives up at some point because it can't reach the
> > bookie. The client maybe replaces bookie b or bookie b eventually comes
> > back and the client proceeds with the adds. In either case, there is a
> hole
> > that can only be fixed by inspecting the ledger.
> >
> > One concern for me in this thread is case (3). I'd expect a client that
> > doesn't crash to not give up, and eventually replace the bookie if it is
> > unresponsive. But, that certainly leads to the memory pressure problem
> that
> > was also mentioned in the thread, for which one potential direction also
> > mentioned is to apply back pressure.
> >
> > Thanks,
> > -Flavio
> >
> > > On 18 Jan 2021, at 12:20, Jack Vanlightly  .INVALID>
> > wrote:
> > >
> > >> Did you guys see any issues with the ledger auditor?
> > >
> > >> The active writer can't guarantee it writing entries to WQ because it
> > can
> > >> crash during retrying adding 

Re: Unbounded memory usage for WQ > AQ ?

2021-01-11 Thread Venkateswara Rao Jujjuri
> new data integrity check that Ivan worked on
The current auditor should take care of this if
"auditorLedgerVerificationPercentage" is set to 100%.
I don't think this is the most efficient way, but I believe it does take
care of filling holes.

On Mon, Jan 11, 2021 at 12:31 AM Jack Vanlightly
 wrote:

> Hi,
>
> I've recently modelled the BookKeeper protocol in TLA+ and can confirm that
> once confirmed, that an entry is not replayed to another bookie. This
> leaves a "hole" as the entry is now replicated only to 2 bookies, however,
> the new data integrity check that Ivan worked on, when run periodically
> will be able to repair that hole.
>
> Jack
>
> On Sat, Jan 9, 2021 at 1:06 AM Venkateswara Rao Jujjuri  >
> wrote:
>
> > [ External sender. Exercise caution. ]
> >
> > On Fri, Jan 8, 2021 at 2:29 PM Matteo Merli 
> > wrote:
> >
> > > On Fri, Jan 8, 2021 at 2:15 PM Venkateswara Rao Jujjuri
> > >  wrote:
> > > >
> > > > > otherwise the write will timeout internally and it will get
> replayed
> > > to a
> > > > new bookie.
> > > > If Qa is met and the writes of Qw-Qa fail after we send the success
> to
> > > the
> > > > client, why would the write replayed on a new bookie?
> > >
> > > I think the original intention was to avoid having 1 bookie with a
> > > "hole" in the entries sequence. If you then lose one of the 2 bookies,
> > > it would be difficult to know which entries need to be recovered.
> > >
> >
> > @Matteo Merli   I don't believe we retry the
> write
> > on bookie if Qa is satisfied and the write to a bookie timedout.
> > Once the entry is ack'ed to the client we move the LAC and can't
> > retroactively change the active segment's ensemble.
> >
> > >  will get replayed to a new bookie
> > This will happen only if we are not able to satisfy Qa and go through
> > ensemble changes.
> > We change the ensemble and tetry write only if bookie write fails before
> > satisfying Qa.
> > We have added a new feature called handling "delayed write failure", but
> > that happens only for
> > new entries not retroactively.
> >
> > I may be missing something here, and not understanding your point.
> >
> > Thanks,
> > JV
> >
> >
> >
> >
> > --
> > 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


Re: Unbounded memory usage for WQ > AQ ?

2021-01-08 Thread Venkateswara Rao Jujjuri
On Fri, Jan 8, 2021 at 2:29 PM Matteo Merli  wrote:

> On Fri, Jan 8, 2021 at 2:15 PM Venkateswara Rao Jujjuri
>  wrote:
> >
> > > otherwise the write will timeout internally and it will get replayed
> to a
> > new bookie.
> > If Qa is met and the writes of Qw-Qa fail after we send the success to
> the
> > client, why would the write replayed on a new bookie?
>
> I think the original intention was to avoid having 1 bookie with a
> "hole" in the entries sequence. If you then lose one of the 2 bookies,
> it would be difficult to know which entries need to be recovered.
>

@Matteo Merli   I don't believe we retry the write
on bookie if Qa is satisfied and the write to a bookie timedout.
Once the entry is ack'ed to the client we move the LAC and can't
retroactively change the active segment's ensemble.

>  will get replayed to a new bookie
This will happen only if we are not able to satisfy Qa and go through
ensemble changes.
We change the ensemble and tetry write only if bookie write fails before
satisfying Qa.
We have added a new feature called handling "delayed write failure", but
that happens only for
new entries not retroactively.

I may be missing something here, and not understanding your point.

Thanks,
JV




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


Re: Unbounded memory usage for WQ > AQ ?

2021-01-08 Thread Venkateswara Rao Jujjuri
> otherwise the write will timeout internally and it will get replayed to a
new bookie.
If Qa is met and the writes of Qw-Qa fail after we send the success to the
client, why would the write replayed on a new bookie?

On Fri, Jan 8, 2021 at 1:47 PM Matteo Merli  wrote:

> On Fri, Jan 8, 2021 at 8:27 AM Enrico Olivelli 
> wrote:
> >
> > Hi Matteo,
> > in this comment you are talking about an issue you saw when WQ is
> greater that AQ
> > https://github.com/apache/bookkeeper/issues/2497#issuecomment-734423246
> >
> > IIUC you are saying that if one bookie is slow the client continues to
> accumulate references to the entries that still have not received the
> confirmation from it.
> > I think that this is correct.
> >
> > Have you seen problems in production related to this scenario ?
> > Can you tell more about them ?
>
> Yes, for simplicity, assume e=3, w=3, a=2.
>
> If one bookie is slow (not down, just slow), the BK client will the
> acks to the user that the entries are written after the first 2 acks.
> In the meantime, it will keep waiting for the 3rd bookie to respond.
> If the bookie responds within the timeout, the entries can now be
> dropped from memory, otherwise the write will timeout internally and
> it will get replayed to a new bookie.
>
> In both cases, the amount of memory used in the client will max at
> "throughput" * "timeout". This can be a large amount of memory and
> easily cause OOM errors.
>
> Part of the problem is that it cannot be solved from outside the BK
> client, since there's no visibility on what entries have 2 or 3 acks
> and therefore it's not possible to apply backpressure. Instead,
> there should be a backpressure mechanism in the BK client itself to
> prevent this kind of issue.
> One possibility there could be to use the same approach as described
> in
> https://github.com/apache/pulsar/wiki/PIP-74%3A-Pulsar-client-memory-limits
> ,
> giving a max memory limit per BK client instance and throttling
> everything after the quota is reached.
>
>
> Matteo
>


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


Re: [ANNOUNCE] New Committer: Anup Ghatage

2020-10-21 Thread Venkateswara Rao Jujjuri
Congratulations Anup!!

On Wed, Oct 21, 2020 at 8:43 AM Anup Ghatage  wrote:

> Thank you!
> I am excited to be a part of this passionate community and hope I can
> contribute to making it even better.
>
> Regards,
> Anup
>
> On Wed, Oct 21, 2020 at 2:05 AM Enrico Olivelli 
> wrote:
>
> > The Project Management Committee (PMC) for Apache BookKeeperhas invited
> > Anup Ghatage to become a committer and we are pleased to announce that he
> > has accepted.
> >
> > Anup Ghatage already contributed important features to our project.
> > He is helping the community with code reviews and by participating in the
> > discussions on the mailing list and on Slack.
> > He is also coordinating the work on our brand new website.
> >
> > Being a committer enables easier contribution to the project since there
> is
> > no need to go via the patch submission process.
> > This should enable better productivity.
> >
> > Welcome aboard Anup !
> > Congratulations
> >
> > Enrico
> >
>
>
> --
> Anup Ghatage
> www.ghatage.com
>


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


Re: Re-working the BookKeeper site

2020-08-10 Thread Venkateswara Rao Jujjuri
+1 I really like the layout. Great job Anup.

On Mon, Aug 10, 2020 at 5:52 AM Enrico Olivelli  wrote:

> Anup,
> do you have any news ?
> I am working on a BP and actually building the website is really hard.
>
> It would be super to be able to setup your idea within a couple of months
>
> thanks again for driving this
> Enrico
>
> Il giorno gio 18 giu 2020 alle ore 07:13 Sijie Guo 
> ha scritto:
>
>> +1
>>
>> @Anup Ghatage  thank you for driving this!
>>
>> - Sijie
>>
>> On Mon, Jun 15, 2020 at 9:07 AM Anup Ghatage  wrote:
>>
>>> Thanks for your comments Enrico.
>>>
>>> This is just the prototype, we shall ask for consent from all the
>>> companies before using their logos.
>>> I checked and the website is not bad on Mobile.
>>> I see the release procedure being similar to that of the pulsar website.
>>> From what I see, all we have to do is create markdown files and push them
>>> to a specific folder and then build, similar to now.
>>> They have a script which does the 'build' and deploy. I'll reach out to
>>> the folks involved to see if there is more to it.
>>>
>>> I also reached out to Matteo Minardi (In slack) and Lamberken (on
>>> github) and they're both enthusiastic about getting this rolling.
>>> We'll come up with a plan and reply on this thread when it's ready.
>>>
>>> Regards,
>>> Anup
>>>
>>>
>>> ..
>>>
>>> On Sun, Jun 14, 2020 at 12:00 AM Enrico Olivelli 
>>> wrote:
>>>
 Anup
 Great idea.
 I am totally +1 to your idea

 Just a couple of items:
 - we must ask for consent before using logos
 - we should make the website look well on mobile
 - think about the release procedure and how we will have to update the
 website after the release (now we have a script but it is awkward)

 Thank you very much
 Enrico

 Il Dom 14 Giu 2020, 02:26 Anup Ghatage  ha scritto:

> Hi Bookies,
>
> We've had quite a few issues reported recently about the site.
> Some are about the fact that the site has dead links and some are
> about the content.
> I tried to build the site recently and it didn't even build.
>
> @Sijie Guo  suggested here
> that
> we should try and use docusaurus for the BookKeeper website as it's much
> easier to maintain. And I must agree! I played around with it a little and
> after using the Pulsar website as a template, I think this is fantastic.
>
> So I propose, we go ahead and start development of the new website
> under 'site2' in the root directory. Once the development is complete, we
> can decommission the old website and have the domain point to our new
> website.
>
> I spent most of today trying to re-imagine the website and have
> charted out the tasks and workflow, so if you're interested, please reply
> to this email and we can get started!
>
> I have put together a test website at:
> http://ghatage.com/bookkeeper-site/
> It's inspired by the Pulsar website and has A LOT of dead links and is
> yet to be populated.
> However, generation and maintenance of the website has been super easy.
> Please check it out and let me know what you guys think!
>
> Regards,
> Anup
>

>>>
>>> --
>>> Anup Ghatage
>>> www.ghatage.com
>>>
>>

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


Re: Slack channel for HerdDB hosted on BK slack

2020-07-08 Thread Venkateswara Rao Jujjuri
+1

On Wed, Jul 8, 2020 at 8:53 PM Sijie Guo  wrote:

> +1
>
> On Wed, Jul 8, 2020 at 1:34 AM Enrico Olivelli 
> wrote:
>
> > Hi,
> > Can I create an #herddb channel on our BookKeeper slack space ?
> >
> > After my talk at PulsarSummit we started having a few new contacts on
> > herddb-dev mailing list, but sometimes slack is more efficient.
> >
> > HerdDB community is still too little to have a dedicated space, it would
> be
> > awkward for users to subscribe to another workspace (I don't know your
> > experience but I am subscribed to lots of Slack workspaces and I don't
> like
> > it very much).
> >
> > There won't be much traffic.
> >
> > Regards
> > Enrico
> >
>


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


Question on disableEnsembleChange

2020-05-31 Thread Venkateswara Rao Jujjuri
I am looking at a feature to disableEnsembleChanges
If the disableEnsembleChangeFeature

is
available  it is calling unsetSuccessAndSendWriteRequest

without any ensemble changes, but the why are we calling
unsetSuccessAndSendWriteRequest, which is actually removing

the Ack
and resending the write.
If there is an ensemble changes this makes sense but in this case ensemble
hasn't changed. Wondering if this is a bug/oversight.


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


Re: BP-38 Publish Bookie Service Info on Metadata Service

2020-03-11 Thread Venkateswara Rao Jujjuri
>The client can derive bookieId from a BookieSocketAddress.

Any reason to tie the bookieId to the hardware? I may be missing something,
but the whole BP is to give flexibility
of Bookie changing its end points. If we make it a UUID, then it is
independent and more flexible right?

On Wed, Mar 11, 2020 at 4:24 AM Matteo Minardi - Diennea <
minardi.mat...@hotmail.it> wrote:

> Hi everyone,
> I'm picking up the implementation of BP-38 right now, my starting point is
> this draft PR https://github.com/apache/bookkeeper/pull/2213 .
>
> Il giorno 19/01/20, 19:17 "Enrico Olivelli"  ha
> scritto:
>
> Hi Bookkeepers,
> any other comment on this BP ?
>
> Only Sijie left his option by now
>
> I am continuing to work at the implementation, but we can't merge it
> if is
> there is not enough consensus in the community
> this is a major change, even if it is 100% backward compatible, but it
> is
> tracing the way for further improvements to the project
> that currently are not possible, like decoupling the concept of
> "bookieId"
> from its network location.
>
> Please take a look and comment, approve or reject the proposal
> https://github.com/apache/bookkeeper/pull/2214
>
> Enrico
>
> Il giorno sab 4 gen 2020 alle ore 06:04 Sijie Guo 
> ha
> scritto:
>
> > Hi Enrico,
> >
> > Thank you for putting all these together. We are interested in this
> > feature.
> >
> > I have made some comments in the proposal. PTAL.
> >
> > Thanks,
> > Sijie
> >
> > On Tue, Dec 17, 2019 at 5:25 AM Enrico Olivelli  >
> > wrote:
> >
> > > Hi,
> > > any interest in this proposal ?
> > >
> > > It will open up the way to having more information about the
> services
> > > exposed by bookies available to the clients and even to the
> > PlacementPolicy
> > > (in the future)
> > >
> > > I would like to see this kind of support in 4.11
> > >
> > > Enrico
> > >
> > > Il giorno ven 13 dic 2019 alle ore 00:30 Enrico Olivelli <
> > > eolive...@gmail.com> ha scritto:
> > >
> > > > Hi all,
> > > > I have created a new BookKeeper Proposal "BP-38 Publish Bookie
> Service
> > > > Info on Metadata Service"
> > > >
> > > > Proposal:
> > > > https://github.com/apache/bookkeeper/pull/2214
> > > >
> > > > Implementation (draft):
> > > > https://github.com/apache/bookkeeper/pull/2213
> > > >
> > > > Master ticket:
> > > > https://github.com/apache/bookkeeper/issues/2215
> > > >
> > > > Any feedback is really appreciated, for small comments you can
> comment
> > > > directly on  PR #2214 (proposal page) other wise we can just
> follow
> > this
> > > > email thread
> > > >
> > > > Best regards
> > > > Enrico
> > > >
> > > >
> > >
> >
>
>
>
>

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


Re: Bookkeeper stickers?

2019-11-14 Thread Venkateswara Rao Jujjuri
+1

On Thu, Nov 14, 2019 at 8:00 AM Anup Ghatage  wrote:

> That would be awesome!
> What can we do for those of us who can't attend the events but would still
> like the stickers?
>
> Regards,
> Anup
>
> On Thu, Nov 14, 2019, 5:06 AM Enrico Olivelli  wrote:
>
> > Hi,
> >
> > We do not have Apache Bookkeeper stickers!
> >
> > I was at ApacheCon Europe and there were stickers for Zookeeper, Pulsar,
> > Maven...
> > Nothing for Bookkeeper
> >
> > https://photos.apachecon.com/?/category/1
> >
> > Shall we order new stickers for new Apache events?
> >
> >
> > Enrico
> >
>


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


Re: Journal Corruption

2019-10-12 Thread Venkateswara Rao Jujjuri
Correct. We can't just ignore it. We needs to be a proper EndOfEntryLog.
How did it work until now? I wrote a simple test to reproduce this. This is
super surprising to me, who we haven't hit it so far?

Filling the file with negative numbers reproduces this problem very easily
by running BookieJournalTest.java: testV5Journal()
Just add the following 3 lines and run testV5Journal()

diff --git
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
index ab1ac7d0b..6fc68a0b9 100644
---
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
+++
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/JournalChannel.java
@@ -162,6 +162,9 @@ class JournalChannel implements Closeable {
 throw new IOException("File " + fn
 + " suddenly appeared, is another bookie process
running?");
 }
+byte[] filldata = new byte[4 * 1024 * 1024];
+Arrays.fill(filldata, (byte)-1);
+Files.write(Paths.get(fn.getPath()), filldata);

 randomAccessFile = new RandomAccessFile(fn, "rw");
 fc = randomAccessFile.getChannel();

On Sat, Oct 12, 2019 at 12:21 PM Enrico Olivelli 
wrote:

> JV,
> As the machine/disk/process may crash/halt unexpectedly during writes we
> can't do many assumptions, the only way is to accept an entry in case of a
> valid read and consider that we reached the end of the journal in case of a
> partial read or garbage.
>
> In your case it is possible that the reader reached an unwritten part of
> the preallocated file (garbage) or it is possible that the file has been
> corrupted.
>
> We can't reliably detect which of the two cases happened, this is bad.
> Even turning off preallocation won't help.
>
> We can fix the issue just by considering that we reached the end of the
> journal but if the file was corrupted we may lose important bits, as
> 'fence' bits
>
> Enrico
>
> Il sab 12 ott 2019, 19:18 Venkateswara Rao Jujjuri  ha
> scritto:
>
> > We have observed Journal Corruption while replaying the journal log.
> > I could be missing something but I don't see we have clear indication of
> > where
> > journal file ends.
> >
> > In the Journal.java:scanJournal() we have multiple breaks but mostly
> > we assume either the journal file ends in the right place or the length
> is
> > zero.
> >
> > scanJournal() {
> > ...
> > while(true) {
> >
> > // entry start offset
> > long offset = recLog.fc.position();
> > // start reading entry
> > lenBuff.clear();
> > fullRead(recLog, lenBuff);
> > if (lenBuff.remaining() != 0) {
> > break;
> > }
> > lenBuff.flip();
> > int len = lenBuff.getInt();
> > if (len == 0) {
> > break;
> > }
> > }
> > 
> >
> > So there is an implicit assumption here that the journal file ends
> >
> > or the length is zero.
> >
> > But... JournalChannel.java:
> > void preAllocIfNeeded(long size) throws IOException {
> > if (bc.position() + size > nextPrealloc) {
> > nextPrealloc += preAllocSize;
> > zeros.clear();
> > fc.write(zeros, nextPrealloc - journalAlignSize);
> > }
> > }
> >
> > We extend the file withe prealloc size at the journalAlignSize.
> >
> > Based on our tunables (in our case preAllocSize:4MB journalAlignSize:512)
> >
> > We could extend the file and the fc.write and we are writing only the
> > last journalAlignSize of zeros. i.e 4MB-512 can be garbage in the file.
> > When the journal is replayed there is a potential for garbage and journal
> > corruption indication.
> >
> > Being said that we haven't see this so far, so the bug may not be
> >
> > this obvious.. anyway just trying to see what I am missing.
> >
> > Here is the stack:
> >
> > 20191010181408.832000 [INFO ] [main] [JournalChannel.java:154] -
> > Opening journal /sfs/sfsdata/journal/current/16cdf99de69.txn
> > Exception in thread "main" java.lang.IllegalArgumentException
> > at java.nio.Buffer.limit(Buffer.java:275)
> > at org.apache.bookkeeper.bookie.Journal.scanJournal(Journal.java:827)
> > at org.apache.bookkeeper.bookie.Bookie.replay(Bookie.java:1047)
> > at org.apache.bookkeeper.bookie.Bookie.readJournal(Bookie.java:1009)
> > at org.apache.bookkeeper.bookie.Bookie.start(Bookie.java:1070)
> > at
> > org.apache.bookkeeper.proto.BookieServer.start(BookieServer.java:141)
> >  

Journal Corruption

2019-10-12 Thread Venkateswara Rao Jujjuri
We have observed Journal Corruption while replaying the journal log.
I could be missing something but I don't see we have clear indication of
where
journal file ends.

In the Journal.java:scanJournal() we have multiple breaks but mostly
we assume either the journal file ends in the right place or the length is
zero.

scanJournal() {
...
while(true) {

// entry start offset
long offset = recLog.fc.position();
// start reading entry
lenBuff.clear();
fullRead(recLog, lenBuff);
if (lenBuff.remaining() != 0) {
break;
}
lenBuff.flip();
int len = lenBuff.getInt();
if (len == 0) {
break;
}
}


So there is an implicit assumption here that the journal file ends

or the length is zero.

But... JournalChannel.java:
void preAllocIfNeeded(long size) throws IOException {
if (bc.position() + size > nextPrealloc) {
nextPrealloc += preAllocSize;
zeros.clear();
fc.write(zeros, nextPrealloc - journalAlignSize);
}
}

We extend the file withe prealloc size at the journalAlignSize.

Based on our tunables (in our case preAllocSize:4MB journalAlignSize:512)

We could extend the file and the fc.write and we are writing only the
last journalAlignSize of zeros. i.e 4MB-512 can be garbage in the file.
When the journal is replayed there is a potential for garbage and journal
corruption indication.

Being said that we haven't see this so far, so the bug may not be

this obvious.. anyway just trying to see what I am missing.

Here is the stack:

20191010181408.832000 [INFO ] [main] [JournalChannel.java:154] -
Opening journal /sfs/sfsdata/journal/current/16cdf99de69.txn
Exception in thread "main" java.lang.IllegalArgumentException
at java.nio.Buffer.limit(Buffer.java:275)
at org.apache.bookkeeper.bookie.Journal.scanJournal(Journal.java:827)
at org.apache.bookkeeper.bookie.Bookie.replay(Bookie.java:1047)
at org.apache.bookkeeper.bookie.Bookie.readJournal(Bookie.java:1009)
at org.apache.bookkeeper.bookie.Bookie.start(Bookie.java:1070)
at org.apache.bookkeeper.proto.BookieServer.start(BookieServer.java:141)
at 
org.apache.bookkeeper.server.service.BookieService.doStart(BookieService.java:58)
at 
org.apache.bookkeeper.common.component.AbstractLifecycleComponent.start(AbstractLifecycleComponent.java:78)
at 
org.apache.bookkeeper.common.component.LifecycleComponentStack.lambda$start$2(LifecycleComponentStack.java:113)
at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:408)
at 
org.apache.bookkeeper.common.component.LifecycleComponentStack.start(LifecycleComponentStack.java:113)
at 
org.apache.bookkeeper.common.component.ComponentStarter.startComponent(ComponentStarter.java:80)
at org.apache.bookkeeper.server.Main.doMain(Main.java:228)
at org.apache.bookkeeper.server.Main.main(Main.java:202)
at org.apache.bookkeeper.proto.BookieServer.main(BookieServer.java:296)



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


Re: Apache Bookkeeper Checksum Validation

2019-10-04 Thread Venkateswara Rao Jujjuri
On Fri, Oct 4, 2019 at 12:16 AM Enrico Olivelli  wrote:

> Karan,
>
> Il giorno gio 3 ott 2019 alle ore 23:41 Karan Mehta
>  ha scritto:
>
> > Thanks for the feedback. I will create a BP soon.
> >
> > > I won't shutdown the bookie, simply fail the write. It may happen in
> case
> > of a partial upgrade of the cluster and a write with a new digest type
> > comes to the bookie
> >
> > Interesting point. As per my assumptions, `All the options assume that
> the
> > server version will be greater than client version.`, this should not
> > happen.
> > I guessed most organisations operate and release in that fashion. I can
> > confirm for Salesforce. If you believe that is not the case, we should
> > discuss.
> >
>
>
> Usually the answer is "yes", the server should be upgraded before upgrading
> the client.
> But currently latest clients are compatible with older servers as far as
> they do not use new features.
> This is a cool "feature", in BookKeeper ecosystem we have very different
> applications.
> In my case it is possible that an application is using an older cluster.
>
> Apart fro that consideration, the real problem is that having a client that
> just asks for an unsupported digest type makes the bookie auto-shutdown is
> kind of a DoS security flawn
>
> It is better to fail the write
>

I agree with Enrico. I am not sure why we should shut down bookie. Just
fail the write and let
Ops decide the corrective action.

JV


>
>
> >
> > > Thinking about the future and about ideas shared with JV some month
> ago,
> > I lean towards having ledger metadata in the bookie. Having metadata
> opens
> > the way to new features, like per ledger storage type
> >
> > Yes it does bring those benefits, however I have two counter-args to it.
> > 1. It adds a RPC call and all the potential complexities of dealing with
> zk
> > in the critical path for at least some writes (later on we can cache
> > obviously).
> > 2. Most of ledger storage or QoS related stuff (some of our internal use
> > cases require that), can also be driven via writeFlags. Hence we decided
> to
> > opt on it.
> >
> > Internally we are going by the writeFlags option for now. We will keep
> the
> > community posted if we make any progress and also would require your help
> > to counter any challenges that we face along the way. Thank you!
> >
>
>
> I am fine with the WriteFlags option, it is consistent with current API and
> WriteFlags appeared just during those discussions about "ledger
> types"/"Qos".
>
> When sending out code please ensure to split the patch into smaller tasks,
> at least two:
> - server side changes
> - client side changes
>
> You could also add an integration test about what happens when a new client
> uses the new WriteFlag against an old bookie, it should receive an error
>
> Enrico
>
>
> >
> >
> > On Thu, Oct 3, 2019 at 11:12 AM Enrico Olivelli 
> > wrote:
> >
> > > Thank you for sharing this work.
> > > Two initial comments:
> > >
> > > Error handling:
> > > Unable to instantiate digest manager for that type
> > > Decline the write, shutdown itself and wait for external orchestrator
> to
> > > restart
> > >
> > >
> > > I won't shutdown the bookie, simply fail the write. It may happen in
> case
> > > of a partial upgrade of the cluster and a write with a new digest type
> > > comes to the bookie
> > >
> > >
> > > Which option is better?
> > > Thinking about the future and about ideas shared with JV some month
> ago,
> > I
> > > lean towards having ledger metadata in the bookie.
> > > Having metadata opens the way to new features, like per ledger storage
> > type
> > >
> > >
> > > Enrico
> > >
> > > Il gio 3 ott 2019, 18:44 Sijie Guo  ha scritto:
> > >
> > > > Hi Karan,
> > > >
> > > > Thank you for your proposal. Can you also add your proposal as a BP
> to
> > > the
> > > > BP list? You can check the BP process here:
> > > > http://bookkeeper.apache.org/community/bookkeeper_proposals/
> > > >
> > > > Thanks,
> > > > Sijie
> > > >
> > > > On Fri, Sep 27, 2019 at 5:53 AM Karan Mehta  > > > .invalid>
> > > > wrote:
> > > >
> > > > > Hello everyone,
> > > > >
> > > > > I wrote up a document here <
> https://salesforce.quip.com/FmlEAnMbtjnU
> > >
> > > > for
> > > > > Apache Bookkeeper Checksum Validation for the issue
> > > > > . I have added
> > > certain
> > > > > options and highlighted the pros/cons of each design. I would like
> to
> > > > hear
> > > > > everyone's thoughts on it. Feel free to comment on the doc to
> suggest
> > > > > ideas. Thanks for your inputs!
> > > > >
> > > > > --
> > > > > Karan Mehta
> > > > >
> > > > > 
> > > > >
> > > >
> > >
> >
> >
> > --
> > Karan Mehta
> >
> > 
> >
>


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


Re: Bug in OnClusterChanged() ?

2019-07-13 Thread Venkateswara Rao Jujjuri
Question: Any idea why we have a special check if we receive
emptyreadOnlyBookies we need to ignore?
It is possible that we can have no readonly bookies. Why not
unconditionally take what came from onClusterChanged
just like writableBookies?

if (!readOnlyBookies.isEmpty()) {
this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies);
}


On Fri, Jul 12, 2019 at 9:25 AM Venkateswara Rao Jujjuri 
wrote:

> Not yet; may be the scope/window is extremely tiny. For this to be a
> problem, only one node has to become RO->Offline.
> If more than one node becomes RO then we don't have this issue. Not sure
> if anyone else looked at it yet.
>
> On Fri, Jul 12, 2019 at 9:03 AM Enrico Olivelli 
> wrote:
>
>> Does anyone have a chance to take a look?
>>
>> Enrico
>>
>> Il mer 12 giu 2019, 19:26 Venkateswara Rao Jujjuri  ha
>> scritto:
>>
>> > I am looking at  onClusterChanged() in
>> > TopologyAwareEnsemblePlacementPolicy.java
>> > and I believe we don't handle the following case.
>> >
>> > 1. Bookie Became RO. We remove this from known bookies and add it to
>> > readOnlyBookies.
>> > 2. Same bookie went down; Now the arguments, writableBookies has no
>> change,
>> > and readOnlyBookies is NULL.
>> > At this point leftBookies, joinedBookies and deadBookies all get
>> evaluated
>> > to NULL.
>> > Also the following check doesn't even update readOnlyBookies
>> >
>> > if (!readOnlyBookies.isEmpty()) {
>> > this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies);
>> > }
>> >
>> > So we will continue to have down bookie as part of our readOnlyBookie.
>> >
>> > Am I missing something?
>> >
>> > --
>> > 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
>
>
>

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


Re: Bug in OnClusterChanged() ?

2019-07-12 Thread Venkateswara Rao Jujjuri
Not yet; may be the scope/window is extremely tiny. For this to be a
problem, only one node has to become RO->Offline.
If more than one node becomes RO then we don't have this issue. Not sure if
anyone else looked at it yet.

On Fri, Jul 12, 2019 at 9:03 AM Enrico Olivelli  wrote:

> Does anyone have a chance to take a look?
>
> Enrico
>
> Il mer 12 giu 2019, 19:26 Venkateswara Rao Jujjuri  ha
> scritto:
>
> > I am looking at  onClusterChanged() in
> > TopologyAwareEnsemblePlacementPolicy.java
> > and I believe we don't handle the following case.
> >
> > 1. Bookie Became RO. We remove this from known bookies and add it to
> > readOnlyBookies.
> > 2. Same bookie went down; Now the arguments, writableBookies has no
> change,
> > and readOnlyBookies is NULL.
> > At this point leftBookies, joinedBookies and deadBookies all get
> evaluated
> > to NULL.
> > Also the following check doesn't even update readOnlyBookies
> >
> > if (!readOnlyBookies.isEmpty()) {
> > this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies);
> > }
> >
> > So we will continue to have down bookie as part of our readOnlyBookie.
> >
> > Am I missing something?
> >
> > --
> > 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


Bug in OnClusterChanged() ?

2019-06-12 Thread Venkateswara Rao Jujjuri
I am looking at  onClusterChanged() in
TopologyAwareEnsemblePlacementPolicy.java
and I believe we don't handle the following case.

1. Bookie Became RO. We remove this from known bookies and add it to
readOnlyBookies.
2. Same bookie went down; Now the arguments, writableBookies has no change,
and readOnlyBookies is NULL.
At this point leftBookies, joinedBookies and deadBookies all get evaluated
to NULL.
Also the following check doesn't even update readOnlyBookies

if (!readOnlyBookies.isEmpty()) {
this.readOnlyBookies = ImmutableSet.copyOf(readOnlyBookies);
}

So we will continue to have down bookie as part of our readOnlyBookie.

Am I missing something?

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


Re: Changing ledger metadata to binary format

2019-05-29 Thread Venkateswara Rao Jujjuri
> "Let's decide this when we need it"

Right. This is the time as we are trying to add fault domain info to the
cookie.

> to make a flag in the metadata store that we flip to enable it.
Right I do recall the same. But given your proposal in this thread, all
readers try to read in binary format,
if that fails they fail back to text mode.
If this is the case, does the following make sense?
1. Bookies are updated first with new version. Replication workers
recognize binary/text format and any metadata updates happen in the same
format.
2. Clients are rolled out later with a caveat that the ledgers created by
new clients can't be read by old clients. This may be ok as long as they
don't share
the same ledgers.
With these two things, do we need an explicit flip to enable?

> The important thing at the time was to make it so clients can _read_
binary metadata

I can read the patches again, but let me ask :) did we take care of bookie
shell also to understand binary format?

On Wed, May 29, 2019 at 3:15 AM Ivan Kelly  wrote:

> > What is our plan to move forward with binary format?
>
> I've not plans regarding it. Moving forward with it will happen when
> someone comes with a metadata change which will break text metadata
> users (i.e. almost any metadata change).
>
> > Anyone using binary format in production? even for new clusters?
>
> We are not.
>
> > What is blocking us from using binary format for the new clusters?
>
> We need a tool to enable it. I can't remember the exact details, but
> there was a conversation about whether to make each users specify the
> metadata version in their configuration, or to make a flag in the
> metadata store that we flip to enable it. I don't think we settled on
> anything, and it turned into a "Let's decide this when we need it"
> type of thing. The important thing at the time was to make it so
> clients can _read_ binary metadata, so that these clients can then
> read future updates.
>
> -Ivan
>


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


Re: Changing ledger metadata to binary format

2019-05-28 Thread Venkateswara Rao Jujjuri
Hi all, bringing back this conversation back to life..

https://github.com/apache/bookkeeper/issues/723

What is our plan to move forward with binary format? Anyone using binary
format in production? even for new clusters?
What is blocking us from using binary format for the new clusters?

On Mon, Jul 30, 2018 at 11:02 AM Sijie Guo  wrote:

> JV,
>
> 1) for the real world use case, this is the problem I have hit which I have
> to manually add a flag in the ledger metadata to avoid serializing ctime.
>
> https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java#L105
>
> 2)  > We have introduced customMetadata in the past and that was not
> mandatory.
> How did we make that work with text format?
>
> by default if you don't use customMetadata, customMetadata is not
> serialized into metadata. so there is no problem for rolling back if you
> don't
> use this feature. so that's why it wasn't a problem for most of the people.
>
> the ctime field was the problem.
>
> 3)
> > At this point, I am not sure if we have identified all the pieces and if
> they fit together.
> > Enrico, are you driving the roll-back story? or whoever, can you please
> come up with a comprehensive proposal?
>
> we have a good-enough story for upgrading/rolling-back on wire-protocol and
> disk formats. however we don't have a good story on metadata part.
> that's why I sort of pulling in the metadata piece only. however if filling
> in a big picture would help discussing this problem, I can drive the whole
> picture
> for this roll-back story.
>
> - Sijie
>
> On Mon, Jul 30, 2018 at 7:55 AM Venkateswara Rao Jujjuri <
> jujj...@gmail.com>
> wrote:
>
> > We have introduced customMetadata in the past and that was not mandatory.
> > How did we make that work with text format?
> >
>
>
>
> > Also, my preference is to wait until we have a need to introduce a new
> > field for this conversion if needed.
> >
> > JV
> >
> > On Mon, Jul 30, 2018 at 12:42 AM, Enrico Olivelli 
> > wrote:
> >
> > > Il giorno lun 30 lug 2018 alle ore 09:17 Ivan Kelly 
> > ha
> > > scritto:
> > >
> > > > >> Thank you for putting this together. It is also good to put this
> as
> > a
> > > > BP,
> > > > >> since it is about the metadata layout.
> > > >
> > > > I'll put a BP up this week once I have initial feedback.
> > > >
> > > > >> > - When writing a metadata, check what is in /ledgers/LAYOUT. If
> it
> > > is
> > > > >> > as above, write using the current text protobuf. If is bumped,
> use
> > > the
> > > > >> > new binary format.
> > > > >
> > > > > isn't it too costly? Adding a zk read for each write. We could add
> a
> > > > watch
> > > > > but is has an important cost
> > > >
> > > > This cost can be amortized. In simplest case, we only read when
> client
> > > > starts (as already happens). Client are able both read and write the
> > > > old format, so clients that read the pre-upgrade version will still
> > > > write in text format, and this is fine.
> > > >
> > >
> > > For me it is fine to read it when client starts. A rolling restart of
> the
> > > client application will be enough to load the new configuration.
> > >
> > >
> > > >
> > > > However, we would want them to eventually see the new format without
> a
> > > > restart. At one extreme, we could put a watch on the znode, but this
> > > > could be quite heavy (I think the only per-client watch we have now
> is
> > > > on /ledgers/available). Alternatively, we could just read
> > > > /ledgers/LAYOUT once per day. This way clients who saw the old
> version
> > > > will eventually get updated.
> > > >
> > >
> > > I prefer NOT to use the watch, this cost will be payed forever even
> when
> > > there will be no configuration changes.
> > >
> > > I would go with the 'load at start' option, it is clear and works fine
> > >
> > >
> > >
> > > >
> > > > > What about having a client side config writeMetadataVersion ?
> > > > > We start a new metadata version, the new one  will be encoded as
> > > binary.
> > > > > By default 4.8 clients will use previous version, as we already do
> > for
> > > &

Re: Cookie for FaultDomain Info?

2019-05-26 Thread Venkateswara Rao Jujjuri
> I will also send soon a request for enhancements and a PR to have the
list of bookies

List of bookies in the cluster? how the available and RO are listed or list
of bookies in the metadata?

On Sat, May 25, 2019 at 9:02 AM Enrico Olivelli  wrote:

> JV,
> sorry for so late reply.
>
>
> Il lun 20 mag 2019, 07:54 Venkateswara Rao Jujjuri  ha
> scritto:
>
> > > we should take care of designing a better API for the placement policy
> > +1 Our placement policy is super complex and confusing.
> >
> > > We could also take into account the ability of adding labels/tags to
> > bookies.
> >
> > Can you add more context and color to your statement?
> > In the k8s world we do have a way to add tags. Can you please elaborate
> > what you are thinking?
> >
>
> Most of my products have strong requirements about multi tenancy and smart
> resource allocation.
> I have always very small clusters (3-5 bookies) so rack/region awareness is
> not my primary focus (I have very few cases of multi region clusters).
> Currently if you want to drive the allocation of resources you have to
> write your own placement policy, put some tenant binding info on 'custom
> metadata' and use some out-of-band (in respect to BK)  information about
> bookies and which bookie can be used for the tenant who is requesting the
> ledger placement.
>
> Having some sort of tags directly set on bookies zk directory will drop the
> need for such out of bound information distribution.
>
> Some year ago we discussed about this topic but the discussion did not go
> anywhere.
> IIRC the reason was that generic labels may be too generic and make the
> placement rules too complex.
>
> I see value on this fault domain id.
>
> I think we should have some information on zookeeper about which fault
> domains are defined.
> We can't discover new fault domains just by listing bookies metadata.
>
> I will also send soon a request for enhancements and a PR to have the list
> of bookies (currently we only have APIs to discovery bookies that are up
> and running).
>  One of my colleagues is working on management tools and we found the lack
> of this information.
> The script approach for dns to switch mapping also makes it impossible to
> understand the topology of the cluster without testing every bookie
> address.
>
> Enrico
>
>
>
> >
> > On Sun, May 19, 2019 at 10:23 PM Enrico Olivelli 
> > wrote:
> >
> > > Il lun 20 mag 2019, 05:03 Sijie Guo  ha scritto:
> > >
> > > > +1 from me. `Cookie` was designed for keeping the informations that
> is
> > > > associated with a bookie (e.g. disk layouts, bookie id and etc).
> > > >
> > > > I think it is making sense to have `FaultZoneId` stored as part of
> the
> > > > cookie.
> > > >
> > >
> > > I agree.
> > >
> > > But we should take care of designing a better API for the placement
> > policy.
> > > We are changing signatures quite often, adding parameters, changing
> > return
> > > type
> > >
> > > We could also take into account the ability of adding labels/tags to
> > > bookies.
> > >
> > > Enrico
> > >
> > >
> > >
> > >
> > > > - Sijie
> > > >
> > > > On Sun, May 19, 2019 at 9:48 AM Venkateswara Rao Jujjuri <
> > > > jujj...@gmail.com>
> > > > wrote:
> > > >
> > > > > In the current code, bookie to faultDomain mapping is supplied
> > through
> > > > > different methods. Salesforce uses a script to read yaml file which
> > > > > contains racks/machines mapping. But I am wondering why can't we
> put
> > > this
> > > > > info in the Cookie? Assuming that these machines can never move
> > across
> > > > > fault zones.
> > > > >
> > > > > Currently cookies contain  version, Host, JourlanDirs, ledgerDirs,
> > and
> > > > > instanceId.
> > > > > If we add faultzoneId to it, it will be always available for
> everyone
> > > to
> > > > > look into.
> > > > > Is there any reason why it would be a bad idea?
> > > > >
> > > > > Thanks,
> > > > > --
> > > > > 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
> >
>


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


Re: Cookie for FaultDomain Info?

2019-05-19 Thread Venkateswara Rao Jujjuri
> we should take care of designing a better API for the placement policy
+1 Our placement policy is super complex and confusing.

> We could also take into account the ability of adding labels/tags to
bookies.

Can you add more context and color to your statement?
In the k8s world we do have a way to add tags. Can you please elaborate
what you are thinking?


On Sun, May 19, 2019 at 10:23 PM Enrico Olivelli 
wrote:

> Il lun 20 mag 2019, 05:03 Sijie Guo  ha scritto:
>
> > +1 from me. `Cookie` was designed for keeping the informations that is
> > associated with a bookie (e.g. disk layouts, bookie id and etc).
> >
> > I think it is making sense to have `FaultZoneId` stored as part of the
> > cookie.
> >
>
> I agree.
>
> But we should take care of designing a better API for the placement policy.
> We are changing signatures quite often, adding parameters, changing return
> type
>
> We could also take into account the ability of adding labels/tags to
> bookies.
>
> Enrico
>
>
>
>
> > - Sijie
> >
> > On Sun, May 19, 2019 at 9:48 AM Venkateswara Rao Jujjuri <
> > jujj...@gmail.com>
> > wrote:
> >
> > > In the current code, bookie to faultDomain mapping is supplied through
> > > different methods. Salesforce uses a script to read yaml file which
> > > contains racks/machines mapping. But I am wondering why can't we put
> this
> > > info in the Cookie? Assuming that these machines can never move across
> > > fault zones.
> > >
> > > Currently cookies contain  version, Host, JourlanDirs, ledgerDirs, and
> > > instanceId.
> > > If we add faultzoneId to it, it will be always available for everyone
> to
> > > look into.
> > > Is there any reason why it would be a bad idea?
> > >
> > > Thanks,
> > > --
> > > 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


Cookie for FaultDomain Info?

2019-05-18 Thread Venkateswara Rao Jujjuri
In the current code, bookie to faultDomain mapping is supplied through
different methods. Salesforce uses a script to read yaml file which
contains racks/machines mapping. But I am wondering why can't we put this
info in the Cookie? Assuming that these machines can never move across
fault zones.

Currently cookies contain  version, Host, JourlanDirs, ledgerDirs, and
instanceId.
If we add faultzoneId to it, it will be always available for everyone to
look into.
Is there any reason why it would be a bad idea?

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


Re: Long poll LAC returning immediately on fenced

2019-03-29 Thread Venkateswara Rao Jujjuri
Does the LAC change on that bookie if ledger is fenced?

On Fri, Mar 29, 2019 at 6:49 AM Ivan Kelly  wrote:

> Hi folks,
>
> I'm seeing a problem where a bookie is getting hammer by long poll
> requests. ~8000rps. This seems to be happening because the long poll
> logic doesn't wait if the ledger is in fenced state [1], but returns
> immediately. So the client ends up in a tight loop if the ledger has
> entered fenced state and doesn't exit (I haven't found the root cause
> for this yet).
>
> Does anyone know why this check for fenced is there? LAC can change
> during a recovery op, so it still makes sense to wait for the timeout.
>
> Cheers,
> Ivan
>
>
> [1]
> https://github.com/apache/bookkeeper/blob/e3d807a32a0a9b69d0ac8db3ca17398373dbee28/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java#L140
>


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


Re: Cutting 4.9.1

2019-03-13 Thread Venkateswara Rao Jujjuri
+1

On Wed, Mar 13, 2019 at 6:36 PM Sijie Guo  wrote:

> +1
>
> On Wed, Mar 13, 2019 at 5:16 PM Enrico Olivelli 
> wrote:
>
> > Hi guys,
> > I need to cut 4.9.1 because we are seeing very often this issue
> >
> >
> https://github.com/apache/bookkeeper/commit/25c7506c0513351c533db643cb10c953d1e6d0b7
> >
> > Please tag any issue you want to merge into 4.9 branch.
> >
> > I will start the release process hopefully within the end of the week
> >
> > Please remember that Ivan started a VOTE thread for the 4.8 branch as
> well
> >
> > Cheers
> > Enrico
> >
>


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


Re: How to handle stale metadata?

2019-02-19 Thread Venkateswara Rao Jujjuri
On Mon, Feb 18, 2019 at 5:49 PM Sijie Guo  wrote:

> On Tue, Feb 19, 2019 at 4:22 AM Venkateswara Rao Jujjuri <
> jujj...@gmail.com>
> wrote:
>
> > Recently we ran into a situation where the LedgerMetadataListener never
> > returned/detected metadata change. Due to this reader had stale metadata
> > and tried to read from bookies that no longer have that ledger, hence
> > NoSuchLedgerExistsException was returned to the caller.
> >
>
> Do the bookies have entries? Or does it change ensemble to cause an empty
> fragment?
>

Replication worker migrated ledgers on to other bookies. But the client is
pointing to old set of bookies.

Those bookies respond NoSuchLedgerExistsException when talking to the empty
> fragment.
> Is that the case? I am not sure how NoSuchLedgerExistsException can be
> propagated to the client.
>

Since client is contacting old bookies, which doesn't have the ledger
anymore, it is returning NoSuchLedger.
and all 3 bookies returned the same error, hence client returned that to
user.


> Can you describe the sequence on how this happened?
>
>
> >
> > 1. I wonder if NoSuchLedgerExistsException is the right error here?
>
>
> >- Client knows that the ledger exists in the metadata. It has valid
> >handle. So ledger *Exists*.
> >-  In this case it is stale metadata so a restart of client took care
> of
> >the situation. But what if the ledger is in ZK, but missing from all
> >bookies? This can be a durability or availability issue based on the
> >bookies in the metadata are part of the cluster or not.
> >- I think we need to have more sophisticated error handling here.
> >Comments?
> >
>
>
>
>
>
> >
> > 2. Having too many watches puts memory pressure on the client.
> >
> >- How about having an option to re-read the metadata on demand w/o
> > watch?
> >
>
> +1 this has been in my todo list for a while. we should provide options to
> do this either by watches or by re-read scheduling or both.
>
>
> >   - Schedule a task to reread metadata on the first bookie failure
> with
> >   NoSuchEntry/NoSuchLedger.
> >   - If all three bookies fail, wait for the outstanding metadata read
> >   to return before failing to user.
> >   - If the metadata is read, and is different from the local copy,
> >   reattempt the read.
> >   - If metadata is not different, then fail with "some new error"
> >   DataLossException or something?
> >- This can cause latency if the metadata is changing a lot, but may be
> >better than constant watches? It could be a configuration option.
> >- We could even think of having both enabled if the reader is super
> >conservative.
> >
> >
> > Thoughts?
> > JV
> >
> >
> > --
> > 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


How to handle stale metadata?

2019-02-18 Thread Venkateswara Rao Jujjuri
Recently we ran into a situation where the LedgerMetadataListener never
returned/detected metadata change. Due to this reader had stale metadata
and tried to read from bookies that no longer have that ledger, hence
NoSuchLedgerExistsException was returned to the caller.

1. I wonder if NoSuchLedgerExistsException is the right error here?

   - Client knows that the ledger exists in the metadata. It has valid
   handle. So ledger *Exists*.
   -  In this case it is stale metadata so a restart of client took care of
   the situation. But what if the ledger is in ZK, but missing from all
   bookies? This can be a durability or availability issue based on the
   bookies in the metadata are part of the cluster or not.
   - I think we need to have more sophisticated error handling here.
   Comments?

2. Having too many watches puts memory pressure on the client.

   - How about having an option to re-read the metadata on demand w/o watch?
  - Schedule a task to reread metadata on the first bookie failure with
  NoSuchEntry/NoSuchLedger.
  - If all three bookies fail, wait for the outstanding metadata read
  to return before failing to user.
  - If the metadata is read, and is different from the local copy,
  reattempt the read.
  - If metadata is not different, then fail with "some new error"
  DataLossException or something?
   - This can cause latency if the metadata is changing a lot, but may be
   better than constant watches? It could be a configuration option.
   - We could even think of having both enabled if the reader is super
   conservative.


Thoughts?
JV


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


Re: Community Meetings 2019

2019-02-10 Thread Venkateswara Rao Jujjuri
Having one standard time is good. But there needs to be participation too,
otherwise it won't be successful.
So lets find a good time that everyone can join. Other proposal is to setup
two meetings in a month, one is convenient for US
and other is convenient for Asia/Europe. Personally I am fine to take call
in the night if it is not between 6-9PM PST.
In the morning I am fine to take early calls even at 7 AM. I am just
stating my availability, and the goal should be larger participation IMO.

JV

On Sun, Feb 10, 2019 at 9:14 PM Sijie Guo  wrote:

> I think it is very difficult to find a good time slot for US, Europe and
> Asia.
>
> I would propose followings:
>
> - keep the regular community meetings at 8AM PST and make sure all meeting
> notes are updated and shared via mailing lists. Release manager should
> drive the community meeting.
> - additionally, if there are topics required ad-hoc calls, we can set up
> ad-hoc calls for the people required to be involved in the call. Also make
> sure all meeting notes are shared to the community.
> - lastly, Ivan and me have been doing bug triages for Pulsar. We might be
> starting doing same things for BookKeeper. It will be running weekly. We
> will document the bug triage process and share the meeting details.
> Everyone are recommended to join the triage as well.
>
> Let me know your thoughts.
>
> - Sijie
>
> On Fri, Feb 8, 2019 at 4:10 PM Enrico Olivelli 
> wrote:
>
> > Hi folks,
> > I think it is time to resume Community Meetings in 2019.
> >
> > Currently we are running the meeting at 8 AM PST, every two weeks.
> >
> > We can decide to change the date, if we find it uncomfortable.
> >
> > Jia and I were proposing to have a variable time for the call,
> > depending on the needs or depending from the person who "calls" for a
> > specific topic.
> >
> > I see that that time is not good for Jia and people from China.
> >
> > The goals of the community meetings are:
> > - Let community members 'meet' each other
> > - Discuss proposals
> > - Discuss about problems
> > - Share our projects/usages of BookKeeper/DL
> >
> > Every decision in ASF must be taken on ML, but we are humans and
> > humans "like" to talk and I find this meetings an important way of
> > "making community"
> >
> > Best regards
> >
> > Enrico
> >
>


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


Re: ExplicitLAC consolidation

2019-02-10 Thread Venkateswara Rao Jujjuri
Thanks for bringing this up. To start with, can we put all forms on LAC in
a doc? i.e we have more than one way to read and write LAC.
1. What are the ways to send LAC to Bookies?
2. What are the ways bookies store the LAC?
3. What are the ways client can learn about LAC
and the interfaces and configuration parameters around it, and what are the
usecases of these?

I may be asking for more work here, but I think that way we can make more
informed decision. What do you think Enrico?
If you lead this, I will be more than happy to fill-in parts of this doc,
and even review them.

Thanks,
JV

On Sun, Feb 10, 2019 at 1:23 AM Enrico Olivelli  wrote:

> Hi Bookkeepers,
> I am trying to draw the best roadmap with the goal of consolidating
> ExplicitLAC feature.
> Currently I have two big topics:
> - on the reader side I would like enable new API users to leverage
> ExplicitLAC, transparently (no new configuration, now readflags, no
> explicit API calls)
> - on the writer side make ExplicitLAC work better with DEFERRED_SYNC (like
> having a background force() together with the sending of ExplicitLAC)
>
> Currently I want to spend time mostly on the reader side, because it will
> enable new clients to use ExplicitLAC.
>
> My current (new) idea is to add a new flag on readEntry() RPC with which
> the client asks for the ExplicitLAC together with the entry.
>
> With this change we can support backward compatibility easily.
> New clients will add that flag and they will be able to read the new
> optional ExplicitLAC field.
> Old bookies will ignore the flag.
> Old clients won't ask for ExplicitLAC.
>
> If there is no ExplicitLAC this new feature won't add costs on the wire.
>
> After this discussion on the ML I will post a BP, if we agree that this
> approach makes sense.
>
> I have started to draft a prototype yet, I would like to hear your opinion
> (as usual I have very limited time)
>
> Regards
> Enrico
>


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


Fwd: [apache/bookkeeper] Your project apache/bookkeeper is using buggy third-party libraries [WARNING] (#1896)

2019-01-06 Thread Venkateswara Rao Jujjuri
-- Forwarded message -
From: FDU-SE-LAB 
Date: Sun, Jan 6, 2019 at 8:33 PM
Subject: [apache/bookkeeper] Your project apache/bookkeeper is using buggy
third-party libraries [WARNING] (#1896)
To: apache/bookkeeper 
Cc: Subscribed 


Hi, there!
We are a research team working on third-party library analysis. We have
found that some widely-used third-party libraries in your project have
major/critical bugs, which will degrade the quality of your project. We
highly recommend you to update those libraries to new versions.
We have attached the buggy third-party libraries and corresponding jira
issue links below for you to have more detailed information.
1 commons-cli commons-cli (pom.xml)
version: 1.2

Jira issues:
Unable to select a pure long option in a group
affectsVersions:1.0;1.1;1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-182?filter=allopenissues
Clear the selection from the groups before parsing
affectsVersions:1.0;1.1;1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-183?filter=allopenissues
Commons CLI incorrectly stripping leading and trailing quotes
affectsVersions:1.1;1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-185?filter=allopenissues
Coding error: OptionGroup.setSelected causes java.lang.NullPointerException
affectsVersions:1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-191?filter=allopenissues
StringIndexOutOfBoundsException in HelpFormatter.findWrapPos
affectsVersions:1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-193?filter=allopenissues
HelpFormatter strips leading whitespaces in the footer
affectsVersions:1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-207?filter=allopenissues
OptionBuilder only has static methods; yet many return an OptionBuilder
instance
affectsVersions:1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-224?filter=allopenissues
Unable to properly require options
affectsVersions:1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-230?filter=allopenissues
OptionValidator Implementation Does Not Agree With JavaDoc
affectsVersions:1.2
https://issues.apache.org/jira/projects/CLI/issues/CLI-241?filter=allopenissues

2 commons-io commons-io (pom.xml)
version: 2.4

Jira issues:
IOUtils copyLarge() and skip() methods are performance hogs
affectsVersions:2.3;2.4
https://issues.apache.org/jira/projects/IO/issues/IO-355?filter=allopenissues
CharSequenceInputStream#reset() behaves incorrectly in case when buffer
size is not dividable by data size
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-356?filter=allopenissues
[Tailer] InterruptedException while the thead is sleeping is silently
ignored
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-357?filter=allopenissues
IOUtils.contentEquals* methods returns false if input1 == input2; should
return true
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-362?filter=allopenissues
Apache Commons - standard links for documents are failing
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-369?filter=allopenissues
FileUtils.sizeOfDirectoryAsBigInteger can overflow
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-390?filter=allopenissues
Regression in FileUtils.readFileToString from 2.0.1
affectsVersions:2.1;2.2;2.3;2.4
https://issues.apache.org/jira/projects/IO/issues/IO-453?filter=allopenissues
Correct exception message in FileUtils.getFile(File; String...)
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-479?filter=allopenissues
org.apache.commons.io.FileUtils#waitFor waits too long
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-481?filter=allopenissues
FilenameUtils should handle embedded null bytes
affectsVersions:2.4
https://issues.apache.org/jira/projects/IO/issues/IO-484?filter=allopenissues
Exceptions are suppressed incorrectly when copying files.
affectsVersions:2.4;2.5
https://issues.apache.org/jira/projects/IO/issues/IO-502?filter=allopenissues

3 commons-codec commons-codec (pom.xml)
version: 1.6

Jira issues:
QuotedPrintableCodec does not support soft line break per the
'quoted-printable' example on Wikipedia
affectsVersions:1.5;1.6
https://issues.apache.org/jira/projects/CODEC/issues/CODEC-121?filter=allopenissues
BeiderMorseEncoder OOM issues
affectsVersions:1.6
https://issues.apache.org/jira/projects/CODEC/issues/CODEC-132?filter=allopenissues
BeiderMorse phonetic filter give uncertain results
affectsVersions:1.6
https://issues.apache.org/jira/projects/CODEC/issues/CODEC-147?filter=allopenissues
DigestUtils.getDigest(String) looses the orginal exception
affectsVersions:1.6
https://issues.apache.org/jira/projects/CODEC/issues/CODEC-152?filter=allopenissues
DigestUtils.getDigest(String) should throw IllegalArgumentException instead
of RuntimeException
affectsVersions:1.6
https://issues.apache.org/jira/projects/CODEC/issues/CODEC-155?filter=allopenissues
DigestUtils: 

Re: Clusterwide vs Client configuration for metadata format version

2018-12-18 Thread Venkateswara Rao Jujjuri
If it is client level configuration, in theory it is possible to have
latest client create v3 ledger while bookies
are still running in the older version right? Who can stop that? if we let
that happen what happens to the replication
logic? How can it handle the new ledger format?

If we go with cluster level, I think using it part of LAYOUT_ZNODE is not
clean.
I think we need to have a form of "cluster version number", or even better
with a combination of  capability/feature
bit-map which can dictate the cluster behavior.

I am assuming that the tool Ivan is talking about is used for existing
clusters to update the cluster version number.
Otherwise the maxLedgerMetadataFormat is used only for new clusters; that
is fine.
But this comes with strict operational guidelines where the
maxLedgerMetadataFormat needs to be updated
after successful upgrade of the entire cluster to new bits. In this case at
least we have a barrier that the entire
bookies are updated which can understand maxLedgerMetadataFormat, and we
support backward compatibility anyway.
But I don't like the way to overload LAYOUT_ZNODE which doesn't make sense
as it is not a layout change.

JV


On Tue, Dec 18, 2018 at 11:37 AM Sam Just  wrote:

> I think both approaches are viable, but I think that the max allowable
> version is more naturally a bk cluster property rather than a bk client
> property.  Controlling this from the client means that the same client
> version deployed to two different clusters might need different settings
> depending on the other clients deployed to those clusters.  Placing it in
> the metadata means that the clients simply pick up the correct version for
> the environment from the ledger metadata without needing additional
> configuration.  However, client config management is likely to be managed
> on a per-cluster basis anyway, so in practice there may be little
> difference.
> -Sam
>
> On Tue, Dec 18, 2018 at 10:01 AM Sam Just  wrote:
>
> > I'll take a look.
> >
> > On Tue, Dec 18, 2018 at 1:39 AM Ivan Kelly  wrote:
> >
> >> JV, Sam, Charan, Andrey, could one of you chime in on this? It's
> >> holding up 4.9 release.
> >>
> >> -Ivan
> >>
> >> On Thu, Dec 13, 2018 at 5:38 PM Ivan Kelly  wrote:
> >> >
> >> > I'd be interested to see the opinion of the salesforce folks on this.
> >> > On Thu, Dec 13, 2018 at 5:35 PM Ivan Kelly  wrote:
> >> > >
> >> > > > I am not sure about this. If clients don't react the changes of
> >> ledger
> >> > > > layout,
> >> > > > the information in ledger layout is just informative, you still
> >> need to
> >> > > > coordinate
> >> > > > both readers and writers. so IMO the version in ledger layout is
> >> not really
> >> > > > useful.
> >> > >
> >> > > The clients react the next time they initialize the ledger manager.
> >> > > Which is exactly the same as would occur with a configuration
> setting.
> >> > >
> >> > > -Ivan
> >>
> >
> >
> > --
> >
> > 
> >
>
>
> --
>
> 
>


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


Re: Community meeting ?

2018-12-12 Thread Venkateswara Rao Jujjuri
I agree; lets do that.

On Wed, Dec 12, 2018 at 8:58 AM Enrico Olivelli  wrote:

> Hi guys,
> tomorrow we would have the Community Meeting.
>
> I wonder if we would better to suspend such activity during Winter
> Holidays.
>
> Maybe it would be a change to revive this date after a period of
> suspension, in January.
>
>
> Thoughts ?
>
> Enrico
>


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


Re: Bug with blockAddCompletions decrement

2018-12-03 Thread Venkateswara Rao Jujjuri
On Mon, Dec 3, 2018 at 1:40 AM Ivan Kelly  wrote:

> > This may not be an issue with the master as we moved to immutable
> metadata,
>
> Not an issue with master as blockAddCompletions has been replaced with
> a simple boolean and failure handling changed to only deal with one
> failure at a time. However, looking at it again, I think I did spot a
> similar bug. will dig in after I send this.
>

Great; looking forward.


> Your email client mangled the code, but as far as I could untangle it.
>
> updateMetadataIfPossible: No bug. decrement gets called when
> ChangeEnsembleCb completes.
>

updateMetadataIfPossible() is called from resolveConflict() -> "return
updateMetadataIfPossible(newMeta);"
There is only one case in the  updateMetadataIfPossible(newMeta); that will
rereadMetadata in other cases
it can just return TRUE so there is no ChangeEnsembleCb.

I will work on  test cases.

Thanks,
JV


> resolveConflict: Looks like a bug. Would be good to had a test case
> that tickles it.
>
> -Ivan
>


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


Bug with blockAddCompletions decrement

2018-12-02 Thread Venkateswara Rao Jujjuri
This may not be an issue with the master as we moved to immutable metadata,
but older releases have an issue with not decrementing  blockAddCompletions.


Sijie/Ivan/Sam can you validate this?


blockAddCompletions gets incremented while initiating the ensemble change
in handleBookieFailure(). Based on how we handle the bookie failure this
should get decremented. Until this comes back to zero there wont be any
addEntryCompletions are sent to the application.



I found two areas where this counter may not get decremented. (Look for
"BUG" in the code snippet below)



   1. 1.
   2.
   3. updateMetadataIfPossible -> This should decrement blockAddCompletions
   if returning TRUE
   4. {
   5. if (metadata.isNewerThan(newMeta)) {
   6. rereadMetadata(this); ==> This is a loop back to reattempt which
   comes back to here.
   7. return true;
   8. }
   9. if (metadata.isConflictWith(newMeta)) {
   10. return false;
   11. }
   12. // update znode version
   13. metadata.setVersion(newMeta.getVersion());
   14. // merge ensemble infos from new meta except last ensemble
   15. // since they might be modified by recovery tool.
   16. metadata.mergeEnsembles(newMeta.getEnsembles());
   17. writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo,
   curBlockAddCompletions,
   18. ensembleChangeIdx));
   19. return true;
   20. > BUG: We are NOT decrementing blockAddCompletions
   21. }
   22.
   23. 2.
   24. resolveConflict()
   25. {
   26. int diff = newMeta.getEnsembles().size() - metadata.getEnsembles().
   size();
   27. if (0 != diff) {
   28. if (-1 == diff) {
   29. // Case 1: metadata is changed by other ones (e.g. Recovery)
   30. return updateMetadataIfPossible(newMeta);
   31. }
   32. return false;
   33. }
   34. if (!areFailedBookiesReplaced(newMeta, ensembleInfo)) {
   35. if (areFailedBookiesReplaced(metadata, ensembleInfo)) {
   36. return updateMetadataIfPossible(newMeta);
   37. }
   38. else??
   39. ===>BUG: There is no else case here. If there can be an else case
   here, we are returning TRUE but not decrementing blockAddCompletions.
   40. } else {
   41. int newBlockAddCompletions = blockAddCompletions.decrementAndGet();
   42. unsetSuccessAndSendWriteRequest(ensembleInfo.replacedBookies);
   43. }
   44. return true;
   45. }

@Sam Just
  let
us discuss this tomorrow.


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


Re: BookKeeper Board report draft - due to today

2018-11-14 Thread Venkateswara Rao Jujjuri
+1 Thanks Enrico.

JV

On Wed, Nov 14, 2018 at 2:15 PM Jia Zhai  wrote:

> +1, Thanks for the help Enrico, Seems only PMC Chair could do it.
>
> On Thu, Nov 15, 2018 at 2:42 AM Enrico Olivelli 
> wrote:
>
> > Sijie,
> > can I submit the report or only the PMC chair can ?
> > The tool requires an authentication but I cannot login with my Apache
> > credentials
> >
> > Enrico
> > Il giorno mer 14 nov 2018 alle ore 19:32 Sijie Guo
> >  ha scritto:
> > >
> > > LGTM +1
> > >
> > > - Sijie
> > >
> > > On Wed, Nov 14, 2018 at 10:21 AM Enrico Olivelli 
> > wrote:
> > >>
> > >> Hi all,
> > >>
> > >> We have a board report due today. Here is a draft. Please take a look
> > and
> > >> let me know what you think.
> > >>
> > >> -
> > >>
> > >> ## Description:
> > >>
> > >> BookKeeper is a scalable, fault-tolerant, and low-latency storage
> > >> service optimized for append-only workloads. It has been used as
> > >> a fundamental service to build high available and replicated services
> > >> in companies like Twitter, Yahoo and Salesforce. It is also the log
> > >> segment store for Apache DistributedLog and message store for Apache
> > Pulsar.
> > >>
> > >> Apache DistributedLog is a high-level API and service layer for
> > >> Apache BookKeeper, providing easier access to the BookKeeper
> > >> primitives. It is a subproject of Apache BookKeeper.
> > >>
> > >> ## Issues:
> > >>
> > >> There are no issues requiring board attention at this time.
> > >>
> > >> ## Activity:
> > >>
> > >>  - 4.8.0 was released on Wed Sep 26 2018
> > >>  - 4.8.1 is going to be released
> > >>  - The growth of Apache Pulsar community also help grow the adoption
> > >> of BookKeeper.
> > >>This helps building the ecosystem around BookKeeper.
> > >> ## Health report:
> > >>
> > >>  - Development continues at a steady pace. We are merging multiple PRs
> > per
> > >> week on average.
> > >>  - Mailing list and slack discussions are brisk, in particularly
> around
> > the
> > >> active projects.
> > >>  - The community holds a bi-weekly meeting on Hangout.
> > >>
> > >> ## PMC changes:
> > >>
> > >>  - Currently 15 PMC members.
> > >>  - No new PMC members added in the last 3 months
> > >>  - Last PMC addition was Enrico Olivelli on Thu Feb 22 2018
> > >>
> > >> ## Committer base changes:
> > >>
> > >>  - Currently 21 committers.
> > >>  - No new committers added in the last 3 months
> > >>  - Last committer addition was Andrey Yegorov at Fri Feb 09 2018
> > >>
> > >> ## Releases:
> > >>
> > >>  - 4.7.1 was released on Tue Jun 19 2018
> > >>  - 4.8.0 was released on Wed Sep 26 2018
> > >>
> > >> ## PMC changes:
> > >>
> > >>  - Currently 15 PMC members.
> > >>  - No new PMC members added in the last 3 months
> > >>  - Last PMC addition was Enrico Olivelli on Fri Feb 23 2018
> > >>
> > >> ## Committer base changes:
> > >>
> > >>  - Currently 21 committers.
> > >>  - No new committers added in the last 3 months
> > >>  - Last committer addition was Andrey Yegorov at Fri Feb 09 2018
> > >>
> > >> ## Releases:
> > >>
> > >>  - 4.8.0 was released on Wed Sep 26 2018
> > >>
> > >> ## Mailing list activity:
> > >>
> > >>  - Discussions mostly run on dev@ mailing list, distributedlog lists
> > >> are mostly unused, now that the project as been merged completely with
> > >> BookKeeper and users started to use the version bundled with
> > >> BookKeeper releases.
> > >>
> > >>  - dev@bookkeeper.apache.org:
> > >> - 103 subscribers (down -1 in the last 3 months):
> > >> - 125 emails sent to list (132 in previous quarter)
> > >>
> > >>  - distributedlog-comm...@bookkeeper.apache.org:
> > >> - 12 subscribers (up 0 in the last 3 months):
> > >> - 0 emails sent to list (2 in previous quarter)
> > >>
> > >>  - distributedlog-...@bookkeeper.apache.org:
> > >> - 41 subscribers (down -1 in the last 3 months):
> > >> - 0 emails sent to list (0 in previous quarter)
> > >>
> > >>  - distributedlog-iss...@bookkeeper.apache.org:
> > >> - 9 subscribers (up 0 in the last 3 months):
> > >> - 4 emails sent to list (2 in previous quarter)
> > >>
> > >>  - distributedlog-u...@bookkeeper.apache.org:
> > >> - 26 subscribers (up 0 in the last 3 months):
> > >> - 0 emails sent to list (0 in previous quarter)
> > >>
> > >>  - iss...@bookkeeper.apache.org:
> > >> - 8 subscribers (down -1 in the last 3 months):
> > >> - 2686 emails sent to list (1952 in previous quarter)
> > >>
> > >>  - u...@bookkeeper.apache.org:
> > >> - 115 subscribers (down -2 in the last 3 months):
> > >> - 20 emails sent to list (39 in previous quarter)
> > >>
> > >>
> > >>
> > >>
> > >> Thanks,
> > >> Enrico
> >
>


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


Re: Community meeting today

2018-10-18 Thread Venkateswara Rao Jujjuri
Yeah; will we have anyone joining to talk about etcd?

On Thu, Oct 18, 2018 at 1:19 AM Enrico Olivelli  wrote:

> Hi guys,
> today we will have the community meeting.
>
> link to the agenda
>
> https://cwiki.apache.org/confluence/display/BOOKKEEPER/2018-10-18+Meeting+Notes
>
> It would be great to talk about ETCD integration status.
>
> With Ivan we are seeing that there is some mess on CI, it seems that
> ETCD tests are always run, even with "java8 tests"
>
> We should talk about a future release manager for 4.9
>
> Enrico
>


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


Re: BookKeeper Community Meeting

2018-10-04 Thread Venkateswara Rao Jujjuri
Mine is iffy. If I can, I will join around 8:15

On Thu, Oct 4, 2018 at 12:48 AM Enrico Olivelli  wrote:

> Hi
> I have created the wiki page for the meeting
>
>
> https://cwiki.apache.org/confluence/display/BOOKKEEPER/2018-10-04+Meeting+Notes
>
> Unfortunately I won't attend this time
>
>
> Enrico
>


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


Re: [DISCUSS] BP-35: 128 bits support

2018-08-18 Thread Venkateswara Rao Jujjuri
it. For the concerns in this BP, we
> > are
> > > > more focused on how we can transition from 64 bit to 128 bit.
> > > >
> > >
> > > Okay
> > >
> > > Last comment:
> > > We are going to have a lot of special cases in layout of data/zk ledger
> > > manager, to handle scopeid = 0.
> >
> >
> > > I am thinking to a brand new installation, without upgrade/BC
> concerns, I
> > > wonder if we have some flag not to have that tricks, I am talking about
> > the
> > > different dbs on rocks db, the layout of indexes...and so on.
> >
> >
> >  I don’t think we should provide this flag since it would make things
> very
> > complicated. In that case, you can just provide a brand new metadata
> > driver, a new ledger storage. The you don’t need to worry about this
> > upgrade story. However that will be brand new implementation, which would
> > be out of the scope of this BP.
> >
>
> Okay
>
> To me it is clear.
>
> +1
>
> Enrico
>
>
> > Sijie
> >
> >
> >
> > >
> > > This is not a real problem but I wonder if we ca handle this case of
> new
> > > applications cleanly
> > >
> > > Enrico
> > >
> > >
> > >
> > > > or as a last
> > > > > alternative we should add a way to list the space of scopeids,
> > because
> > > > > otherwise the application must store the list of scopeids, and for
> it
> > > it
> > > > > will be a set of half UUIDs.
> > > > >
> > > > > Thanks for driving this
> > > > >
> > > > > Enrico
> > > > >
> > > > >
> > > > >
> > > > > Il ven 17 ago 2018, 23:52 Venkateswara Rao Jujjuri <
> > jujj...@gmail.com>
> > > > ha
> > > > > scritto:
> > > > >
> > > > > > Thanks a lot, Sijie for the awesome doc.
> > > > > > Let's pool some more thoughts into this and size the work.
> > > > > >
> > > > > > Thanks,
> > > > > > JV
> > > > > >
> > > > > > On Fri, Aug 17, 2018 at 2:36 PM, Sijie Guo 
> > > wrote:
> > > > > >
> > > > > > > Hi all,
> > > > > > >
> > > > > > > As promised in last community meeting, I put up all the
> thoughts
> > > and
> > > > > > > discussion with JV I had into a BP for supporting 128-bit
> ledger
> > > id.
> > > > I
> > > > > > > tried my best to cover all the aspects that I can think of.
> There
> > > can
> > > > > be
> > > > > > > places that I miss. so please take a look and let me know what
> > you
> > > > > think.
> > > > > > >
> > > > > > > BP PR: https://github.com/apache/bookkeeper/pull/1611
> > > > > > > Google doc:
> > > > > > >
> > https://docs.google.com/document/d/1cu54dNSV2ZrdWCi40LcyX8NxXGRCW
> > > > > > > 0609T_ewmK9BWM
> > > > > > >
> > > > > > > - Sijie
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Jvrao
> > > > > > ---
> > > > > > First they ignore you, then they laugh at you, then they fight
> you,
> > > > then
> > > > > > you win. - Mahatma Gandhi
> > > > > >
> > > > > --
> > > > >
> > > > >
> > > > > -- Enrico Olivelli
> > > > >
> > > >
> > > --
> > >
> > >
> > > -- Enrico Olivelli
> > >
> >
> --
>
>
> -- Enrico Olivelli
>



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


Re: [DISCUSS] BP-35: 128 bits support

2018-08-17 Thread Venkateswara Rao Jujjuri
Thanks a lot, Sijie for the awesome doc.
Let's pool some more thoughts into this and size the work.

Thanks,
JV

On Fri, Aug 17, 2018 at 2:36 PM, Sijie Guo  wrote:

> Hi all,
>
> As promised in last community meeting, I put up all the thoughts and
> discussion with JV I had into a BP for supporting 128-bit ledger id. I
> tried my best to cover all the aspects that I can think of. There can be
> places that I miss. so please take a look and let me know what you think.
>
> BP PR: https://github.com/apache/bookkeeper/pull/1611
> Google doc:
> https://docs.google.com/document/d/1cu54dNSV2ZrdWCi40LcyX8NxXGRCW
> 0609T_ewmK9BWM
>
> - Sijie
>



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


Re: Usefulness of ensemble change during recovery

2018-08-06 Thread Venkateswara Rao Jujjuri
I don't think it a good idea to leave the tail to the replication.
This could lead to the perception of data loss, and it's more evident in
the case of larger WQ and disparity with AQ.
If we determine LLAC based on having 'a copy', which is never acknowledged
to the client, and if that bookie goes down(or crashes and burns)
before replication worker gets a chance, it gives the illusion of data
loss. Moreover, we have no way to determine the real data loss vs
this scenario where we have never acknowledged the client.


On Mon, Aug 6, 2018 at 12:32 AM, Sijie Guo  wrote:

> On Mon, Aug 6, 2018 at 12:08 AM Ivan Kelly  wrote:
>
> > >> Recovery operates on a few seconds of data (from the last LAC written
> > >> to the end of the ledger, call this LLAC).
> > >
> > > the data during this duration can be very large if the traffic of the
> > > ledger is large. That has
> > > been observed at Twitter's production. so when we are talking about "a
> > few
> > > seconds of data",
> > > we can't assume the amount of data is little. That says the recovery
> can
> > be
> > > taking time than
> >
> > Yes, it can be large, but still it is only a few seconds worth of
> > data. It is the amount of data that can be transmitted in the period
> > of one roundtrip, as the next roundtrip will update the LAC.
>
>
> > I didn't mean to imply the data was small. I was implying that the
> > data was small in comparison to the overall size of that ledger.
>
>
> > > what we can expect, so if we don't handle failures during recovery how
> we
> > > are able to ensure
> > > we have enough data copy during recovery.
> >
> > Consider a e3w3a2 ledger, there's two cases where you can lose a
> > bookie during recover.
> >
> > Case one, one bookie is lost. You can still recover from as ack=2 is
> > available.
> > Case two, two bookies are lost. You can't recover, but ledger is
> > unavailable anyhow, since any entry in the ledger may only have been
> > replicated to 2.
> >
> > However, with e3w3a3 I guess you wouldn't be able to recover at all,
> > and we have to handle that case.
> >
> > > I am not sure "make ledger metadata immutable" == "getting rid of
> merging
> > > ledger metadata".
> > > because I don't think these are same thing. making ledger metadata
> > > immutable will make code
> > > much clearer and simpler because the ledger metadata is immutable. how
> > > getting rid of merging
> > > ledger metadata is a different thing, when you make ledger metadata
> > > immutable, it will help make
> > > merging ledger metadata on conflicts clearer.
> >
> > I wouldn't call it merging in this case.
>
>
> That's fine.
>
>
> > Merging implies taking two
> > valid pieces of metadata and getting another usable, valid metadata
> > from it.
> > What happens with immutable metadata, is that you are taking one valid
> > metadata, and applying operations to it. So in the failure during
> > recovery place, we would have a list of AddEnsemble operations which
> > we add when we try to close.
> >
> > In theory this is perfectly valid and clean. It just can look messy in
> > the code, due to how the PendingAddOp reaches back into the ledger
> > handle to get the current ensemble.
> >
>
> That's okay since it is reality which we have to face anyway. But the most
> important thing
> is that we can't get rid of ensemble changes during ledger recovery.
>
>
> >
> > So, in conclusion, I will keep the handling.
>
>
> Thank you.
>
>
> > In any case, these
> > changes are all still blocked on
> > https://github.com/apache/bookkeeper/pull/1577.
> >
> > -Ivan
> >
>



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


Re: Changing ledger metadata to binary format

2018-07-30 Thread Venkateswara Rao Jujjuri
> As far as I understand we are working on this change in order to be ready
> with a full "rollback" story, so we have to be one version ahead.

If that is the case, I think we need to look at the entire roll-back story.
At this point, I am not sure if we have identified all the pieces and if
they fit together.
Enrico, are you driving the roll-back story? or whoever, can you please
come up with
a comprehensive proposal?

Thanks,
JV




On Mon, Jul 30, 2018 at 8:05 AM, Enrico Olivelli 
wrote:

> Il giorno lun 30 lug 2018 alle ore 16:55 Venkateswara Rao Jujjuri <
> jujj...@gmail.com> ha scritto:
>
> > We have introduced customMetadata in the past and that was not mandatory.
> > How did we make that work with text format?
> > Also, my preference is to wait until we have a need to introduce a new
> > field for this conversion if needed.
> >
>
> JV,
> So you are saying that you prefer to defer this change to the first case,
> when it will be really needed ?
>
> As far as I understand we are working on this change in order to be ready
> with a full "rollback" story, so we have to be one version ahead.
>
> Enrico
>
>
>
>
> >
> > JV
> >
> > On Mon, Jul 30, 2018 at 12:42 AM, Enrico Olivelli 
> > wrote:
> >
> > > Il giorno lun 30 lug 2018 alle ore 09:17 Ivan Kelly 
> > ha
> > > scritto:
> > >
> > > > >> Thank you for putting this together. It is also good to put this
> as
> > a
> > > > BP,
> > > > >> since it is about the metadata layout.
> > > >
> > > > I'll put a BP up this week once I have initial feedback.
> > > >
> > > > >> > - When writing a metadata, check what is in /ledgers/LAYOUT. If
> it
> > > is
> > > > >> > as above, write using the current text protobuf. If is bumped,
> use
> > > the
> > > > >> > new binary format.
> > > > >
> > > > > isn't it too costly? Adding a zk read for each write. We could add
> a
> > > > watch
> > > > > but is has an important cost
> > > >
> > > > This cost can be amortized. In simplest case, we only read when
> client
> > > > starts (as already happens). Client are able both read and write the
> > > > old format, so clients that read the pre-upgrade version will still
> > > > write in text format, and this is fine.
> > > >
> > >
> > > For me it is fine to read it when client starts. A rolling restart of
> the
> > > client application will be enough to load the new configuration.
> > >
> > >
> > > >
> > > > However, we would want them to eventually see the new format without
> a
> > > > restart. At one extreme, we could put a watch on the znode, but this
> > > > could be quite heavy (I think the only per-client watch we have now
> is
> > > > on /ledgers/available). Alternatively, we could just read
> > > > /ledgers/LAYOUT once per day. This way clients who saw the old
> version
> > > > will eventually get updated.
> > > >
> > >
> > > I prefer NOT to use the watch, this cost will be payed forever even
> when
> > > there will be no configuration changes.
> > >
> > > I would go with the 'load at start' option, it is clear and works fine
> > >
> > >
> > >
> > > >
> > > > > What about having a client side config writeMetadataVersion ?
> > > > > We start a new metadata version, the new one  will be encoded as
> > > binary.
> > > > > By default 4.8 clients will use previous version, as we already do
> > for
> > > > > journal and fileinfo on bookies.
> > > > >
> > > > > Each ledger is independent from the others, there is no need of a
> > > global
> > > > > flag written on zk.
> > > >
> > > > There is a global property here though,which only the administrator
> > > > can control. It is "Can all clients in the cluster/application read
> > > > the new ledger metadata version?". This is why I'd like to keep it as
> > > > a centralized flag rather than allowing each client to decide for
> > > > themselves.
> > > >
> > >
> > > ok, let's keep it
> > >
> > > Enrico
> > >
> > >
> > >
> > > >
> > > > > Don't we already have some tool? (Maybe I only have seen such tool
> in
> > > my
> > > > > company applications)
> > > >
> > > > It may already exist, I haven't checked.
> > > >
> > > > -Ivan
> > > >
> > >
> >
> >
> >
> > --
> > 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


Re: Changing ledger metadata to binary format

2018-07-30 Thread Venkateswara Rao Jujjuri
We have introduced customMetadata in the past and that was not mandatory.
How did we make that work with text format?
Also, my preference is to wait until we have a need to introduce a new
field for this conversion if needed.

JV

On Mon, Jul 30, 2018 at 12:42 AM, Enrico Olivelli 
wrote:

> Il giorno lun 30 lug 2018 alle ore 09:17 Ivan Kelly  ha
> scritto:
>
> > >> Thank you for putting this together. It is also good to put this as a
> > BP,
> > >> since it is about the metadata layout.
> >
> > I'll put a BP up this week once I have initial feedback.
> >
> > >> > - When writing a metadata, check what is in /ledgers/LAYOUT. If it
> is
> > >> > as above, write using the current text protobuf. If is bumped, use
> the
> > >> > new binary format.
> > >
> > > isn't it too costly? Adding a zk read for each write. We could add a
> > watch
> > > but is has an important cost
> >
> > This cost can be amortized. In simplest case, we only read when client
> > starts (as already happens). Client are able both read and write the
> > old format, so clients that read the pre-upgrade version will still
> > write in text format, and this is fine.
> >
>
> For me it is fine to read it when client starts. A rolling restart of the
> client application will be enough to load the new configuration.
>
>
> >
> > However, we would want them to eventually see the new format without a
> > restart. At one extreme, we could put a watch on the znode, but this
> > could be quite heavy (I think the only per-client watch we have now is
> > on /ledgers/available). Alternatively, we could just read
> > /ledgers/LAYOUT once per day. This way clients who saw the old version
> > will eventually get updated.
> >
>
> I prefer NOT to use the watch, this cost will be payed forever even when
> there will be no configuration changes.
>
> I would go with the 'load at start' option, it is clear and works fine
>
>
>
> >
> > > What about having a client side config writeMetadataVersion ?
> > > We start a new metadata version, the new one  will be encoded as
> binary.
> > > By default 4.8 clients will use previous version, as we already do for
> > > journal and fileinfo on bookies.
> > >
> > > Each ledger is independent from the others, there is no need of a
> global
> > > flag written on zk.
> >
> > There is a global property here though,which only the administrator
> > can control. It is "Can all clients in the cluster/application read
> > the new ledger metadata version?". This is why I'd like to keep it as
> > a centralized flag rather than allowing each client to decide for
> > themselves.
> >
>
> ok, let's keep it
>
> Enrico
>
>
>
> >
> > > Don't we already have some tool? (Maybe I only have seen such tool in
> my
> > > company applications)
> >
> > It may already exist, I haven't checked.
> >
> > -Ivan
> >
>



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


Re: BookKeeper code coverage incredibly dropped in June from 72% to 54% !!

2018-07-24 Thread Venkateswara Rao Jujjuri
I don't see bookkeeper-server in the https://coveralls.io/builds/17501203
report.  Rest looks almost identical.
That must be causing the overall drop.

On Tue, Jul 24, 2018 at 8:20 AM, Enrico Olivelli 
wrote:

> Hi,
> In June, the 14th, code coverage dropped
>
> here: 72%
> https://coveralls.io/builds/17447847
>
> here: 54%
> https://coveralls.io/builds/17501203
>
> This commit seems the cause (as reported by Coveralls.io)
> https://github.com/apache/bookkeeper/commit/c5a36219bbb7cfce343137bf3c361f
> 7eb80bd64e
>
> Anyone has cycles to take a look ?
>
>
> Enrico
>



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


Re: [DISCUSS] ExplicitLAC on the reader's side

2018-07-10 Thread Venkateswara Rao Jujjuri
+1

On Tue, Jul 10, 2018 at 1:11 AM, Enrico Olivelli 
wrote:

> Hi,
> Now that Charan is working on making ExplicitLAC persistent I would like to
> start the discussion about merging the reader side view of LAC (ExplicitLAC
> + Pibbybacked/regular LAC)
>
> Major points:
> - Now (4.8) ExplicitLAC is to be read with LedgerHandle#readExplicitLAC
> - ExplicitLAC is stored on LedgerStorage
> - Regular LAC is stored inside each entry
> - We have to support Long Poll
> - We have to give access to ExplicitLAC on the new API (but I don't want to
> add a readExplicitLAC on the new API)
> - ExplicitLAC on the write side is only a matter of configuring the
> scheduler, no changes are to be done on application code
>
> Desiderata (from me):
>
> The reader only uses Long-Poll or ReadHandle#readLastAddConfirmed and
> transparently it receives the most accurate information from ExplicitLAC
> and PiggyBacked LAC
>
> In order to achieve this goal we will have to work on many parts, this is
> my list:
> - add current ExplicitLAC on readResponses
> - merge ExplicitLAC and regular LAC on client side
> - support watches for Long-Poll on Bookie side, to be triggered by
> ExplicitLAC
>
>
> Enrico
>



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


Re: Bookie disappears from ZK but continues to be up and running - shall we add a self check ?

2018-06-01 Thread Venkateswara Rao Jujjuri
@Enrico
Let me understand the issue: Bookies are up and running but ZK doesn't show
bookies on the list.

Do you see if session expired or not? or bookies are hung in some way?

We have seen multiple situations in this area:
1. Bookie process is around, but zk session lost. Alost all the time we ran
into this when bookie is hung in some sense.
2. Bookies are down, but ZK still shows the Bookie in RW list. We suspect
this is a ZK bug which got fixed in later releases.
3. Bookies are hug at disk, but was able to keep up ZK session.

Does your case fit into any of these situations? Or do you believe that the
Bookie is healthy and up but lost ZK session?
If so how did you validate that your Bookie is healthy.?

JV


On Fri, Jun 1, 2018 at 12:11 AM, Enrico Olivelli 
wrote:

> Il ven 1 giu 2018, 08:49 Sijie Guo  ha scritto:
>
> > I don't think there is any zk changes between 4.6.2 and 4.7.0. Are you
> sure
> > the upgrade fixes the problem?
> >
>
> I have checked several times and it seems to me that every zk fix in 4.7.0
> has been cherry picked to 4.6.2.
> It is only a fact that with the upgrade the issue does not appear. Maybe it
> is too early to say that it is working.
>
> I will send news
>
> Enrico
>
>
>
> > - Sijie
> >
> > On Thu, May 31, 2018 at 11:30 PM, Enrico Olivelli 
> > wrote:
> >
> > > Seems that al the sites which are reporting this kind of problems are
> > ONLY
> > > on 4.6.2.
> > >
> > > After an upgrade to 4.7.0 apparently the problem disappears.
> > >
> > > I will send news next week
> > >
> > > Enrico
> > >
> > > Il dom 20 mag 2018, 19:18 Enrico Olivelli  ha
> > > scritto:
> > >
> > > > My guess is that is about using zk ACLs
> > > > I have no evidence
> > > >
> > > > Enrico
> > > >
> > > >
> > > > Il mar 15 mag 2018, 14:09 Enrico Olivelli  ha
> > > > scritto:
> > > >
> > > >> Il giorno mar 15 mag 2018 alle ore 14:04 Sijie Guo <
> > guosi...@gmail.com>
> > > >> ha scritto:
> > > >>
> > > >>> On Tue, May 15, 2018 at 4:45 AM, Enrico Olivelli <
> > eolive...@gmail.com>
> > > >>> wrote:
> > > >>>
> > > >>> > Hi,
> > > >>> > it is quite some time that we are seeing Bookies in staging
> > > >>> environments
> > > >>> > which disappear from ZK but appartently are still up and running.
> > > >>> >
> > > >>> > I have not dug deeply into this problem, but at first glance it
> > > should
> > > >>> be
> > > >>> > related to ZK session expiration, those machines are heavily
> loaded
> > > >>> > sometimes and it is not surprising that ZK session expires.
> > > >>> >
> > > >>>
> > > >>> There should be already a logic on re-registration after session
> > > expired,
> > > >>> no?
> > > >>>
> > > >>
> > > >> Yes. The fact is that in the month we are seeing this strange
> > behaviour.
> > > >> I don't know if it could be a regression on 4.7.
> > > >> I have no reports from production sites, but in production we have
> > > >> dedicated machines for bookies.
> > > >>
> > > >>
> > > >>>
> > > >>> ZooKeeper stats should always show whether a bookie is able to
> > connect
> > > to
> > > >>> zookeeper. That would probably tell you what happens.
> > > >>>
> > > >>
> > > >> I will check, thank you for your suggestion.
> > > >>
> > > >> Enrico
> > > >>
> > > >>
> > > >>
> > > >>>
> > > >>>
> > > >>> >
> > > >>> > Apart from searching for a bug, I wonder if would it be useful an
> > > >>> automatic
> > > >>> > self check of the bookie, something like a periodic check which
> > asks
> > > >>> to the
> > > >>> > Registration Manager if the bookie is listed in the expected
> bookie
> > > >>> list
> > > >>> > (readonly/available)
> > > >>> >
> > > >>> > This will be useful even if we are not using ZK as well, now that
> > we
> > > >>> have
> > > >>> > this great abstraction of ZK
> > > >>> >
> > > >>> > Thoughts ?
> > > >>> >
> > > >>> > Enrico
> > > >>> >
> > > >>>
> > > >> --
> > > >
> > > >
> > > > -- Enrico Olivelli
> > > >
> > > --
> > >
> > >
> > > -- Enrico Olivelli
> > >
> >
> --
>
>
> -- Enrico Olivelli
>



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


Re: Scanning the list of entries present on a bookie

2018-04-22 Thread Venkateswara Rao Jujjuri
Apparently, there is an assumption in this computation that the bookie sees
entries in sequence.
When ES>WQ bookie won't see them in sequence. I guess I need to understand
what you are trying to
do with in-memory cursor at the bookie. In our protocol, only the
client(writer) will have the complete view and
bookie view is persisted in the index files. If you want to know what is
persisted in the journal, then you may need
to maintain some in-memory value but that is just one value isn't it? It
could be I don't fully understand what is the intended change
here, maybe let's have a quick call or discuss at next Thursday meeting?

On Sat, Apr 21, 2018 at 12:37 AM, Enrico Olivelli <eolive...@gmail.com>
wrote:

> Il sab 21 apr 2018, 09:11 Venkateswara Rao Jujjuri <jujj...@gmail.com> ha
> scritto:
>
> > Sorry, maxEntryId can't be used for LAC.  Let me review the patch and
> > respond tomorrow. :)
> > I was mostly thinking of sync-on-close which is my use case. Intermittent
> > syncs will need a more robust way to define the LAC.
> >
>
>
> Thank you JV,
> I think that about LAC updates we are already on our way as discussed on
> BP14 and there is already my patch about client side LAC management (not
> mergeable untill we merge server side changes).
>
> My problem is about reconstructing the knowledge of which entries are
> stored on the bookie after a restart.
> In my current impl I am keeping an in memory 'cursor' which tracks which
> entries are on bookie (fsync'd to journal), but after a restart this cursor
> is to be recreated.
> We need this cursor because we can have temporary holes in the sequence of
> entries, think about WriteAdvHandle.
> LAC does not advance in presence of holes, and in particular it does not
> advance without fsync.
>
> Even without ensemble changes a bookie can restart without breaking a
> writer, for instance in the case of no writes during the restart.
>
> Enrico
>
>
> > On Sat, Apr 21, 2018 at 12:00 AM, Venkateswara Rao Jujjuri <
> > jujj...@gmail.com> wrote:
> >
> > > @Enrico I thought I had a pretty simple view on this.
> > > 1. No ensemble changes allowed on a ledger in this mode.
> > > 2. A sync is sent to all bookies on the ensemble and it is required
> that
> > > the sync must be successful to proceed forward.
> > > 3. Sync and Write are serialized. or sync can return maxEntryId.
> > > 4. If we get a successful sync from all bookies, then the ' smallest
> > > maxEntryId ' returned will be the LAC.
> > > 5. An implicit sync is sent on close.
> > >
> > > IMO,  no ensemble change will make the whole thing simple and avoids
> > scans
> > > from client side.
> > >
> > >
> > > On Wed, Apr 18, 2018 at 2:02 AM, Enrico Olivelli <eolive...@gmail.com>
> > > wrote:
> > >
> > >> Any other suggestion? JV, Sijie?
> > >> My current idea for bp14 is to have a client driven scan.
> > >>
> > >>
> > >>
> > >> Enrico
> > >>
> > >> Il ven 13 apr 2018, 10:22 Enrico Olivelli <eolive...@gmail.com> ha
> > >> scritto:
> > >>
> > >> >
> > >> >
> > >> > Il gio 12 apr 2018, 19:59 Sam Just <sj...@salesforce.com> ha
> scritto:
> > >> >
> > >> >> IIRC, InterleavedLedgerStorage has for each ledger an index file
> > >> >> mapping the entries to entry logger offsets, you could probably
> scan
> > >> >> that directly (particularly if you included a lower ?bound --
> > probably
> > >> >> the client's current idea of the LAC).
> > >> >> -Sam
> > >> >>
> > >> >
> > >> > Thank you Sam,
> > >> > I think that the major point is about introducing a standard
> interface
> > >> to
> > >> > be implemented by LedgerStorage, now we already have 3
> > implementations.
> > >> >
> > >> > Options are:
> > >> > 1) let the client drive the scan
> > >> > 2) add a scan() method to LedgerStorage
> > >> > 3) add a getMaxEntry() to LedgerStorage
> > >> >
> > >> > For bp14 I just need 1) or 3) but 1) is also a protocol issue and so
> > it
> > >> is
> > >> > more expensive for long term maintenance
> > >> >
> > >> > Enrico
> > >> >
> > >> >>
> > >> >> On Thu, Apr 12, 2018 at 12:31 AM, Enrico Olivelli <
> > eolive...@gmail.com
&

Re: Scanning the list of entries present on a bookie

2018-04-21 Thread Venkateswara Rao Jujjuri
Sorry, maxEntryId can't be used for LAC.  Let me review the patch and
respond tomorrow. :)
I was mostly thinking of sync-on-close which is my use case. Intermittent
syncs will need a more robust way to define the LAC.

On Sat, Apr 21, 2018 at 12:00 AM, Venkateswara Rao Jujjuri <
jujj...@gmail.com> wrote:

> @Enrico I thought I had a pretty simple view on this.
> 1. No ensemble changes allowed on a ledger in this mode.
> 2. A sync is sent to all bookies on the ensemble and it is required that
> the sync must be successful to proceed forward.
> 3. Sync and Write are serialized. or sync can return maxEntryId.
> 4. If we get a successful sync from all bookies, then the ' smallest
> maxEntryId ' returned will be the LAC.
> 5. An implicit sync is sent on close.
>
> IMO,  no ensemble change will make the whole thing simple and avoids scans
> from client side.
>
>
> On Wed, Apr 18, 2018 at 2:02 AM, Enrico Olivelli <eolive...@gmail.com>
> wrote:
>
>> Any other suggestion? JV, Sijie?
>> My current idea for bp14 is to have a client driven scan.
>>
>>
>>
>> Enrico
>>
>> Il ven 13 apr 2018, 10:22 Enrico Olivelli <eolive...@gmail.com> ha
>> scritto:
>>
>> >
>> >
>> > Il gio 12 apr 2018, 19:59 Sam Just <sj...@salesforce.com> ha scritto:
>> >
>> >> IIRC, InterleavedLedgerStorage has for each ledger an index file
>> >> mapping the entries to entry logger offsets, you could probably scan
>> >> that directly (particularly if you included a lower ?bound -- probably
>> >> the client's current idea of the LAC).
>> >> -Sam
>> >>
>> >
>> > Thank you Sam,
>> > I think that the major point is about introducing a standard interface
>> to
>> > be implemented by LedgerStorage, now we already have 3 implementations.
>> >
>> > Options are:
>> > 1) let the client drive the scan
>> > 2) add a scan() method to LedgerStorage
>> > 3) add a getMaxEntry() to LedgerStorage
>> >
>> > For bp14 I just need 1) or 3) but 1) is also a protocol issue and so it
>> is
>> > more expensive for long term maintenance
>> >
>> > Enrico
>> >
>> >>
>> >> On Thu, Apr 12, 2018 at 12:31 AM, Enrico Olivelli <eolive...@gmail.com
>> >
>> >> wrote:
>> >> > Hi BookKeepers,
>> >> > during implementation of BP-14 I am facing a problem so I am asking
>> for
>> >> > suggestions.
>> >> >
>> >> > My need is to be able to know the list of all entries stored on a
>> >> > LedgerStorage given a ledgerId.
>> >> >
>> >> > Scanning from 0 to LedgerStorage#getLastAddConfirmed() does not
>> seem to
>> >> > work because we have to deal with WriteAdvHandle, so there can be
>> >> temporary
>> >> > "gaps" in the sequence of entries.
>> >> >
>> >> > I can have a writer which writes entries 0,1,5,6,7. Its LAC will be
>> at
>> >> most
>> >> > 1 as entries 2,3,4 are not written yet.
>> >> > I need on the bookie to able to know that entries 0, 1, 5, 6, 7 are
>> >> stored
>> >> > on LedgerStorage.
>> >> >
>> >> > I cannot issue a scan from 0 to Long.MAX_VALUE, my current 'solution'
>> >> it to
>> >> > make the client (writer) send the 'maximum entry id' and perform a
>> scan
>> >> > from 0 to maxEntryId.
>> >> > In the example the writer will send a forceLedger RPC with
>> maxEntryId =
>> >> 7.
>> >> >
>> >> > This is need only for recoveries are bookie restart because I have to
>> >> > reconstruct the knowledge about which entries have been persisted
>> >> durably
>> >> > on the Bookie.
>> >> >
>> >> > I am not very expert about LedgerStorage implementations, and I don't
>> >> know
>> >> > if it would be feasible to have such 'scan all entries' method.
>> >> >
>> >> > This is the code I am talking about
>> >> >
>> >> https://github.com/apache/bookkeeper/pull/1317/files#diff-3b
>> 81b1c90d1f51017627b3c032676168R1210
>> >> >
>> >> > Any help is really appreciated
>> >> > Enrico
>> >>
>> > --
>> >
>> >
>> > -- 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


Re: Scanning the list of entries present on a bookie

2018-04-21 Thread Venkateswara Rao Jujjuri
@Enrico I thought I had a pretty simple view on this.
1. No ensemble changes allowed on a ledger in this mode.
2. A sync is sent to all bookies on the ensemble and it is required that
the sync must be successful to proceed forward.
3. Sync and Write are serialized. or sync can return maxEntryId.
4. If we get a successful sync from all bookies, then the ' smallest
maxEntryId ' returned will be the LAC.
5. An implicit sync is sent on close.

IMO,  no ensemble change will make the whole thing simple and avoids scans
from client side.


On Wed, Apr 18, 2018 at 2:02 AM, Enrico Olivelli 
wrote:

> Any other suggestion? JV, Sijie?
> My current idea for bp14 is to have a client driven scan.
>
>
>
> Enrico
>
> Il ven 13 apr 2018, 10:22 Enrico Olivelli  ha
> scritto:
>
> >
> >
> > Il gio 12 apr 2018, 19:59 Sam Just  ha scritto:
> >
> >> IIRC, InterleavedLedgerStorage has for each ledger an index file
> >> mapping the entries to entry logger offsets, you could probably scan
> >> that directly (particularly if you included a lower ?bound -- probably
> >> the client's current idea of the LAC).
> >> -Sam
> >>
> >
> > Thank you Sam,
> > I think that the major point is about introducing a standard interface to
> > be implemented by LedgerStorage, now we already have 3 implementations.
> >
> > Options are:
> > 1) let the client drive the scan
> > 2) add a scan() method to LedgerStorage
> > 3) add a getMaxEntry() to LedgerStorage
> >
> > For bp14 I just need 1) or 3) but 1) is also a protocol issue and so it
> is
> > more expensive for long term maintenance
> >
> > Enrico
> >
> >>
> >> On Thu, Apr 12, 2018 at 12:31 AM, Enrico Olivelli 
> >> wrote:
> >> > Hi BookKeepers,
> >> > during implementation of BP-14 I am facing a problem so I am asking
> for
> >> > suggestions.
> >> >
> >> > My need is to be able to know the list of all entries stored on a
> >> > LedgerStorage given a ledgerId.
> >> >
> >> > Scanning from 0 to LedgerStorage#getLastAddConfirmed() does not seem
> to
> >> > work because we have to deal with WriteAdvHandle, so there can be
> >> temporary
> >> > "gaps" in the sequence of entries.
> >> >
> >> > I can have a writer which writes entries 0,1,5,6,7. Its LAC will be at
> >> most
> >> > 1 as entries 2,3,4 are not written yet.
> >> > I need on the bookie to able to know that entries 0, 1, 5, 6, 7 are
> >> stored
> >> > on LedgerStorage.
> >> >
> >> > I cannot issue a scan from 0 to Long.MAX_VALUE, my current 'solution'
> >> it to
> >> > make the client (writer) send the 'maximum entry id' and perform a
> scan
> >> > from 0 to maxEntryId.
> >> > In the example the writer will send a forceLedger RPC with maxEntryId
> =
> >> 7.
> >> >
> >> > This is need only for recoveries are bookie restart because I have to
> >> > reconstruct the knowledge about which entries have been persisted
> >> durably
> >> > on the Bookie.
> >> >
> >> > I am not very expert about LedgerStorage implementations, and I don't
> >> know
> >> > if it would be feasible to have such 'scan all entries' method.
> >> >
> >> > This is the code I am talking about
> >> >
> >> https://github.com/apache/bookkeeper/pull/1317/files#diff-
> 3b81b1c90d1f51017627b3c032676168R1210
> >> >
> >> > Any help is really appreciated
> >> > Enrico
> >>
> > --
> >
> >
> > -- Enrico Olivelli
> >
> --
>
>
> -- Enrico Olivelli
>



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


Re: Replication Worker and targetBookie.

2018-03-08 Thread Venkateswara Rao Jujjuri
On Thu, Mar 8, 2018 at 11:33 AM, Sijie Guo <guosi...@gmail.com> wrote:

> On Thu, Mar 8, 2018 at 8:07 AM, Venkateswara Rao Jujjuri <
> jujj...@gmail.com>
> wrote:
>
> > On Thu, Mar 8, 2018 at 2:38 AM, Ivan Kelly <iv...@apache.org> wrote:
> >
> > > > Given that RackAwareEnsemble policy defaults to finding a replacement
> > > > bookie within
> > > > the same rack, when a bookie is lost in a rack, the entire cluster
> will
> > > be
> > > > replicating
> > > > to the same 'rack'. This puts a lot of pressure on the rack and also
> > > takes
> > > > a longer time
> > > > to bring up the replication levels.
> > >
> > > I agree this has potential to be problematic.
> > >
> > > Perhaps we should provide a switch to RackAwareEnsemble,
> > > 'preferReplaceInSameRack'.
> > >
> > > > I would think the right fix is to bring back the targetBookie concept
> > > (with
> > > > a configuration parameter) and add placement check predicate on top
> of
> > > it.
> > > > When this is configured
> > > > each bookie picks up the work,  checks if the ensemble placement
> policy
> > > > gets satisfied,
> > > > if so replicate it, if not move on.
> > >
> > > I don't think adding a predicate argument (I guess a
> > > BiPredicate<Set, BookieSocketAddress>?) to the
> > > recover bookie call makes sense. There is already a way to customize
> > > this behaviour, by passing in a EnsemblePlacementPolicy on
> > > Configuration of the client. The behaviour you want can be achieved by
> > > taking one of the current EnsemblePlacementPolicies and overriding
> > > replaceBookie, though I guess that's not very user-friendly. However,
> > > even if it was user-friendly, how would we make it easy for users to
> > > supply a placementpolicy or a even a predicate, as you suggested, to
> > > the autorecovery daemon.
> > >
> >
> > In the old model if the bookie is writable
> > AND is not part of ensemble, replicate to the local(target) bookie.
> > My proposal is t add anotehr AND condition.
> >
> > if bookie is writable AND not part of ensemble AND satisfies Enseble
> > Placement Policy
> > write to local(target) bookie.
> >
>
>
> I think this is a good change to take. but we need to differentiate things:
>
> 1) if bookie recovery is running a separate daemon, we don't need to do any
> changes here.
>

Why do you say so?

replaceBookie() -> selectFromNetworkLocation always tries to replace

the bookie from "bookieToReplace" rack. Goes to random/Anywhere from

the cluster on BKNotEnoughBookiesException.

This has two issues.

1. One rack gets the entire responsibility (write) to replace lost bookie.

2. If no space on this rack, we are going random on the cluster not
honoring the placmeent (This is a generic problem)



> 2) if bookie recovery is running along with the bookie, we construct a
> ensemble placement policy which wrap over the rack-aware/region-aware one
> and override the predicate to take local bookie into account.
>
>
> However there is a problem which this "predicate" model, because it can
> potentially churn the metadata storage, since now some bookies are actually
> doing nothing but polling the ur replication ledger list. so a long term
> direction is to change how auditor distributes replication tasks to
> replication workers.
>

Generally it is a watch on the underreplicated znode right?

JV

>
>
>
> >
> > Thanks,
> > JV
> >
> >
> > > -Ivan
> > >
> >
> >
> >
> > --
> > 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


Re: Replication Worker and targetBookie.

2018-03-08 Thread Venkateswara Rao Jujjuri
On Thu, Mar 8, 2018 at 2:38 AM, Ivan Kelly  wrote:

> > Given that RackAwareEnsemble policy defaults to finding a replacement
> > bookie within
> > the same rack, when a bookie is lost in a rack, the entire cluster will
> be
> > replicating
> > to the same 'rack'. This puts a lot of pressure on the rack and also
> takes
> > a longer time
> > to bring up the replication levels.
>
> I agree this has potential to be problematic.
>
> Perhaps we should provide a switch to RackAwareEnsemble,
> 'preferReplaceInSameRack'.
>
> > I would think the right fix is to bring back the targetBookie concept
> (with
> > a configuration parameter) and add placement check predicate on top of
> it.
> > When this is configured
> > each bookie picks up the work,  checks if the ensemble placement policy
> > gets satisfied,
> > if so replicate it, if not move on.
>
> I don't think adding a predicate argument (I guess a
> BiPredicate?) to the
> recover bookie call makes sense. There is already a way to customize
> this behaviour, by passing in a EnsemblePlacementPolicy on
> Configuration of the client. The behaviour you want can be achieved by
> taking one of the current EnsemblePlacementPolicies and overriding
> replaceBookie, though I guess that's not very user-friendly. However,
> even if it was user-friendly, how would we make it easy for users to
> supply a placementpolicy or a even a predicate, as you suggested, to
> the autorecovery daemon.
>

In the old model if the bookie is writable
AND is not part of ensemble, replicate to the local(target) bookie.
My proposal is t add anotehr AND condition.

if bookie is writable AND not part of ensemble AND satisfies Enseble
Placement Policy
write to local(target) bookie.

Thanks,
JV


> -Ivan
>



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


Replication Worker and targetBookie.

2018-03-07 Thread Venkateswara Rao Jujjuri
With the fixes to ISSUE #596 ISSUE
#583(6fcabfc80c5bd6cdfa9252cdfc4ed209cc0620a6)
which allowed auto replication should honor ensemble placement policy,
this patch also removed `targetBookie` from ReplicationWorker which means
any node can replicate 'to' any other node in the cluster.

This is great but I believe it exposes an issue.

Given that RackAwareEnsemble policy defaults to finding a replacement
bookie within
the same rack, when a bookie is lost in a rack, the entire cluster will be
replicating
to the same 'rack'. This puts a lot of pressure on the rack and also takes
a longer time
to bring up the replication levels.

I would think the right fix is to bring back the targetBookie concept (with
a configuration parameter) and add placement check predicate on top of it.
When this is configured
each bookie picks up the work,  checks if the ensemble placement policy
gets satisfied,
if so replicate it, if not move on.

Comments?

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


Re: BookKeeper Durability issues/proposals

2018-02-16 Thread Venkateswara Rao Jujjuri
Durability is perfect. Thanks.

On Fri, Feb 16, 2018 at 9:35 AM, Sijie Guo <guosi...@gmail.com> wrote:

> JV,
>
> I added a label "durability" (not sure if it is a good name or not, feel
> free to propose a better label name) for categorizing the issues created
> for BP-31 on addressing durability issues.
>
> - Sijie
>
> On Thu, Jan 18, 2018 at 7:52 AM, Sijie Guo <guosi...@gmail.com> wrote:
>
> > Thanks JV.
> >
> > Comments are left in the google doc.
> >
> > Let's continue the discussion on the google doc. Create the corresponding
> > BPs/issues for individual problems, so that we can move this stuff
> forward.
> >
> > - Sijie
> >
> > On Tue, Jan 16, 2018 at 12:33 PM, Venkateswara Rao Jujjuri <
> > jujj...@gmail.com> wrote:
> >
> >> Here is the document I wrote on the durability.
> >> Please review/comment. We can crate issues based on this.
> >>
> >> https://docs.google.com/document/d/1LYZSiJ2x5mXMOjQqMWU7ZBRJ
> >> D8YVDkB-hRPZ474O7Qw/edit?usp=sharing
> >>
> >>
> >>
> >> --
> >> 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


Re: BP-30: table service as a contrib module in bookkeeper

2018-02-07 Thread Venkateswara Rao Jujjuri
A great step to move forward. BP-29 and BP-30 along with reorganizing ZK
will help the BK to shape perfect MDS abstraction.
While BP-30 is ambitious, it is a perfect way to start ambitious projects.
:)

JV

On Wed, Feb 7, 2018 at 6:49 AM, Enrico Olivelli  wrote:

> It is very interesting! Thank you.
> I will look into it soon
>
> Enrico
>
> Il mer 7 feb 2018, 15:24 Sijie Guo  ha scritto:
>
> > Hi all,
> >
> > I started a proposal of contributing a table (aka key/value) service
> > component as a contrib module to the bookkeeper community. This BP
> together
> > with other BPs I sent last week forms the idea of how we can do on
> > improving metadata management in bookkeeper (I will talk a bit more at
> the
> > end of this email).
> >
> > **why it was developed**
> >
> > Two main categories of use cases were driving the need of a key/value
> like
> > service.
> >
> > One is metadata storage, bookkeeper needs a key/value like storage
> > (currently it is zookeeper) to store the ledger's metadata, systems built
> > on top of bookkeeper like distributedlog/pulsar also follow the pattern
> > that bookkeeper is using. They all need a key/value like storage to store
> > their metadata. We all know zookeeper is the bottleneck of the
> scalability.
> > And it is also an issue marker to production systems (based on my biased
> > production experiences).
> >
> > The other one is state storage in real-time/streaming
> > analytics/computation. In streaming analytics, the computation jobs
> usually
> > process streaming data. they usually need to store some sort of state of
> > the computation operators into a storage and serve the computation state
> as
> > final results for queries. Those state are usually represented in
> key/value
> > forms, and usually backed by wal. BookKeeper has been used in this area
> via
> > distributedlog/pulsar for storing and serving log / streaming data. It is
> > ideal for bookkeeper also able to store and serve state data for the sake
> > of unification, simplification and also reducing the complexity of
> > deployment and operations.
> >
> > Hence we prototyped/developed a table service component as an add-on to
> > bookkeeper. We'd like to contribute this as a contrib module to
> bookkeeper
> > and continue the development, integration and evaluation in the
> bookkeeper
> > community.
> >
> > We hope this can be like bookkeeper in zookeeper. bookkeeper was a
> contrib
> > module in zookeeper, and it is developed in the community and grown into
> > what it is now.
> >
> > **how it is aligned with metadata storage**
> >
> > BP-28, BP-29 and BP-30. They are related at some extend.
> >
> > BP-28 is more a cleanup proposal to carry-on Jia's work (on service
> > discovery interfaces). This is to produce a clean metadata api module,
> > define a clean dependency between
> > bookkeeper implementation and metadata service, and allow we really
> plugin
> > different
> > metadata services without touching/changing bookkeeper implementation.
> >
> > BP-29 and BP-30 can be thought as two different metadata service
> > implementation based
> > on the metadata api contract defined in BP-28.
> >
> > BP-29 is to use Etcd as the metadata service, while BP-30 is to have a
> > built-in key/value service as the metadata service. Both BP-29 and BP-30
> > have pros and cons. However they
> > are not against to each other. Allowing two concurrent approaches will
> help
> > us understand
> > more on metadata management in bookkeeper and its ecosystem (e.g. dlog,
> > pulsar), which
> > will lead the project head in a healthy direction.
> >
> > **Proposed Changes**
> >
> > This proposal is to propose this table service as a contrib module under
> > `stream` directory just as how we handle `dlog`. We can mark it as
> > "preview"/"alpha" in 4.7 and continue the development of this module in
> > bookkeeper community.
> >
> > The details of the proposal can be found in the google doc attached
> below:
> >
> >
> > https://docs.google.com/document/d/155xAwWv5IdOitHh1NVMEwCMGgB28M
> 3FyMiQSxEpjE-Y/edit#heading=h.56rbh52koe3f
> >
> > Please take a look. Comments are welcome.
> >
> > - Sijie
> >
>
>
> --
>
>
> -- Enrico Olivelli
>



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


MetadataVersionException handling in doAsyncCloseInternal()

2018-02-06 Thread Venkateswara Rao Jujjuri
We have bunch of code in doAsyncCloseInternal() to handle
MetadataVersionException.
I am wondering if there is a valid case where we need to handle this. Stay
with me. :)

When do we get this error?

1. When the client tried to write the metadata twice. Same metadata twice
because of the client tried twice and the first attempt made it. (retry
logic in bk). If so, the error is benign. (Though I have a question on
metadata.isNewerThan(newMeta) logic which I will ask towards the end.)

2. Close might be happening for two reasons. 1. Application initiated 2.
Internal initiated because of a hard error on the write path. We really
don't need to fail with version mismatch for internal closes, as it really
doesn't matter, because our intention is to fail the write and we would
anyway. Just close the local handle we are done.
In the case of the application initiated close, the version mismatch can
happen only if the fencing logic took over and changed some metadata. But
we are guaranteed to protected up until the LAC  of the writer anyway.
Since the leger is getting closed, it really doesn't matter if the ZK has
more updated metadata version.

So my question is, do we really need to handle MetadataVersionException on
write?

 My next question is do we have anything fishy in the isNewerThan logic?

boolean isNewerThan(LedgerMetadata newMeta) {
if (null == version) {
return false;
}
return Version.Occurred.AFTER == version.compare(newMeta.version);
}
---
MetadataVersion mv = (MetadataVersion) v;
int res = version - mv.version;
if (res == 0) {
return Occurred.CONCURRENTLY;
} else if (res < 0) {
return Occurred.BEFORE;
} else {
return Occurred.AFTER;
}
---

Since newMetadata is newer in this case, its version is going to be higher
hence res is -ve, and this is returning  Occurred.BEFORE, but it must be
Occurred.AFTER as the metadata update happened 'after' what we have. Or the
isNewerThan logic is inverted.

if (!metadata.isNewerThan(newMeta)
 && !metadata.isConflictWith(newMeta)) {
} else {
LOG.warn("Conditional update ledger metadata for ledger {} failed.",
ledgerId);
cb.closeComplete(rc,
LedgerHandle.this, ctx);
}

In this case, even though local metadata is NOT newer than newMeta because
of the logic we are failing.


So two questions:
1. Do we really need to handle MetadataVersionMismatch in close?
2. Is this metadata.isNewerTha() logic is corretc?

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


Re: BP-28: Etcd as metadata store

2018-02-02 Thread Venkateswara Rao Jujjuri
Thanks for pushing this Sijie. I had a quick glance at it and looks like
this is a stand-alone/place-holder code, and not connected yet to kick
tires.
I was under the impression you did have something we can hook up and play
with. I could be wrong. :)
Not that its a big deal to build that connection and start playing with,
just checking.
When you originally developed did you u actually played with etcd as MDS ?
any testing?

Again thanks a lot for pushing your work. This can be a great start.

On Fri, Feb 2, 2018 at 3:52 AM, Sijie Guo  wrote:

> Hi all,
>
> JV and me had a chat today about metadata storage. As we discussed, I am
> raising a BP for exploring using Etcd as the metadata store and pushing a
> prototype change (which I did that one year and half ago).
>
> The prototype branch was just for showing how the metadata interfaces can
> be implemented using jetcd api. It only implements some of operations that
> required attentions, such as how do we do service discovery, ledger id
> generation, cas update/delete and iteration / get a list of bookies.
>
> I would like someone from the community (maybe from salesforce) can pickup
> the prototype and push it forward. so he can play closer with etcd api and
> know better how ectd will perform as a metadata store comparing to
> zookeeper.
>
> Please take a look and comments are welcome.
>
> BP: https://github.com/apache/bookkeeper/pull/1113
> Prototype: https://github.com/apache/bookkeeper/pull/1114
>
> - Sijie
>



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


Re: [DISCUSS] Deprecate FlatLedgerManager

2018-02-01 Thread Venkateswara Rao Jujjuri
+1

On Thu, Feb 1, 2018 at 12:20 PM, Ivan Kelly  wrote:

> +1
>
> Flat is an vestigial. Prereq to removal is a tool for upgrading ledger
> metadata.
>
> -Ivan
>
> On Thu, Feb 1, 2018 at 8:14 PM, Sijie Guo  wrote:
> > Hi all,
> >
> > `FlatLedgerManager` is not useful at production and most of the
> production
> > environment I knew don't use `FlatLedgerManager`. But the default value
> in
> > the configuration is still setting to `FlatLedgerManager`, which doesn't
> > sound right for the new users. so I think it is the time to raise the
> > discussion of deprecating `FlatLedgerManager` and make
> > `HierarchicalLedgerManager` as the default.
> >
> > I created an issue for tracking this. Please comment if you see any
> problem
> > of making `HierarchicalLedgerManager` as the default.
> >
> > https://github.com/apache/bookkeeper/issues/1101
> >
> > - Sijie
>



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


Re: [ANNOUNCE] Apache BookKeeper 4.6.1 released

2018-01-31 Thread Venkateswara Rao Jujjuri
Awesome work Enrico! Thanks a lot!!

On Wed, Jan 31, 2018 at 12:59 PM, Ivan Kelly  wrote:

> Thanks for all your work in getting this out Enrico!
>
> On Wed, Jan 31, 2018 at 9:38 AM, Sijie Guo  wrote:
> > Congrats Enrico! Thank you so much for driving 4.6.1 release.
> >
> > Thanks everyone for making contributions to bookkeeper and making this
> > happen!
> >
> > - Sijie
> >
> > On Wed, Jan 31, 2018 at 12:04 AM, Enrico Olivelli 
> > wrote:
> >>
> >> The Apache BookKeeper team is proud to announce Apache BookKeeper
> version
> >> 4.6.1.
> >>
> >> Apache BookKeeper is a scalable, fault-tolerant, and low-latency storage
> >> service optimized for
> >> real-time workloads. It has been used for a fundamental service to build
> >> reliable services.
> >> It is also the log segment store for Apache DistributedLog and the
> message
> >> store for Apache Pulsar.
> >>
> >> This is the 8th release of the Apache BookKeeper.
> >>
> >> This is a bugfix release, it fixes bugs around Index Persistence Manager
> >> and provides enhancements about Java 9 support.
> >>
> >> For BookKeeper release details and downloads, visit:
> >>
> >> http://www.apache.org/dyn/closer.cgi/bookkeeper
> >>
> >> BookKeeper 4.6.1 Release Notes are at:
> >>
> >> https://bookkeeper.apache.org/docs/4.6.1/overview/releaseNotes/
> >>
> >> We would like to thank the contributors that made the release possible.
> >>
> >> Regards,
> >>
> >> The BookKeeper Team
> >
> >
>



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


Re: [VOTE] Release 4.6.1, release candidate #0

2018-01-28 Thread Venkateswara Rao Jujjuri
+1
Release notes is good
checksums are good
source and binary pages are good.

On Sun, Jan 28, 2018 at 2:50 PM, Yiming Zang 
wrote:

> +1
>
> - MD5 SHA signature looks good
> - Source code looks good
> - Binary looks good
>
> On Sun, Jan 28, 2018 at 6:59 AM, David Rusek  wrote:
>
> > I'll take a look first thing tomorrow morning.
> >
> > On Sun, Jan 28, 2018 at 3:23 AM Enrico Olivelli 
> > wrote:
> >
> > > Thank you all,
> > > We need another PMC +1
> > >
> > > Enrico
> > >
> > > Il ven 26 gen 2018, 17:42 Ivan Kelly  ha scritto:
> > >
> > > > +1
> > > >
> > > > - sha1, md5, gpg sigs are good
> > > > - NOTICE and LICENSE are "good"
> > > > - src passes rat, spotbugs, unit tests pass eventually
> > > > - binary package and maven repo jar passed a run through Jepsen
> > > >
> > > > Good work Enrico! And sorry about the delay in voting.
> > > >
> > > > -Ivan
> > > >
> > > > On Wed, Jan 24, 2018 at 8:52 PM, Sijie Guo 
> wrote:
> > > > > +1
> > > > >
> > > > > - package checksums are good
> > > > > - source package is good
> > > > > - binary package is good
> > > > >
> > > > > - Sijie
> > > > >
> > > > > On Tue, Jan 23, 2018 at 11:16 PM, Enrico Olivelli <
> > eolive...@gmail.com
> > > >
> > > > > wrote:
> > > > >
> > > > >> Thank you Jia,
> > > > >>
> > > > >> We need at least 3 PMC votes...
> > > > >>
> > > > >> Enrico
> > > > >>
> > > > >> Il sab 20 gen 2018, 16:32 Jia Zhai  ha
> > scritto:
> > > > >>
> > > > >> > +1,
> > > > >> >
> > > > >> > - verified packages checksum (md5, asc and sha1 all good)
> > > > >> >
> > > > >> > - the source package build and test all run successfully.
> > > > >> >
> > > > >> > - NOTICE, License headers in binary package look good.
> > > > >> >
> > > > >> > - in binary package, bin/bookkeeper and command localbookie runs
> > > well.
> > > > >> >
> > > > >> > On Fri, Jan 19, 2018 at 9:58 PM, Enrico Olivelli <
> > > eolive...@gmail.com
> > > > >
> > > > >> > wrote:
> > > > >> >
> > > > >> > > Hi everyone,
> > > > >> > > Please review and vote on the release candidate #0 for the
> > version
> > > > >> > > 4.6.1, as follows:
> > > > >> > > [ ] +1, Approve the release
> > > > >> > > [ ] -1, Do not approve the release (please provide specific
> > > > comments)
> > > > >> > >
> > > > >> > > The complete staging area is available for your review, which
> > > > includes:
> > > > >> > > * Release notes [1]
> > > > >> > > * The official Apache source and binary distributions to be
> > > deployed
> > > > >> > > to dist.apache.org [2]
> > > > >> > > * All artifacts to be deployed to the Maven Central Repository
> > [3]
> > > > >> > > * Source code tag "release-4.6.1" with gitsha
> > > > >> > > 131e7baaf3ba6f2308b45bdbefa3cf9f2b6208ac [4]
> > > > >> > >
> > > > >> > > BookKeeper's KEYS file contains PGP keys we used to sign this
> > > > >> > > release:
> > > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> > > > >> > >
> > > > >> > > Please download these packages and review this release
> > candidate:
> > > > >> > >
> > > > >> > > - Review release notes
> > > > >> > > - Download the source package (verify md5, shasum, and asc)
> and
> > > > follow
> > > > >> > the
> > > > >> > > instructions to build and run the bookkeeper service.
> > > > >> > > - Download the binary package (verify md5, shasum, and asc)
> and
> > > > follow
> > > > >> > the
> > > > >> > > instructions to run the bookkeeper service.
> > > > >> > > - Review maven repo, release tag, licenses, and any other
> things
> > > you
> > > > >> > think
> > > > >> > > it is important to a release.
> > > > >> > >
> > > > >> > > The vote will be open for at least 72 hours. It is adopted by
> > > > majority
> > > > >> > > approval, with at least 3 PMC affirmative votes.
> > > > >> > >
> > > > >> > > Thanks,
> > > > >> > > Enrico Olivelli
> > > > >> > >
> > > > >> > > [1] https://github.com/apache/bookkeeper/pull/980
> > > > >> > > [2] https://dist.apache.org/repos/dist/dev/bookkeeper/
> > > > >> > > bookkeeper/bookkeeper-4.6.1-rc/
> > > > >> > > [3] https://repository.apache.org/content/repositories/
> > > > >> > > orgapachebookkeeper-1024/
> > > > >> > > [4]
> > > https://github.com/apache/bookkeeper/releases/tag/release-4.6.1
> > > > >> > >
> > > > >> >
> > > > >> --
> > > > >>
> > > > >>
> > > > >> -- Enrico Olivelli
> > > > >>
> > > >
> > >
> > >
> > > --
> > >
> > >
> > > -- Enrico Olivelli
> > >
> >
>



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


Re: [DISCUSS] Thoughts on enabling rerunFailedTests in surefire?

2018-01-24 Thread Venkateswara Rao Jujjuri
Great Idea. In addition, I would like to propose :

Strictly tests (especially CI tests) must not be flaky. It really doesn't
carry much use. As everyone tends to ignore them.
Step-1: Separate them into a different pool, and you won't get a red mark
for failing this pool/group.
Step-2: Fix each test to be consistent and move it out of the optional pool.
Step-3: If there are any tests that can't be consistent but makes sense
(???) to keep them part of CI, we can leave them in this flaky/optional
pool.

Thanks,
JV

On Wed, Jan 24, 2018 at 8:56 AM, Sijie Guo  wrote:

> On Wed, Jan 24, 2018 at 5:20 AM Ivan Kelly  wrote:
>
> > Hi folks,
> >
> > Looking at open PRs right now, there's a lot of red Xs
> >
> >
> > https://screenshotscdn.firefoxusercontent.com/images/
> 8df9daf7-d71a-4988-a9cc-69d9c10c2636.png
> >
> > I see only 3 green. I know it's not all flakes, but a good amount of
> > it is. Enough to make a red X meaningless.
> >
> > So I propose we enable the rerunFailingTest option in surefire. This
> > will rerun the flakes. I suggest we set to 2. Anything that it flaking
> > more than once per run needs to be looked at.
> >
> >
> > http://maven.apache.org/surefire/maven-surefire-plugin/test-mojo.html#
> rerunFailingTestsCount
>
>
> Good idea. +1
>
>  rerunFailingTestsCount>
> >
> > As part of this, I suggest that we also institute a no submission on
> > red policy (i.e. tests have to get full green to be merged).
>
>
> FYI. I created one issue for same purpose before:
>
> https://github.com/apache/bookkeeper/issues/1031
>
>
>
> >
> > Regards,
> > Ivan
> >
>



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


BookKeeper Durability issues/proposals

2018-01-16 Thread Venkateswara Rao Jujjuri
Here is the document I wrote on the durability.
Please review/comment. We can crate issues based on this.

https://docs.google.com/document/d/1LYZSiJ2x5mXMOjQqMWU7ZBRJD8YVDkB-hRPZ474O7Qw/edit?usp=sharing



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


Re: BP14 help wanted

2018-01-12 Thread Venkateswara Rao Jujjuri
Thanks for the update Enrico. Is there any estimation on how long your
priorities are going to be different?
couple of weeks? or more than that?

On Fri, Jan 12, 2018 at 11:21 AM, Enrico Olivelli 
wrote:

> Hi BookKeepers,
> During this period I have changed my priorities, I am working on releases
> of other products and actually I will be able to work at full time at BP14
> only after some week, I need it for summer.
>
> If someone has cycles to help I will appreciate, feel free to pick up
> current patches and move forward. I will continue to follow that work, that
> is very important for Bookkeeper as storage engine.
>
> Enrico
> --
>
>
> -- Enrico Olivelli
>



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


Dec 14 2018 - MoM

2017-12-14 Thread Venkateswara Rao Jujjuri
   1.

   The release is delayed because of licensing issue.
   2. Sam informed that salesforce is in the process of upstreaming local
   changes. Expect to see lots of pull requests from us.
   3. Charan requested the team, especially Enrico Olivelli
    and Sijie Guo
    to review his
   off-branch multiple entrylog patch. Just for the direction, as this patch
   needs to be rebased on upstream.
   4. JV informed that he opened an issue to populate MDC context around
   the stack; We have internal patch, but upstream threading context changed
   so much, we need to do a different patch.
   5. We briefly talked about the Ledger Balancer issue, did not spend a
   lot of time.
   6. JV proposed to cancel all meetings for the rest of the year, and
   everyone agreed.



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


Re: [ANNOUNCE] Apache BookKeeper 4.5.1 released

2017-11-22 Thread Venkateswara Rao Jujjuri
Congrats and Great work Enrico!!

Awesome Job.

On Wed, Nov 22, 2017 at 6:29 PM, Matteo Merli  wrote:

> Great work Enrico!
>
> On Wed, Nov 22, 2017 at 12:53 PM Enrico Olivelli 
> wrote:
>
> > The Apache BookKeeper team is proud to announce Apache BookKeeper version
> > 4.5.1.
> >
> > Apache BookKeeper is a scalable, fault-tolerant, and low-latency
> > storage service optimized for
> > real-time workloads. It has been used for a fundamental service to
> > build reliable services.
> > It is also the log segment store for Apache DistributedLog and the
> > message store for Apache Pulsar.
> >
> > This is the 6th release of the Apache BookKeeper.
> >
> > This is a bugfix release, it fixes bugs around parallel recovery,
> > Prometheus stats provider and placement policies.
> >
> > For BookKeeper release details and downloads, visit:
> >  http://www.apache.org/dyn/closer.cgi/bookkeeper
> >
> > BookKeeper 4.5.1 Release Notes are at:
> > https://bookkeeper.apache.org/docs/4.5.1/overview/releaseNotes/
> >
> > We would like to thank the contributors that made the release possible.
> >
> > Regards,
> >
> > The BookKeeper Team
> >
> --
> Matteo Merli
> 
>



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


Re: Question regarding Checkpoint logic in SortedLedgerStorage

2017-10-13 Thread Venkateswara Rao Jujjuri
Charan this looks like an issue to me. Do we have a defect/issue opened?

On Fri, Oct 13, 2017 at 4:18 PM, Sijie Guo  wrote:

> Charan,
>
> Didn't mean to say the logic is correct. I was just trying to point out
> the points that I remembered for checkpoint.
>
> I am currently traveling, so I don't have code available to check the
> sequence. I can check the logic when I am close to the laptop.
>
> Sijie
>
>
> On Oct 14, 2017 6:11 AM, "Charan Reddy G"  wrote:
>
> Hey Sijie,
>
> I'm not questioning the semantics of checkpoint or the optimization which
> was added with Bookkeeper-564. But my concern is are we sure, checkpoint
> logic/code is correct and "marker is only updated when all the entries
> added before are persisted.", in the case of SortedLedgerStorage. Can you
> please go through the scenario I mentioned in my email. From what I
> understood, if entryLog is rotated because of addEntry request from GC,
> then we set the 'currentMark' of journal as 'lastCheckpoint' in
> checkpointHolder. The entries added before this 'lastCheckpoint' are still
> in EntryMemTable. When next checkpoint happens, we are not actually
> persisting entries which were in EntryMemTable but we are marking
> 'lastCheckpoint' as LogMarkCheckpoint (in checkpointComplete method).
>
> Thanks,
> Charan
>
> On Fri, Oct 13, 2017 at 2:46 PM, Sijie Guo  wrote:
>
> > The core of the checkpoint is:
> >
> > - marker is only updated when all the entries added before are persisted.
> > That means it doesn't affect correctness if entries added after are
> > flushed.
> >
> > - the flush in entry log files is just writing data to filesystem. The
> real
> > fsync happens after checkpoint. The separate is for performance
> > consideration.
> >
> >
> >
> > On Oct 12, 2017 11:34 PM, "Charan Reddy G" 
> > wrote:
> >
> > > Hey Sijie/IvanK,
> > >
> > > With
> > > https://github.com/apache/bookkeeper/commit/
> > d175ada58dcaf78f0a70b0ebebf489
> > > 255ae67b5f
> > > you introduced Bookkeeper-564 : Better checkpoint mechanism -
> Scheduling
> > > checkpoint only when rotating an entry log file.
> > >
> > > I'm trying to understand how it would work in the following scenario
> > > - using SortedLedgerStorage
> > > - since it is SortedLedgerStorage entries would be in EntryMemtable
> > > - GarbageCollectorThread.EntryLogScanner.process method calls
> > entryLogger
> > > .addEntry(ledgerId, entry)
> > > - in EntryLogger.addEntry method, lets say it comes to know it has
> > reached
> > > EntryLogLimit and creates NewLog
> > > -  since current active entrylog is rotated,
> > > EntryLogListener.onRotateEntryLog is called
> > > - which sets the currentMark of journal to checkpointHolder. Point to
> > note,
> > > that all the entries added to the Bookie are not added to entryLog yet,
> > > there are entries still in entrymemtable
> > > - lets say SyncThread tries to checkpoint at this instant
> > >
> > > now the concern is, in SortedLedgerStorage.checkpoint method, before
> > > calling super.checkpoint(checkpoint), it does memTable.flush(this,
> > > checkpoint); But memTable.flush would just add entries to the current
> > > active entrylog (BufferedLogChannel) and it doesn't guarantee
> > persistence.
> > > super(InterLeavedLedgerStorage).checkpoint will only flushRotatedLogs
> > > (persists) and finally mark the checkpointcomplete with
> 'lastcheckpoint',
> > > but the 'lastCheckpoint' in the checkpointHolder would also include the
> > > entries which were in Entrymemtable and are not actually persisted in
> the
> > > whole process. Is there issue in SortedLedgerStorage checkpoint logic?
> > >
> > > @Override
> > > public Checkpoint checkpoint(final Checkpoint checkpoint) throws
> > > IOException {
> > > Checkpoint lastCheckpoint = checkpointHolder.
> > getLastCheckpoint();
> > > // if checkpoint is less than last checkpoint, we don't need to
> > do
> > > checkpoint again.
> > > if (lastCheckpoint.compareTo(checkpoint) > 0) {
> > > return lastCheckpoint;
> > > }
> > > memTable.flush(this, checkpoint);
> > > return super.checkpoint(checkpoint);
> > > }
> > >
> > > Thanks,
> > > Charan
> > >
> >
>
>
>


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


Re: [VOTE] Move completely to github

2017-10-09 Thread Venkateswara Rao Jujjuri
+1

On Mon, Oct 9, 2017 at 2:17 AM, Ivan Kelly  wrote:

> My vote is a +1
>
> On Mon, Oct 9, 2017 at 11:17 AM, Ivan Kelly  wrote:
> > I forgot the timeframe. The deadline to vote is midday UTC, on
> > Thursday 12th October.
> >
> >
> > On Mon, Oct 9, 2017 at 11:16 AM, Ivan Kelly  wrote:
> >> Hi folks,
> >>
> >> We discussed this in
> >> http://mail-archives.apache.org/mod_mbox/bookkeeper-dev/
> 201710.mbox/ajax/%3CCAO2yDyarL1%2Bf7AWB89NfUH3Ji35RZVaR-
> CmLYV16i-1RW%2BA8Lg%40mail.gmail.com%3E
> >>
> >> But we never formally called a vote, so here's the vote.
> >>
> >> The bylaws don't actually cover votes like this, so lets use lazy
> >> majority, active committers.
> >>
> >> -Ivan
>



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


Re: Cookies and empty disks

2017-10-09 Thread Venkateswara Rao Jujjuri
Can we have a doc to put all these things? Thread has grown enough to cause
confusion.

Immediate things.
1. Don't assume new bookie if journal dir is empty.
2. Put cookies through bookie format, and bookie never boots on an empty
cookie or mismatched cookie.
3. We can live with operations procedure to deal with incarnation issue.
Infact we run an automated bookie decomm script which runs through the
entire metadata and makes sure that the bookie is not part of any ledger.

For next step:
1. Establish incarnation support.
2. Deal with bitrot.

Makes sense?

JV

On Mon, Oct 9, 2017 at 8:55 AM, Sijie Guo  wrote:

> On Oct 9, 2017 1:54 AM, "Ivan Kelly"  wrote:
>
> Hi folks,
>
> I was travelling over the weekend, so I didn't have a chance to reply
> to anything on this thread. First off, as Enrico said, there's a lot
> of different topics being discussed at once. Perhaps each should be
> broken into a github issue, and then we can continue each conversation
> there, as it's getting a but unwieldy for email.
>
> I've created a cookie monster project, which we can throw all the issues
> into.
> https://github.com/apache/bookkeeper/projects/1
>
> There's a few individual opinions I'd like to give here though.
>
> > Needing the check the instance of the bookie when auditing
>
> The auditor, while it does check when bookies have disappeared, it
> also periodically checks all ledgers by reading the first and last
> entry of each segment. So even if a bookie has resurrected, the
> auditor will find that it is missing entries it is supposed to have.
>
> > UUID in ledger metadata
>
> At least for the write path, I'm not sure if this is needed, but
> consider the following.
>
> Only one writer can "vote" on the entries of the ledger. Other writers
> are fencing writers. A fencing writer has to hit a majority of bookies
> to proceed to closing the ledger. Unless a majority have been wiped,
> it will not proceed to close as an empty ledger. However, if a
> majority have been wiped, the correct behaviour would be for it not be
> possible to close the ledger, as we cannot know what the end of the
> ledger is.
>
> That said, not boot if any ledger refers to a bookie could solve this.
>
> > No ledgers referencing bookie? (Sijie's suggestion)
>
> I'm resistant this idea, because it assumes a central oracle where all
> ledgers can be queried. I know we currently have this, but I don't
> think it scales for each bookie to read the metadata of the whole
> system.
>
> In any case, why not instead of refusing to start if any ledgers
> reference the bookie, on boot the bookie checks which ledgers it is
> supposed to have, and if it doesn't have them, start pulling the data
> for them itself. While doing this replication it should avoid all new
> writes.
>
>
> Yes, that's another thing we need to improve for auto recovery. It is not
> only on boot, you need to do it periodically, in the garbage collection
> thread. The bookie need to scan what ledgers are missing and what entries
> are missing and replicate them.
>
>
>
> > Storing the list of files in the cookie? (Enrico's suggestion)
>
> I don't think this is needed. The purpose of the cookie is to protect
> against stuff like a mount not coming up, or a machine being
> completely wiped. We assume that on a journalled filesystem, files
> don't just disappear arbitrarily. There may be corruption in
> individual files, but see my first point.
>
> Anyhow, as I said earlier, we should decide the broad topics here and
> move into issues. I've made a first pass.
>
> Regards,
> Ivan
>



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


Re: Cookies and empty disks

2017-10-06 Thread Venkateswara Rao Jujjuri
athological
scenario
where AQ bookies have went through this scenario, which is ~ 3


2) Ledger Changed.
>
> It is similar as what the case that Ivan' described. If a writer becomes
> 'network partitioned', and the ledger is deleted during this period, after
> the writer comes back, the writer can still successfully write entries to
> the bookies, because the ledgers are already deleted and all the fencing
> bits are gone.
>
> This violates the expectation of "fencing". but I am not sure we need to
> spend time on fixing this, because the ledger is already explicitly deleted
> by the application. so I think the behavior should be categorized as
> "undefined", just like "deleting a ledger when a writer is still writing
> entries" is a undefined behavior.
>
>
> To summarize my thought on this:
>
> 1. we need to revert the cookie behaviour to the original behavior. make
> sure the cookie works as expected.
> 2. introduce UUID or epoch in the cookie. client connection should carry
> namespace uuid and bookie uuid when establishing the connection.
> 3. work on BP-4 to have a complete lifecycle management to take bookie out
> and add bookie out.
>
> 1 is the immediate fix, so correct operations can still guarantee the
> correctness.
>

I agree we need to take care of #1 ASAP and have a Issues opened and
designs for #2 and #3.

Thanks,
JV

>
> - Sijie
>
>
>
> On Fri, Oct 6, 2017 at 9:35 AM, Venkateswara Rao Jujjuri <
> jujj...@gmail.com>
> wrote:
>
> > > However, imagine that the fenced message is only in the journal on b2,
> > > b2 crashes, something wipes the journal directory and then b2 comes
> > > back up.
> >
> > In this case what happened?
> > 1. We have WQ = 1
> > 2. We had data loss (crash and comeup clean)
> >
> > But yeah, in addition to dataloss we have fencing violation too.
> > The problem is not just wiped journal dir, but how we recognize the
> bookie.
> > Bookie is just recognized by its ip address, not by its incarnation.
> > Bookie1 at T1  (b1t1) ; and same bookie1 at T2 after bookie format (b1t2)
> > should be two different bookies, isn;t it?
> > this is needed for the replication worker and the auditor too.
> >
> > Also, bookie needs to know if the writer/reader is intended to read from
> > b1t2 not from b1t1.
> > Looks like we have a hole here? Or I may not be fully understanding
> cookie
> > verification mechanism.
> >
> > Also as Ivan pointed out, we appear to think the lack of journal is
> > implicitly a new bookie, but overall cluster doesn't differentiate
> between
> > incarnations.
> >
> > Thanks,
> > JV
> >
> >
> >
> >
> >
> > On Fri, Oct 6, 2017 at 8:46 AM, Ivan Kelly <iv...@apache.org> wrote:
> >
> > > > The case you described here is "almost correct". But there is an key
> > > here:
> > > > B2 can't startup itself if journal disk is wiped out, because the
> > cookie
> > > is
> > > > missed.
> > > This is what I expected to see, but isn't the case.
> > > 
> > >   List journalCookies = Lists.newArrayList();
> > > // try to read cookie from journal directory.
> > > for (File journalDirectory : journalDirectories) {
> > > try {
> > > Cookie journalCookie =
> > > Cookie.readFromDirectory(journalDirectory);
> > > journalCookies.add(journalCookie);
> > > if (journalCookie.isBookieHostCreatedFromIp()) {
> > > conf.setUseHostNameAsBookieID(false);
> > > } else {
> > > conf.setUseHostNameAsBookieID(true);
> > > }
> > > } catch (FileNotFoundException fnf) {
> > > newEnv = true;
> > > missedCookieDirs.add(journalDirectory);
> > > }
> > > }
> > > 
> > >
> > > So if a journal is missing the cookie, newEnv is set to true. This
> > > disabled the later checks.
> > >
> > > > Hower it can still happen in a different case: bit flap. In your
> case,
> > if
> > > > fence bit in b2 is already persisted on disk, but it got corrupted.
> > Then
> > > it
> > > > will cause the issue you described. One problem is we don't have
> > checksum
> > > > on the index file header when it stores those fence bits.
> > > Yes, this is also an issue.
> > >
> > > -Ivan
> > >
> >
> >
> >
> > --
> > 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


Re: Cookies and empty disks

2017-10-06 Thread Venkateswara Rao Jujjuri
On Fri, Oct 6, 2017 at 10:01 AM, Ivan Kelly <iv...@apache.org> wrote:

> On Fri, Oct 6, 2017 at 6:35 PM, Venkateswara Rao Jujjuri
> <jujj...@gmail.com> wrote:
> >> However, imagine that the fenced message is only in the journal on b2,
> >> b2 crashes, something wipes the journal directory and then b2 comes
> >> back up.
> >
> > In this case what happened?
> > 1. We have WQ = 1
> > 2. We had data loss (crash and comeup clean)
> Ah, maybe this was unclear. I meant, WQ = 2, the fencing is persisted
> fully on b1, but only onto the journal on b2.
>
> > But yeah, in addition to dataloss we have fencing violation too.
> > The problem is not just wiped journal dir, but how we recognize the
> bookie.
> > Bookie is just recognized by its ip address, not by its incarnation.
> > Bookie1 at T1  (b1t1) ; and same bookie1 at T2 after bookie format (b1t2)
> > should be two different bookies, isn;t it?
> This is something I want to change also for the zk stuff. There should
> be a uuid generated for the bookie when it is formatted, so that we
> can distinguish between instances. This uuid should be part of the
> cookie. Once we detect the uuid for a bookie has changed, all ledgers
> on that bookie should be checked somehow.
>

Just that may not be sufficient.
1. The UUID needs to be part of the ledger metadata so that the auditor
knows it is looking at different bookie.
2. Bookie need to know if the writes and reads from the client are intended
for it or not. If not in your case C1 can come back to life and start to
write without any problem.



> -Ivan
>



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


Re: Cookies and empty disks

2017-10-06 Thread Venkateswara Rao Jujjuri
> However, imagine that the fenced message is only in the journal on b2,
> b2 crashes, something wipes the journal directory and then b2 comes
> back up.

In this case what happened?
1. We have WQ = 1
2. We had data loss (crash and comeup clean)

But yeah, in addition to dataloss we have fencing violation too.
The problem is not just wiped journal dir, but how we recognize the bookie.
Bookie is just recognized by its ip address, not by its incarnation.
Bookie1 at T1  (b1t1) ; and same bookie1 at T2 after bookie format (b1t2)
should be two different bookies, isn;t it?
this is needed for the replication worker and the auditor too.

Also, bookie needs to know if the writer/reader is intended to read from
b1t2 not from b1t1.
Looks like we have a hole here? Or I may not be fully understanding cookie
verification mechanism.

Also as Ivan pointed out, we appear to think the lack of journal is
implicitly a new bookie, but overall cluster doesn't differentiate between
incarnations.

Thanks,
JV





On Fri, Oct 6, 2017 at 8:46 AM, Ivan Kelly  wrote:

> > The case you described here is "almost correct". But there is an key
> here:
> > B2 can't startup itself if journal disk is wiped out, because the cookie
> is
> > missed.
> This is what I expected to see, but isn't the case.
> 
>   List journalCookies = Lists.newArrayList();
> // try to read cookie from journal directory.
> for (File journalDirectory : journalDirectories) {
> try {
> Cookie journalCookie =
> Cookie.readFromDirectory(journalDirectory);
> journalCookies.add(journalCookie);
> if (journalCookie.isBookieHostCreatedFromIp()) {
> conf.setUseHostNameAsBookieID(false);
> } else {
> conf.setUseHostNameAsBookieID(true);
> }
> } catch (FileNotFoundException fnf) {
> newEnv = true;
> missedCookieDirs.add(journalDirectory);
> }
> }
> 
>
> So if a journal is missing the cookie, newEnv is set to true. This
> disabled the later checks.
>
> > Hower it can still happen in a different case: bit flap. In your case, if
> > fence bit in b2 is already persisted on disk, but it got corrupted. Then
> it
> > will cause the issue you described. One problem is we don't have checksum
> > on the index file header when it stores those fence bits.
> Yes, this is also an issue.
>
> -Ivan
>



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


Re: [DISCUSS] BP-16: remove zookeeper dependency from bookkeeper client

2017-09-16 Thread Venkateswara Rao Jujjuri
If the real use of this work item is to use:
-  Only one bookie
- Bookie is the metadata server too

Maybe it is ok. But I am worried about the *magnitude* of the code refactor
to accommodate this.
I really like the idea of finding an alternate to ZK, but I think we need
to have that direction ironed out
before rushing with the first step without knowing what the next step is.
Again, I am worried about the
code refactor and resulting regression.

Removing ZK is one part, but changing from centralized metadata server
(ZK/etcd kind) model to distributing
metadata across bookies in the cluster is a HUGE leap. We are looking at
implementing our own version of Paxos.

It is possible I am not understanding the full intent here and would love
to have more discussion.
Can we have more discussion on Thursday call again?

Thanks,
JV



On Sat, Sep 16, 2017 at 3:17 AM, Enrico Olivelli 
wrote:

> Yep
> Thank you Jia
>
> Enrico
>
> On sab 16 set 2017, 12:10 Jia Zhai  wrote:
>
> > Since there is not objective. would like to make this BP approved.
> >
> > On Wed, Sep 13, 2017 at 4:24 PM, Sijie Guo  wrote:
> >
> > > On Wed, Sep 13, 2017 at 1:18 AM, Enrico Olivelli 
> > > wrote:
> > >
> > > > 2017-09-13 10:10 GMT+02:00 Sijie Guo :
> > > >
> > > > > On Wed, Sep 13, 2017 at 12:16 AM, Enrico Olivelli <
> > eolive...@gmail.com
> > > >
> > > > > wrote:
> > > > >
> > > > > > I think that this is a good direction to go.
> > > > > >
> > > > > > I believe to the reasons about ZK in huge systems even it is not
> my
> > > > case
> > > > > so
> > > > > > I cannot add comments on this usecase.
> > > > > >
> > > > > > I am fine with direction as long as we are still going to support
> > > > > > ZooKeeper.
> > > > > > BookKeeper is in the Hadoop / ZooKeeper ecosystem and several
> > > products
> > > > > rely
> > > > > > on ZK too, for instance in my systems it is usual to have
> > > > > > BookKeeper/Kafka/HBase/Majordodo  and so I am not going to
> live
> > > > > > without
> > > > > > zookeeper in the short/mid term.
> > > > > >
> > > > > > I am really OK in dropping ZK because for "simple" systems in
> fact
> > > when
> > > > > you
> > > > > > need only BK having the burden of setting up a zookeeper server
> is
> > > > weird
> > > > > > for customers. I usually re-distribute BK + ZK with my
> applications
> > > and
> > > > > we
> > > > > > are talking about little clusters of up to 10 machines.
> > > > > >
> > > > >
> > > > > Just to clarify - we are not dropping ZK here. we are just
> proposing
> > to
> > > > > have a ledger manager implementation that doesn't depend on
> zookeeper
> > > > > directly.
> > > > > We are not modifying any existing ledger manager implementation.
> > > > >
> > > >
> > > >
> > > > Yep, we are on the same page
> > > > for this proposal the bookie will be a sort of "proxy" between the
> > client
> > > > and the actual ledger manager implementation which will "live" inside
> > the
> > > > bookie
> > > > it is only a new ledger manager to be used in clients, this ledger
> > > manager
> > > > will issue RPCs (or kind of "streaming" RPCs) to a list of bookies
> > > >
> > > >
> > > > >
> > > > >
> > > > > >
> > > > > > The direction on this proposal is OK for me and it is very like
> the
> > > > work
> > > > > I
> > > > > > was starting about "standalone mode".
> > > > >
> > > > >
> > > > > > I think it will be very easy to support the case of having a
> single
> > > > > bookie
> > > > > > with this approach or even client+ bookie in the same JVM,
> > > > > > Having multiple bookies will make us to add some other
> coordination
> > > > > > facility between bookies, I would like to know if there is
> already
> > > some
> > > > > > idea about this, are we going to use another product like
> > > etcd,jgroups
> > > > or
> > > > > > implement our own coordination protocol ?
> > > > >
> > > > >
> > > > > we are not replacing A with B, even with zookeeper. the ledger
> > > management
> > > > > is already abstracted in interfaces.
> > > > > the users can use whatever system they prefer as the metadata
> store.
> > > > >
> > > > > our direction is to provide an option to store metadata as well as
> > data
> > > > in
> > > > > bookies. so in this option, there is no external metadata storage
> > > needed.
> > > > >
> > > >
> > > > Sorry. Maybe my curiosity is not clear.
> > > > If you have multiple bookies and each bookie holds its own version of
> > > > metadata, how do you coordinate them ? which will be the source of
> > truth
> > > ?
> > > > Maybe we should start a new email thread in the future to talk about
> > > > "alternative distributed metadata storages"
> > > >
> > >
> > > It is out of the scope of this BP. We will have a next BP to cover this
> > > part.
> > >
> > >
> > >
> > >
> > > >
> > > > Any way the meaning and the scope of the proposal is clear to me and
> I
> > am
> > > > really OK with it, I hope 

Re: Relax durability

2017-08-31 Thread Venkateswara Rao Jujjuri
Hi all,

It has been a great and lively discussion. I can say this is one of the
highly trended topics in the recent BK community discussion.
Kudos to Enrico for starting this.

Enrico, Sijie and I met and discussed this further and came up with the
following consensus on how to move forward.

* Introduce LedgerType/LedgerProperties which goes into ZK metadata.
* No changes to AddEntry API (application view); but AddEntry RPC will add
a flag to bookies to inform about the type/durability.
* Introduce a sync() RPC which needs to be called explicitly on RD ledgers.
* No changes to LAC and how we update it.
* No changes to the behavior of readEntries() API, which reads only until
LAC.
* Applications can use readUnConcirmed API to read until last add pushed.
* Segregate stats based on the ledgertype.


Enrico is going to merge two docs and publish a detailed design. Thanks a
lot Enrico


On Mon, Aug 21, 2017 at 10:01 PM, Sijie Guo  wrote:

> On Aug 21, 2017 5:44 AM, "Enrico Olivelli"  wrote:
>
> As the issue is really huge, I need to narrow the design and implementation
> efforts to a specific case at the moment: I am interested in having a
> per-ledger flag to not require fsynch on entries on journal.
>
>
> It is good to narrow down the implementation. However because there are
> different requirements from different people. It would be good to discuss
> and cover all thoughts.
>
>
> If the "no-synch" flag is applied per ledger than we have to decide what to
> do on the LAC protocol, I see two opposite ways:
> 1) the LAC will never advanced (no fsynch is guaranteed on journal)
> 2) the LAC is advanced as usual but it will be possible to have missing
> entries
>
>
> Personally I am -1 to approach 2) as for the reasons I stated in previous
> emails.
>
>
> There is a "gray" situation:
> 3) as entries will be interleaved on the journal with entries of other
> "synch" ledgers it will be possible to detect some kind of "synched"
> entries and return the info to the writing client which in turn will be
> able to advance the LAC:
> this option is not useful as the behavior is unpredictable
>
> For my "urgent" usecase I would prefer 2), but 1) is possible too, because
> I am using LedgerHandlerAdv (I have manual allocation of entry ids) +
> readUnconfirmedEntries (which allows to read entries even if LAC did not
> advance)
>
>
> As JV suggested, please start the design doc and let's iterate over it
> before the implementation.
>
>
> -- Enrico
>
>
> 2017-08-19 14:09 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > On ven 18 ago 2017, 20:12 Sijie Guo  wrote:
> >
> >> /cc (distributedlog-dev@)
> >>
> >> I know JV has similar use cases. This might require a broad discussion.
> >> The
> >> most tricky part would be LAC protocol - when can the client advance the
> >> LAC. I think a BP, initially with a google doc shared to the community
> >> would be good to start the discussion. because I would expect a lot
> points
> >> to discuss for this topic. Once we finalize the details, we can copy the
> >> google doc content back to the wiki page.
> >>
> >
> > Thank you Sijie and JV for pointing me to the right direction.
> > I had underestimated the problems related to the ensemble changes, and
> > also effectively in my projects  it can happen that a single
> 'transaction'
> > can span more then one ledger so the ordering issues are nore complex
> than
> > I expected. If somehow it would be possible to keep ordering inside the
> > scope of a single ledger it is very hard to get it using multiple
> ledgers.
> >
> > Next week I will write the doc, but I think I am going to split the
> > problem into multiple parts.
> > I see that the LAC must be advanced only when an fsynch is done. This
> will
> > preserve correctness as Sijie told.
> >
> > I think that the problems related to the ordering of events must be
> > addressed at application level and it would be the best thing to have
> such
> > support in DL.
> >
> > For instance at first glance I omage that we should add in BK some
> support
> > in order to let the application receive notifications of changes to LAC
> to
> > the writer more easily.
> >
> > The first step would be to add a new flag to addEntry to receive
> > acknowledge on fwrite and flush (with the needed changes to the journal),
> > and in the addresponse a flag wjich tells that the entry has been synched
> > or only flushed, and handle the LAC according to this information.
> >
> > Other comments inline
> > Enrico
> >
> >
> >
> >
> >
> >> Other comments inline:
> >>
> >>
> >> On Thu, Aug 17, 2017 at 4:42 AM, Enrico Olivelli 
> >> wrote:
> >>
> >> > Hi,
> >> > I am working with my colleagues at an implementation to relax the
> >> > constraint that every acknowledged entry must have been successfully
> >> > written and fsynced to disk at journal level.
> >> >
> >> > The idea is to have a flag in addEntry to ask for 

Re: [DISCUSS] BP-14 Relax Durability

2017-08-29 Thread Venkateswara Rao Jujjuri
I don't believe I fully followed your second case. But even in this case,
your major concern is about the additional 'sync' RPC?
or something else that the LedgerType proposal won't work?



On Mon, Aug 28, 2017 at 7:35 AM, Enrico Olivelli <eolive...@gmail.com>
wrote:

> I think that having a set of options on the ledger metadata will be a good
> enhancement and I am sure we will do it as soon as it will be needed, maybe
> we do not need it now.
>
> Actually I think we will need to declare this durability-level at entry
> level to support some uses cases in BP-14 document, let me explain two of
> my usecases for which I need it:
>
> At higher level we have to choices:
>
> A) per-ledger durability options (JV proposal)
> all addEntry operations are durable or non-durable and there is an explicit
> 'sync' API (+ forced sync at close)
>
> B) per-entry durability options (original BP-14 proposal)
> every addEntry has an own durable/non-durable option (sync/no-sync), with
> the ability to call 'sync' without addEntry (+ forced sync at close)
>
> I am speaking about the the database WAL case, I am using the ledger as
> segment for the WAL of a database and I am writing all data changes in the
> scope of a 'transaction' with the relaxed-durability flag, then I am
> writing the 'transaction committed' entry with "strict durability"
> requirement, this will in fact require that all previous entries are
> persisted durably and so that the transaction will never be lost.
>
> In this scenario we would need an addEntry + sync API in fact:
>
> using option  A) the WAL will look like:
> - open ledger no-sync = true
> - addEntry (set foo=bar)  (this will be no-sync)
> - addEntry (set foo=bar2) (this will be no-sync)
> - addEntry (commit)
> - sync
>
> using option B) the WAL will look like
> - open ledger
> - addEntry (set foo=bar), no-sync
> - addEntry (set foo=bar2), no-sync
> - addEntry (commit), sync
>
> in case B) we are "saving" one RPC call to every bookie (the 'sync' one)
> same for single data change entries, like updating a single record on the
> database, this with BK 4.5 "costs" only a single RPC to every bookie
>
> Second case:
> I am using BookKeeper to store binary objects, so I am packing more
> 'objects' (named sequences of bytes) into a single ledger, like you do when
> you write many records to a file in a streaming fashion and keep track of
> offsets of the beginning of every record (LedgerHandeAdv is perfect for
> this case).
> I am not using a single ledger per 'file' because it kills zookeeper to
> create many ledgers very fast, in my systems I have big busts of writes,
> which need to be really "fast", so I am writing multiple 'files' to every
> single ledger. So the close-to-open consistency at ledger level is not
> suitable for this case.
> I have to write as fast as possible to this 'ledger-backed' stream, and as
> with a 'traditional'  filesystem I am writing parts of each file and than
> requiring 'sync' at the end of each file.
> Using BookKeeper you need to split big 'files' into "little" parts, you
> cannot transmit the contents as to "real" stream on network.
>
> I am not talking about bookie level implementation details I would like to
> define the high level API in order to support all the relevant known use
> cases and keep space for the future,
> at this moment adding a per-entry 'durability option' seems to be very
> flexible and simple to implement, it does not prevent us from doing further
> improvements, like namely skipping the journal.
>
> Enrico
>
>
>
> 2017-08-26 19:55 GMT+02:00 Enrico Olivelli <eolive...@gmail.com>:
>
> >
> >
> > On sab 26 ago 2017, 19:19 Venkateswara Rao Jujjuri <jujj...@gmail.com>
> > wrote:
> >
> >> Hi all,
> >>
> >> As promised during Thursday call, here is my proposal.
> >>
> >> *NOTE*: Major difference in this proposal compared to Enrico’s
> >> <https://docs.google.com/document/d/1JLYO3K3tZ5PJGmyS0YK_-
> >> NW8VOUUgUWVBmswCUOG158/edit#heading=h.q2rewiqndr5v>
> >> is
> >> making the durability a property of the ledger(type) as opposed to
> >> addEntry(). Rest of the technical details have a lot of similarities.
> >>
> >
> > Thank you JV. I have just read quickly the doc and your view is centantly
> > broader.
> > I will dig into the doc as soon as possible on Monday.
> > For me it is ok to have a ledger wide configuration I think that the most
> > important decision is about the API we will provide as in the future it
> > will be difficult to change it.
> >
> >
> 

Re: [DISCUSS] BP-14 Relax Durability

2017-08-26 Thread Venkateswara Rao Jujjuri
Hi all,

As promised during Thursday call, here is my proposal.

*NOTE*: Major difference in this proposal compared to Enrico’s

is
making the durability a property of the ledger(type) as opposed to
addEntry(). Rest of the technical details have a lot of similarities.

https://docs.google.com/document/d/1g1eBcVVCZrTG8YZliZP0LVqvWpq432ODEghrGVQ4d4Q/edit?usp=sharing

On Thu, Aug 24, 2017 at 1:14 AM, Enrico Olivelli 
wrote:

> Thank you all for the comments and for taking a look to the document so
> soon.
> I have updated the doc, we will discuss the document at the meeting,
>
>
> Enrico
>
> 2017-08-24 2:27 GMT+02:00 Sijie Guo :
>
> > Enrico,
> >
> > Thank you so much! It is a great effort for putting this up. Overall
> looks
> > good. I made some comments, we can discuss at tomorrow's community
> meeting.
> >
> > - Sijie
> >
> > On Wed, Aug 23, 2017 at 8:25 AM, Enrico Olivelli 
> > wrote:
> >
> > > Hi all,
> > > I have drafted a first proposal for BP-14 - Relax Durability
> > >
> > > We are talking about limiting the number of fsync to the journal while
> > > preserving the correctness of the LAC protocol.
> > >
> > > This is the link to the wiki page, but as the issue is huge we prefer
> to
> > > use Google Documents for sharing comments
> > > https://cwiki.apache.org/confluence/display/BOOKKEEPER/
> > > BP+-+14+Relax+durability
> > >
> > > This is the document
> > > https://docs.google.com/document/d/1JLYO3K3tZ5PJGmyS0YK_-
> > > NW8VOUUgUWVBmswCUOG158/edit?usp=sharing
> > >
> > > All comments are welcome
> > >
> > > I have added DL dev list in cc as the discussion is interesting for
> both
> > > groups
> > >
> > > Enrico Olivelli
> > >
> >
>



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


Re: [DISCUSS] BP-14 Relax Durability

2017-08-26 Thread Venkateswara Rao Jujjuri
Hi all,

As promised during Thursday call, here is my proposal.

*NOTE*: Major difference in this proposal compared to Enrico’s

is
making the durability a property of the ledger(type) as opposed to
addEntry(). Rest of the technical details have a lot of similarities.

https://docs.google.com/document/d/1g1eBcVVCZrTG8YZliZP0LVqvWpq432ODEghrGVQ4d4Q/edit?usp=sharing

On Thu, Aug 24, 2017 at 1:14 AM, Enrico Olivelli 
wrote:

> Thank you all for the comments and for taking a look to the document so
> soon.
> I have updated the doc, we will discuss the document at the meeting,
>
>
> Enrico
>
> 2017-08-24 2:27 GMT+02:00 Sijie Guo :
>
> > Enrico,
> >
> > Thank you so much! It is a great effort for putting this up. Overall
> looks
> > good. I made some comments, we can discuss at tomorrow's community
> meeting.
> >
> > - Sijie
> >
> > On Wed, Aug 23, 2017 at 8:25 AM, Enrico Olivelli 
> > wrote:
> >
> > > Hi all,
> > > I have drafted a first proposal for BP-14 - Relax Durability
> > >
> > > We are talking about limiting the number of fsync to the journal while
> > > preserving the correctness of the LAC protocol.
> > >
> > > This is the link to the wiki page, but as the issue is huge we prefer
> to
> > > use Google Documents for sharing comments
> > > https://cwiki.apache.org/confluence/display/BOOKKEEPER/
> > > BP+-+14+Relax+durability
> > >
> > > This is the document
> > > https://docs.google.com/document/d/1JLYO3K3tZ5PJGmyS0YK_-
> > > NW8VOUUgUWVBmswCUOG158/edit?usp=sharing
> > >
> > > All comments are welcome
> > >
> > > I have added DL dev list in cc as the discussion is interesting for
> both
> > > groups
> > >
> > > Enrico Olivelli
> > >
> >
>



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


Re: Relax durability

2017-08-21 Thread Venkateswara Rao Jujjuri
As I mentioned in my email,  a mixture of sync/relaxed entries of a ledger
with the combination of ensemble changes makes LAC inaccurate. Let us have
a document and iterate over that before writing code.

Thanks,
JV

On Mon, Aug 21, 2017 at 5:35 AM, Enrico Olivelli 
wrote:

> As the issue is really huge, I need to narrow the design and implementation
> efforts to a specific case at the moment: I am interested in having a
> per-ledger flag to not require fsynch on entries on journal.
>
> If the "no-synch" flag is applied per ledger than we have to decide what to
> do on the LAC protocol, I see two opposite ways:
> 1) the LAC will never advanced (no fsynch is guaranteed on journal)
> 2) the LAC is advanced as usual but it will be possible to have missing
> entries
>
> There is a "gray" situation:
> 3) as entries will be interleaved on the journal with entries of other
> "synch" ledgers it will be possible to detect some kind of "synched"
> entries and return the info to the writing client which in turn will be
> able to advance the LAC:
> this option is not useful as the behavior is unpredictable
>
> For my "urgent" usecase I would prefer 2), but 1) is possible too, because
> I am using LedgerHandlerAdv (I have manual allocation of entry ids) +
> readUnconfirmedEntries (which allows to read entries even if LAC did not
> advance)
>
> -- Enrico
>
>
> 2017-08-19 14:09 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > On ven 18 ago 2017, 20:12 Sijie Guo  wrote:
> >
> >> /cc (distributedlog-dev@)
> >>
> >> I know JV has similar use cases. This might require a broad discussion.
> >> The
> >> most tricky part would be LAC protocol - when can the client advance the
> >> LAC. I think a BP, initially with a google doc shared to the community
> >> would be good to start the discussion. because I would expect a lot
> points
> >> to discuss for this topic. Once we finalize the details, we can copy the
> >> google doc content back to the wiki page.
> >>
> >
> > Thank you Sijie and JV for pointing me to the right direction.
> > I had underestimated the problems related to the ensemble changes, and
> > also effectively in my projects  it can happen that a single
> 'transaction'
> > can span more then one ledger so the ordering issues are nore complex
> than
> > I expected. If somehow it would be possible to keep ordering inside the
> > scope of a single ledger it is very hard to get it using multiple
> ledgers.
> >
> > Next week I will write the doc, but I think I am going to split the
> > problem into multiple parts.
> > I see that the LAC must be advanced only when an fsynch is done. This
> will
> > preserve correctness as Sijie told.
> >
> > I think that the problems related to the ordering of events must be
> > addressed at application level and it would be the best thing to have
> such
> > support in DL.
> >
> > For instance at first glance I omage that we should add in BK some
> support
> > in order to let the application receive notifications of changes to LAC
> to
> > the writer more easily.
> >
> > The first step would be to add a new flag to addEntry to receive
> > acknowledge on fwrite and flush (with the needed changes to the journal),
> > and in the addresponse a flag wjich tells that the entry has been synched
> > or only flushed, and handle the LAC according to this information.
> >
> > Other comments inline
> > Enrico
> >
> >
> >
> >
> >
> >> Other comments inline:
> >>
> >>
> >> On Thu, Aug 17, 2017 at 4:42 AM, Enrico Olivelli 
> >> wrote:
> >>
> >> > Hi,
> >> > I am working with my colleagues at an implementation to relax the
> >> > constraint that every acknowledged entry must have been successfully
> >> > written and fsynced to disk at journal level.
> >> >
> >> > The idea is to have a flag in addEntry to ask for acknowledge not
> after
> >> the
> >> > fsync in journal but only when data has been successfully written and
> >> > flushed to the SO.
> >> >
> >> > I have the requirement that if an entry requires synch all the entries
> >> > successfully sent 'before' that entry (causality) are synched too,
> even
> >> if
> >> > they have been added with the new relaxed durability flag.
> >>
> >>
> >> > Imagine a database transaction log, during a transaction I will write
> >> every
> >> > change to data to the WAL with the new flag, and only the commit
> >> > transaction command will be added with synch requirement. The idea is
> >> that
> >> > all the changes inside the scope of the transaction have a meaning
> only
> >> if
> >> > the transaction is committed, so it is important that the commit entry
> >> > won't be lost and if that entry isn't lost all of the other entries of
> >> the
> >> > same transaction aren't lost too.
> >> >
> >>
> >> can you do:
> >>
> >> - lh.asyncAddEntry('entry-1')
> >> - lh.asyncAddEntry('entry-2')
> >> - lh.addEntry('commit')
> >>
> >> ?
> >>
> >
> > Yes, currently ut is the best we can do and I am 

Re: Relax durability

2017-08-17 Thread Venkateswara Rao Jujjuri
On Thu, Aug 17, 2017 at 4:42 AM, Enrico Olivelli 
wrote:

> Hi,
> I am working with my colleagues at an implementation to relax the
> constraint that every acknowledged entry must have been successfully
> written and fsynced to disk at journal level.
>
> The idea is to have a flag in addEntry to ask for acknowledge not after the
> fsync in journal but only when data has been successfully written and
> flushed to the SO.
>
> I have the requirement that if an entry requires synch all the entries
> successfully sent 'before' that entry (causality) are synched too, even if
> they have been added with the new relaxed durability flag.
>
> Imagine a database transaction log, during a transaction I will write every
> change to data to the WAL with the new flag, and only the commit
> transaction command will be added with synch requirement. The idea is that
> all the changes inside the scope of the transaction have a meaning only if
> the transaction is committed, so it is important that the commit entry
> won't be lost and if that entry isn't lost all of the other entries of the
> same transaction aren't lost too.
>

This is a great usecase and we would love to work  with you on this.
I believe there are lot of corner cases. What happens if there is an
ensemble change
after few 'relaxed' writes but before 'sync' write? Sync flag goes to only
the new ensemble
and the data on the old ensemble with 'relaxed' flag has no guarentees.

I have a use case where we would like to do this at ledger grnanualarity
basically close to open consistency.


> I have another use case. In another project I am storing binary objects
> into BK and I have to obtain great performance even on single disk bookie
> layouts (journal + data + index on the same partition). In this project it
> is acceptable to compensate the risk of not doing fsynch if requesting
> enough replication.
> IMHO it will be somehow like the Kakfa idea of durability, as far as I know
> Kafka by default does not impose fsynch but it leaves all to the SO and to
> the fact that there is a minimal configurable number of replicas which are
> in-synch.
>
> There are many open points, already suggested by Matteo, JV and Sijie:
> - LAC protocol?
> - replication in case of lost entries?
> - under production load mixing non synched entries with synched entries
> will not give much benefits
>
>
> For the LAC protocol I think that there is no impact, the point is that the
> LastAddConfirmed is the max entryid which is known to have been
> acknowledged to the writer, so durability is not a concern. You can loose
> entries even with fsynch, just by loosing all the disks which contains the
> data. Without fsynch it is just more probable.
>
> Replication: maybe we should write in the ledger metadata that the ledger
> allows this feature and deal with it. But I am not sure, I have to
> understand better how LaderHandleAdv deals with sparse entryids inside the
> re-replication process
>
> Mixed workload: honestly I would like to add this feature to limit the
> number of fsynch, and I expect to have lots of bursts of unsynched entries
> to be interleaved with a few synched entries. I know that this feature is
> not to be encouraged in general but only for specific cases, like the
> stories of LedgerHandleAdv or readUnconfirmedEntries
>
> If this makes sense to you I will create a BP and attach a first patch
>
> Enrico
>
>
>
>
>
> --
>
>
> -- Enrico Olivelli
>



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


Re: [DISCUSS] Release 4.6.0 and 4.5.1

2017-08-12 Thread Venkateswara Rao Jujjuri
On Fri, Aug 11, 2017 at 10:39 PM, Enrico Olivelli 
wrote:

> Il sab 12 ago 2017, 04:35 Jia Zhai  ha scritto:
>
> > Hi,
> > I would like to have a a try on 4.6.0.
> >
> > On Sat, Aug 12, 2017 at 10:25 AM, Sijie Guo  wrote:
> >
> > > 4.5.0 is out. Although we haven't reached a conclusion whether we want
> to
> > > adopt "Time Based Release Plan" yet, I would still like to start the
> > > discussion on the next releases.
> > >
> > > The next *feature* release will be 4.6.0 and we will probably have a
> bug
> > > fix 4.5.1 for branch 4.5. all the features will go to master and be
> > > included in 4.6.0; bug fixes will first go to master, if they impact
> > 4.5.0,
> > > those changes will be cherry-pick to branch-4.5 (on merging) and be
> > > included in 4.5.1.

> >
> > > Besides that, there are a few things we need to discuss and plan here:
> > >
> > > - what changes that we are planning to include in 4.6.0? If you already
> > > have ideas, please bring them upfront. If you don't have anyone now,
> you
> > > can create issues later and mark them as `4.6.0`
> >
>
> I will create issues as soon as possible, but my (my company) areas of
> urgent work are:
> - java9 build, maybe non yes official support
>
> - relax durability, that is to not wait for synch on journal for some
> entries
>
>
>
> >
> > > [Jia] we have some changes that we'd like to get in. I will create
> issues
> > soon.
> >
> >
> > > - who will be the release managers to drive the next two releases? one
> is
> > > for 4.6.0 feature release, the other one is 4.5.1 bug fix release. we
> > > should start a rotation between committers to drive the releases. Any
> > > volunteers for driving the releases?
> > >
> >
> > [Jia] I would like to have a try on 4.6.0.
> >
>
> I am available, for 4.6, 4.7 or even for 4.5.1
>
> >
> >
> > > - how long that we should target for 4.6.0 and 4.5.1? If we adopted
> `Time
> > > Based Release Plan`, we will be targeting at 4.6.0 release in November.
> > For
> > > bug fix release, it can be a shorter one, around one month.
> >
>
> You think that we will need soon some horfix? 4.4 lasted one year and we
> did not need any 4.4.1. Maybe now 4.5 will be more adopted?
>
>
This is good point. Usually if we have any customers that are purely based
on the community 4.5.0 release
then it warrants identifying what fixes need to be rolled back from master
(4.6). So it really depends on the
needs of 4.5.0 users, and the release of 4.5.1 should be need basis. Its ok
if we never need to do 4.5.0 and
come out 4.6.0, but we need to follow the rule that every new change should
go into master first, as suggested
by Sijie. I like time based release idea and lets pick a date . :)

JV


> Enrico
>
> >
> > > [Jia] Sound good.
> >
> >
> > > Thoughts, Volunteers?
> > >
> > > - Sijie
> > >
> >
> --
>
>
> -- Enrico Olivelli
>



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


Re: [ANNOUNCE] Apache BookKeeper 4.5.0 released

2017-08-11 Thread Venkateswara Rao Jujjuri
Thanks a lot Sijie for patiently walking me through the process.
Now I have real appreciation for the release mangers. ;)

On Fri, Aug 11, 2017 at 10:59 AM, Sijie Guo  wrote:

> Awesome! Well done! This is a great milestone to the bookkeeper community!
>
> Thanks JV for doing the release! Thanks everyone for making this happen.
>
> Sijie
>
> On Aug 11, 2017 9:56 AM, "JV Jujjuri"  wrote:
>
> > The Apache BookKeeper team is proud to announce Apache BookKeeper version
> > 4.5.0.
> >
> > This is the fifth release of BookKeeper as an Apache Top Level Project!
> >
> > The 4.5.0 release incorporates hundreds of new fixes, improvements, and
> > features since previous major release, 4.4.0, which was released over a
> > year ago.
> >
> > It is a big milestone in Apache BookKeeper community, converging from
> three
> > main branches (Salesforce, Twitter and Yahoo).
> >
> > For BookKeeper release details and downloads, visit:
> >
> > http://bookkeeper.apache.org/releases/
> >
> > BookKeeper 4.5.0 Release Notes are at:
> >
> > http://bookkeeper.apache.org/docs/4.5.0/overview/releaseNotes/
> >
> > We would like to thank all the contributors that made the release
> possible.
> >
> > Regards,
> >
> > The Apache BookKeeper Team
> >
>



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


Re: [ANNOUNCE] Apache BookKeeper 4.5.0 released

2017-08-11 Thread Venkateswara Rao Jujjuri
Thanks a lot Sijie for patiently walking me through the process.
Now I have real appreciation for the release mangers. ;)

On Fri, Aug 11, 2017 at 10:59 AM, Sijie Guo  wrote:

> Awesome! Well done! This is a great milestone to the bookkeeper community!
>
> Thanks JV for doing the release! Thanks everyone for making this happen.
>
> Sijie
>
> On Aug 11, 2017 9:56 AM, "JV Jujjuri"  wrote:
>
> > The Apache BookKeeper team is proud to announce Apache BookKeeper version
> > 4.5.0.
> >
> > This is the fifth release of BookKeeper as an Apache Top Level Project!
> >
> > The 4.5.0 release incorporates hundreds of new fixes, improvements, and
> > features since previous major release, 4.4.0, which was released over a
> > year ago.
> >
> > It is a big milestone in Apache BookKeeper community, converging from
> three
> > main branches (Salesforce, Twitter and Yahoo).
> >
> > For BookKeeper release details and downloads, visit:
> >
> > http://bookkeeper.apache.org/releases/
> >
> > BookKeeper 4.5.0 Release Notes are at:
> >
> > http://bookkeeper.apache.org/docs/4.5.0/overview/releaseNotes/
> >
> > We would like to thank all the contributors that made the release
> possible.
> >
> > Regards,
> >
> > The Apache BookKeeper Team
> >
>



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


Re: [DISCUSS] move github notifications to other mailing list

2017-08-10 Thread Venkateswara Rao Jujjuri
Also if there is a way, I would like github to send email to the tagged
person in addition to alias.
Not sure if github can support that, that way tagged person can get quicker
attention instead of processing
every email on the issues@ or dev@ list. Not sure if github can support
that feature.

JV

On Thu, Aug 10, 2017 at 2:34 PM, Sijie Guo  wrote:

> Hi all,
>
> JV and me had a short discussion about emails in the dev mailing list. Most
> of the emails are coming from github notifications. If you don't have
> filters setup correctly, it might be annoying and hard to follow the real
> discussion happening in dev@.
>
> Shall we consider moving all the github notifications to other mailing
> lists? like issues@? Any thoughts?
>
> - Sijie
>



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


[RESULT] [VOTE] Release 4.5.0, release candidate #0

2017-08-10 Thread Venkateswara Rao Jujjuri
I'm happy to announce that we have unanimously approved this release.

There are 8 approving votes, 3 of which are binding:

* JV Jujjuri

* Sijie Guo

* Matteo Merli

* Jia Zhai

* Govind Menon

* Sam Just

* Leigh Stewart

There are no disapproving votes.

Thanks everyone!


On Thu, Aug 10, 2017 at 12:37 AM, Sijie Guo <guosi...@gmail.com> wrote:

> +1
>
> - signatures are good
> - source package: build, rat check, and test
> - binary package: able to run local bookie, and tested on a docker
> environment.
> - distributedlog integration: distributedlog core library is able to
> compile, and all tests passed.
>
> - Sijie
>
> On Tue, Aug 8, 2017 at 4:07 PM, Leigh Stewart <agrodel...@gmail.com>
> wrote:
>
> > +1 (non-binding)
> >
> > - release notes look good
> > - built and ran localbookie from source package successfully
> > - ran localbookie from bin package successfully
> >
> >
> > On Tue, Aug 8, 2017 at 11:45 AM, Sam Just <sj...@salesforce.com> wrote:
> >
> > > +1
> > >
> > > On Tue, Aug 8, 2017 at 10:22 AM, Matteo Merli <mme...@apache.org>
> wrote:
> > >
> > > > +1
> > > >
> > > > Checked src and bin package
> > > >  * Signatures ok
> > > >  * Build
> > > >  * Rat
> > > >  * Run local bookie
> > > >(I had to set allowLoopback=true in conf/bk_server.conf for that.
> I
> > > > agree we can
> > > > document it and improve it later.)
> > > >
> > > > Matteo
> > > >
> > > > On Mon, Aug 7, 2017 at 7:13 AM Venkateswara Rao Jujjuri <
> > > jujj...@gmail.com
> > > > >
> > > > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > Please review and vote on the release candidate #0 for version
> 4.5.0,
> > > as
> > > > > follows:
> > > > >
> > > > > [ ] +1, Approve the release
> > > > > [ ] -1, Do not approve the release (please provide specific
> comments)
> > > > >
> > > > > The complete staging area is available for your review, which
> > includes:
> > > > >
> > > > > * Release Notes [1]
> > > > > * The official Apache source and binary distributions to be
> deployed
> > to
> > > > > dist.apache.org [2]
> > > > > * All artifacts to be deployed to the Maven Central Repository [3]
> > > > > * Source code tag "release-4.5.0" [4]
> > > > >
> > > > > BookKeeper's KEY file contains PGP keys we use to sign this
> release:
> > > > > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> > > > >
> > > > > Please review this release candidate.
> > > > >
> > > > > - Review release notes
> > > > > - Download the source package (verify md5, shasum, and asc) and
> > follow
> > > > the
> > > > > instructions to build and run the bookkeeper service.
> > > > > - Download the binary package (verify md5, shasum, and asc) and
> > follow
> > > > the
> > > > > instructions to run the bookkeeper service.
> > > > > - Review maven repo, release tag, licenses, and any other things
> you
> > > > think
> > > > > it is important to a release.
> > > > >
> > > > > [1] https://github.com/apache/bookkeeper/pull/402
> > > > > [2] https://dist.apache.org/repos/dist/dev/bookkeeper/4.5.0-rc0/
> > > > > [3] https://repository.apache.org/content/repositories/
> > > > > orgapachebookkeeper-1012/
> > > > > [4] https://github.com/apache/bookkeeper/tree/release-4.5.0
> > > > >
> > > > > --
> > > > > Jvrao
> > > > > ---
> > > > > First they ignore you, then they laugh at you, then they fight you,
> > > then
> > > > > you win. - Mahatma Gandhi
> > > > >
> > > > --
> > > > Matteo Merli
> > > > <mme...@apache.org>
> > > >
> > >
> >
>



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


[VOTE] Release 4.5.0, release candidate #0

2017-08-07 Thread Venkateswara Rao Jujjuri
Hi everyone,

Please review and vote on the release candidate #0 for version 4.5.0, as
follows:

[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)

The complete staging area is available for your review, which includes:

* Release Notes [1]
* The official Apache source and binary distributions to be deployed to
dist.apache.org [2]
* All artifacts to be deployed to the Maven Central Repository [3]
* Source code tag "release-4.5.0" [4]

BookKeeper's KEY file contains PGP keys we use to sign this release:
https://dist.apache.org/repos/dist/release/bookkeeper/KEYS

Please review this release candidate.

- Review release notes
- Download the source package (verify md5, shasum, and asc) and follow the
instructions to build and run the bookkeeper service.
- Download the binary package (verify md5, shasum, and asc) and follow the
instructions to run the bookkeeper service.
- Review maven repo, release tag, licenses, and any other things you think
it is important to a release.

[1] https://github.com/apache/bookkeeper/pull/402
[2] https://dist.apache.org/repos/dist/dev/bookkeeper/4.5.0-rc0/
[3] https://repository.apache.org/content/repositories/
orgapachebookkeeper-1012/
[4] https://github.com/apache/bookkeeper/tree/release-4.5.0

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


Re: [VOTE] Slack Channel for BookKeeper

2017-07-28 Thread Venkateswara Rao Jujjuri
+1
Just curious on why decided to transfer rather than create a new one.
Fine either way.

On Fri, Jul 28, 2017 at 12:22 AM, Francesco Caliumi - Diennea <
francesco.cali...@diennea.com> wrote:

> +1 non binding
>
> On Thu, 2017-07-27 at 21:45 +0200, Ivan Kelly wrote:
>
> +1
>
> On Thu, Jul 27, 2017 at 8:10 PM, Matteo Merli  mailto:matteo.me...@gmail.com>> wrote:
>
>
> +1
>
> --
> Matteo Merli
> >
>
> On Thu, Jul 27, 2017 at 11:06 AM, Enrico Olivelli  >
> wrote:
>
>
>
> +1 non binding
>
> Enrico
>
> Il gio 27 lug 2017, 20:02 Sijie Guo  si...@gmail.com>> ha scritto:
>
>
>
> Start a vote thread for transferring the DL slack channel to BK.
>
> The propose is:
>
> - transfer the dl slack channel apachedistributedlog.slack.com to
> apachebookkeeper.slack.com
> - the owner will be BookKeeper PMC (priv...@bookkeeper.apache.org priv...@bookkeeper.apache.org>)
>
> One note:
>
> - the slack channel is for informal/immediate discussion. no decisions
>
>
> are
>
>
> made in slack channel. decision related discussions should be recorded in
> ASF (either mailing list, JIR or wiki)
>
> Please vote +1, 0, -1. The vote will be open for 72 hours.
>
> - Sijie
>
>
>
> --
>
>
> -- Enrico Olivelli
>
>
>
> --
>
> Francesco Caliumi
> Developer @ Diennea - MagNews
> Tel.: (+39) 0546 066100 - Int. 266
> Viale G.Marconi 30/14 - 48018 Faenza (RA)
>
> [Magnews.it]
>
> [Linkedin]
>  [Twitter]   [Facebook] <
> http://www.facebook.com/pages/MagNews/197617841797>  [Newsletter] <
> http://www.magnews.it/it/iscriviti-alla-newsletter>
>
> 
>
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed
> email marketing! http://www.magnews.it/newsletter/
>
> The information in this email is confidential and may be legally
> privileged. If you are not the intended recipient please notify the sender
> immediately and destroy this email. Any unauthorized, direct or indirect,
> disclosure, copying, storage, distribution or other use is strictly
> forbidden.
>



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


Re: ZooKeeper-less - single bookie setup

2017-07-27 Thread Venkateswara Rao Jujjuri
We have inMemoryMetaStore.java already to store metadata locally isn't it?
there are some tests that make use of that too. So if you don't need ZK for
consensus
and you can use this inMemoryMetaStore for metadata store.

On Thu, Jul 27, 2017 at 12:42 AM, Sijie Guo  wrote:

> it might be better to call it 'local model' or 'standalone mode', rather
> than 'zookeeper-less'.
>
> On Wed, Jul 26, 2017 at 8:26 PM, Enrico Olivelli 
> wrote:
>
> > Hi,
> > I would like to share this idea with you before starting a real design
> > effort for this new configuration.
>
>
> > Sometimes I have to run Bookie + BookKeeper inside the same JVM,
> > essentially because I want my applications to run both in "cluster mode"
> > and in "single instance mode".
> > I have some other use case of embedding the Bookie, but this is not the
> > scope of this email.
> >
> > In order to run BookKeeper we need ZooKeeper and running ZooKeeper inside
> > the same process is tricky and you know, it is not the suggested
> > configuration, due to several aspects (out of the scope of this email
> > too)
> >
> > I would like to make it run without ZooKeeper at all.
> > This idea will support my use case (Bookie + Client inside the same JVM)
> > and the single Bookie use case, which is a feature that could be useful
> in
> > general.
> > Some other projects, like HBase, have "single server" vs "multi server"
> > setup, and so I think it will be a good idea for the project and for
> > gathering more interest and use cases.
> >
> > We are using ZooKeeper for (at least) these purposes:
> > - bookie discovery
> > - ledger metadata management
> > - auditor leader election
> > - global configuration settings  (like autorecovery enabled/disabled)
> >
> > my case is to have only one Bookie, so things are really simpler, no
> > auditor, no shared configuration, no underreplication/overreplication,
> no
> > need for 'discovery'
> >
> > In order to address the bookie discovery we can just have some
> client-side
> > configuration to set the address of the Bookie and have some special
> > PlacementPolicy which only returns the same bookie.
> >
> > The big thing is to implement the ledger metadata management, because
> both
> > the Bookie and the client must deal with it.
> > My idea is to add to the Bookie protocol all the functions to access
> > LedgersMetadata (create/list/update with CAS.), this RPCs will call
> the
> > underlying LedgerManager
> >
>
> I think this is in general very good approach to take. It will make the
> bookie client not depends on metadata store directly and delegating the
> heavy work to the server (bookie side). Once the client is thin, it is easy
> to implement clients in other languages.
>
> However, I do not think it is necessarily for your use case. You can make
> this as a separate topic. For RPC, I would suggest exploring gPRC (it is
> protobuf + netty), that would be much better for metadata rpcs rather than
> implementing the logic in bookie protocol.
>
>
> >
> > On the client side we will not use ZK but execute RPCs on the Bookie
> > endpoint (a special LedgerManager which do not use ZK but uses RPCs)
> >
> > On the Bookie side we will have a local ledger metadata management which
> > persists data on disk, we could use existing Key-Value stores, like
> RocksDB
> > or MapDB, or what ever we prefer, we need something which supports CAS
> and
> > eventually let us put bounds on used memory.
> >
>
> In your use case, you might be able to just implement a ledger manager
> using an embedded k/v.
>
>
> >
> > Does this make sense to you ?
> >
> > -- Enrico
> >
>



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


Re: BP-11: New BookKeeper Website

2017-07-25 Thread Venkateswara Rao Jujjuri
Awesome !! lets make it dense. :)

On Tue, Jul 25, 2017 at 12:15 PM, Yiming Zang 
wrote:

> This looks awesome!
>
> On Tue, Jul 25, 2017 at 12:09 PM, Sijie Guo  wrote:
>
> > Hi all,
> >
> > Luc Perkins is helping on a new bookkeeper documentation site for BP-11.
> > Although it is still WIP, we'd like to first share the beta version with
> > the community to get some feedbacks.
> >
> > The new website: https://lucperkins.github.io/bookkeeper/
> > The pull request (still WIP) : https://github.com/apache/
> > bookkeeper/pull/293
> >
> > - Sijie
> >
>



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


Re: BookKeeper 4.5.0 performance regression ?

2017-07-24 Thread Venkateswara Rao Jujjuri
On Mon, Jul 24, 2017 at 3:06 AM, Enrico Olivelli 
wrote:

> 2017-07-21 20:37 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > Il ven 21 lug 2017, 20:32 Sijie Guo  ha scritto:
> >
> >> As the discussion in a separate thread, it might be making sense to
> check
> >> what is the difference between using pooled allocator and unpooled
> >> allocator using v3 protocol. Also considering comparing using heap
> buffer
> >> and direct buffer as well.
> >>
> >> I am suspecting this might contribute latency.
> >>
> >
> > Yep, I am looking in this direction too.
> > I see that many frequent writes lead to an huge use of non heap memory,
> > even bounding the JVM with MaxDirectMemory with max 1GB all, the 12GB of
> my
> > laptop blow away during the run of my benchmark.
> > I suspect it is something in direct memory or something in SO caches.
> > I am not very skilled in SO linux memory diagnostics
> >
>
>
> I wrote a new "write intensive" benchmark, and the only thing I have
> noticed is that Linux is using as much RAM as possible for disk caches,
> this is the expected behavior on Linux.
>

Yes, and this is good behavior. Why keep something unused?


> This is not memory allocated to the process itself.
> There is no difference from 4.4 and 4.5 from this aspect.
>
> I have tried the journalRemoveFromPageCachebut it brings no improvement.
>
> I did some tests as suggested by Flavio, separating the client and the
> bookie (even on different machines). I can say there is no "leak" nor on
> client side neither on bookie side.
>
>
> finally during my benchmarks I noticed that the real "performance drop"
> happens after this lines of log
>
> lug 24, 2017 12:00:56 PM org.apache.bookkeeper.bookie.EntryLogger
> flushRotatedLogs
> INFO: Synced entry logger 0 to disk
>
> I am now investigating why after the first "flushRotatedLogs" bookie is
> slowing down
>
>
Thanks for the update. Eager to learn what is the culprit.

JV


>
> Enrico
>
>
>
>
> >
> > Enrico
> >
> >
> >
> >>
> >>
> >>
> >> - Sijie
> >>
> >> On Thu, Jul 20, 2017 at 4:49 AM, Enrico Olivelli 
> >> wrote:
> >>
> >> > Kishore, do you have news?
> >> >
> >> > Il ven 14 lug 2017, 09:05 Enrico Olivelli  ha
> >> > scritto:
> >> >
> >> > > At the meeting we told the Kishore will perform some benchmarks on
> his
> >> > > side.
> >> > > He will take a look at my code, and we are going to share the
> results.
> >> > > Maybe it will be possible to share the results of benchmarks done
> from
> >> > > Kishore at Salesforce too.
> >> > >
> >> > > The primary goal is to understand the differences between 4.4 and
> 4.5,
> >> > for
> >> > > instance if we there is a need to change JVM/BK configuration in
> >> order to
> >> > > make 4.5 perform as 4.4.
> >> > >
> >> > > @Sijie I hope I have answered your questions.
> >> > >
> >> > >
> >> > > -- Enrico
> >> > >
> >> > >
> >> > > 2017-07-13 9:29 GMT+02:00 Enrico Olivelli :
> >> > >
> >> > >>
> >> > >>
> >> > >> 2017-07-13 4:11 GMT+02:00 Sijie Guo :
> >> > >>
> >> > >>> On Wed, Jul 12, 2017 at 10:35 PM, Enrico Olivelli <
> >> eolive...@gmail.com
> >> > >
> >> > >>> wrote:
> >> > >>>
> >> > >>> > Sijie, JV, just a recap my point of view:
> >> > >>> > - considering latency = "time for asynchAddEntry to complete"
> >> > >>> > - there is a some difference from 4.4 and 4.5 in the usage of
> >> memory,
> >> > >>> but
> >> > >>> > no so clear
> >> > >>> > - the type of GC (parallel vs G1) does not impact very much but
> >> with
> >> > >>> G1 you
> >> > >>> > achieve best latency
> >> > >>> >
> >> > >>>
> >> > >>> Did you try CMS for v3 protocol?
> >> > >>>
> >> > >>
> >> > >> BK 4.5 + v3 procotol + CMS + throttle = 0
> >> > >>
> >> > >> #0 Total wall clock time: 309.0 ms, total callbacks time: 249753
> ms,
> >> > >> entry size 0,034 MB -> 249,75 ms per entry (latency),0,3 ms per
> entry
> >> > >> (throughput) 110,6 MB/s throughput
> >> > >> #1 Total wall clock time: 192.0 ms, total callbacks time: 156268
> ms,
> >> > >> entry size 0,034 MB -> 156,27 ms per entry (latency),0,2 ms per
> entry
> >> > >> (throughput) 178,0 MB/s throughput
> >> > >> #2 Total wall clock time: 129.0 ms, total callbacks time: 95279 ms,
> >> > entry
> >> > >> size 0,034 MB -> 95,28 ms per entry (latency),0,1 ms per entry
> >> > (throughput)
> >> > >> 265,0 MB/s throughput
> >> > >> #3 Total wall clock time: 125.0 ms, total callbacks time: 94661 ms,
> >> > entry
> >> > >> size 0,034 MB -> 94,66 ms per entry (latency),0,1 ms per entry
> >> > (throughput)
> >> > >> 273,4 MB/s throughput
> >> > >> #4 Total wall clock time: 112.0 ms, total callbacks time: 83015 ms,
> >> > entry
> >> > >> size 0,034 MB -> 83,02 ms per entry (latency),0,1 ms per entry
> >> > (throughput)
> >> > >> 305,2 MB/s throughput
> >> > >> #5 Total wall clock time: 111.0 ms, total callbacks time: 82839 ms,
> >> > entry
> >> > >> size 0,034 MB -> 82,84 ms per entry 

Re: ByteBuf memory leak after Netty 4 switch on the client API

2017-07-21 Thread Venkateswara Rao Jujjuri
Can we also add some stats around this to track these. ?  Discipline around
memory release is new concept to Java world.


On Fri, Jul 21, 2017 at 12:59 AM Enrico Olivelli 
wrote:

> Hi,
> I have just file this issue
> https://github.com/apache/bookkeeper/issues/271
>
> After the switch to Netty 4 I noticed this "API change", that is very
> important to be documented and/or to be addresses.
>
> Inside LedgerEntry object we retain a ByteBuf, which is turn is
> automatically 'released' only in this cases:
>
>- getEntry() is called
>- getEntryInputStream() is called and the InputStream is closed
>- the ByteBuf is manually closed by the client
>
> The real tricky thing is that if the client calls readEntry and the
> Enumeration is not fully processed we are going to leak ByteBufs and so
> head/direct memory.
>
> My proposal:
> introduce some "entry reference counting" and ensure that all generated
> entries by a LedgerHandler are "disposed" on LedgerHandler.close() and make
> sure that when BookKeeper client is closed all of the LedgerHandlers
> process their own disposal procedure
>
>
> -- Enrico
>
-- 
Sent from iPhone


Re: Question regarding Synchronization in InterleavedLedgerStorage

2017-07-17 Thread Venkateswara Rao Jujjuri
On Fri, Jul 14, 2017 at 6:00 PM, Sijie Guo  wrote:

>
>
> On Sat, Jul 15, 2017 at 8:06 AM, Charan Reddy G 
> wrote:
>
>> Hey,
>>
>> In InterleavedLedgerStorage, since the initial version of it (
>> https://github.com/apache/bookkeeper/commit/4a94ce1d8184f5f
>> 38def015d80777a8113b96690 and https://github.com/apache/book
>> keeper/commit/d175ada58dcaf78f0a70b0ebebf489255ae67b5f), addEntry and
>> processEntry methods are synchronized. If it is synchronized then I dont
>> get what is the point in having 'writeThreadPool' in
>> BookieRequestProcessor, if anyhow they are going to be executed
>> sequentially because of synchronized addEntry method in
>> InterleavedLedgerStorage.
>>
>
> When InterleavedLedgerStore is used in the context of SortedLedgerStore,
> the addEntry and processEntry are only called when flushing happened. The
> flushing happens in background thread, which is effectively running
> sequentially. But adding to the memtable happens concurrently.
>
> The reason of having 'writeThreadPool' is more on separating writes and
> reads into different thread pools. so writes will not be affected by reads.
> In the context of SortedLedgerStore, the 'writeThreadPool' adds the
> concurrency.
>
>
>>
>> If we look at the implementation of addEntry and processEntry method,
>> 'somethingWritten' can be made threadsafe by using AtomicBoolean,
>> ledgerCache.updateLastAddConfirmed and entryLogger.addEntry methods are
>> inherently threadsafe.
>>
>> I'm not sure about semantics of ledgerCache.putEntryOffset method here.
>> I'm not confident enough to say if LedgerCacheImpl and IndexInMemPageMgr
>> (and probably IndexPersistenceMgr) are thread-safe classes.
>>
>
> LedgerCacheImpl and IndexInMemPageMgr are thread-safe classes. You can
> confirm this from SortedLedgerStorage.
>
>
>>
>> As far as I understood, if ledgerCache.putEntryOffset is thread safe,
>> then I dont see the need of synchronization for those methods. In any case,
>> if they are not thread-safe can you please say why it is not thread-safe
>> and how we can do more granular synchronization at LedgerCacheImpl level,
>> so that we can remove the need of synchrnoization at
>> InterleavedLedgerStorage level.
>>
>
> I don't see any reason why we can't remove the synchronization.
>
>
>>
>> I'm currently working on Multiple Entrylogs -
>> https://issues.apache.org/jira/browse/BOOKKEEPER-1041.
>>
>
> I am wondering if your multiple-entrylogs approach is making things
> complicated. I have been thinking there can be a simpler approach achieving
> the same goal: for example, having a ledger storage comprised of N
> interleaved/sorted ledger storages, which they share same LedgerCache, but
> having different memtables (for sortedledgerstore) and different entry log
> files.
>

This is more cleaner approach. @charan can you comment?

JV


>
>
>> To reap the benefits of multipleentrylogs feature from performance
>> perspective, this synchrnoization should be taken care or atleast bring it
>> down to more granular synchronization (if possible).
>>
>> @Override
>> synchronized public long addEntry(ByteBuffer entry) throws
>> IOException {
>> long ledgerId = entry.getLong();
>> long entryId = entry.getLong();
>> long lac = entry.getLong();
>> entry.rewind();
>> processEntry(ledgerId, entryId, entry);
>> ledgerCache.updateLastAddConfirmed(ledgerId, lac);
>> return entryId;
>> }
>>
>> synchronized protected void processEntry(long ledgerId, long entryId,
>> ByteBuffer entry, boolean rollLog)
>> throws IOException {
>> somethingWritten = true;
>> long pos = entryLogger.addEntry(ledgerId, entry, rollLog);
>> ledgerCache.putEntryOffset(ledgerId, entryId, pos);
>> }
>>
>> Thanks,
>> Charan
>>
>
>


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


Re: Bookie Http Endpoint

2017-07-17 Thread Venkateswara Rao Jujjuri
+ Dustin

On Mon, Jul 17, 2017 at 12:30 PM, Sijie Guo  wrote:

> + Yiming
>
> I would suggest the people who already started the implementations (either
> from Twitter or Salesforce) taking the lead. because they either already
> had the implementation or are working on the implementation. I think the
> goal is to consolidate existing implementations to see if we can have a
> unified effort on this.
>
> - Sijie
>
> On Mon, Jul 17, 2017 at 5:39 PM, Enrico Olivelli 
> wrote:
>
> > Hi all,
> > A discussion has been started about introducing an HTTP endpoint to the
> > bookie.
> > There has been a proposal from Twitter and this is the patch
> > https://github.com/apache/bookkeeper/pull/210
> > On Salesforce there is an ongoing implementation too.
> > I have added that we should provide a Servlet based approach or at least
> > define a public API.
> > We should start a discussion and maybe a BP.
> > We need a leader for the discussion
> >
> > Any volunteer?
> > Enrico
> > --
> >
> >
> > -- Enrico Olivelli
> >
>



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


Re: BookKeeper and code-coverage - JaCoco

2017-07-17 Thread Venkateswara Rao Jujjuri
Adding Dustin to the thread.

On Wed, Jul 5, 2017 at 12:57 AM, Enrico Olivelli 
wrote:

> I am attaching a report (I am using Google Drive as most email clients will
> block attachments with html inside) .
> Just untar and play with your browser
>
>
> 2017-07-04 18:18 GMT+02:00 Dávid Szigecsán :
> > SonarCloud is free for open source.
> >
> > https://about.sonarcloud.io/
>
> Yes I known
> recently we have activated Travis CI, I think that the trend is to
> integrate with such free tools in order to achieve the best quality with
> the minimum effort
>
>
> -- Enrico
>
>
> >
> > 2017-07-04 18:15 GMT+02:00 Enrico Olivelli :
> >
> >> Il mar 4 lug 2017, 18:08 Dávid Szigecsán  ha
> scritto:
> >>
> >> > +1
> >> >
> >> > What do you think about sonar? :)
> >> >
> >>
> >> I am using sonar in projects in my company, it is great.
> >> But it needs a server, so it is not easy to use for everyone
> >>
> >> Enrico
> >>
> >> >
> >> > 2017-07-04 17:24 GMT+02:00 Enrico Olivelli :
> >> >
> >> > > Hi all,
> >> > > as we are in the mood of enhancing automatic QA and quality of
> >> > > BookKeeper I have created this issue in order to introduce JaCoCo in
> >> > > the build/test pipeline
> >> > >
> >> > > https://github.com/apache/bookkeeper/issues/232
> >> > >
> >> > > I will do some tests and report to the comunity
> >> > >
> >> > > -- Enrico
> >> > >
> >> >
> >> --
> >>
> >>
> >> -- Enrico Olivelli
> >>
> ​
>  bookkeeper-server-jacoco.tar.gz
>  drive_web>
> ​
>



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


Re: BeWeekly Community Meeting

2017-07-13 Thread Venkateswara Rao Jujjuri
Today is my duty to drop kids, and I can join only after 8:30. Please go
ahead and I will catch the second half.

Thanks,
JV

On Thu, Jul 13, 2017 at 3:58 AM, Enrico Olivelli 
wrote:

> Hi,
> this is just a reminder and invitation for all the new contributors (like
> David) to our Community Meeting.
> Every two weeks we 'meet' in Hangout to talk about the status of the
> project and the current activity.
>
> Feel free to join, this is the link our Wiki, it contains the updated link
> to the hangout room
> https://cwiki.apache.org/confluence/display/BOOKKEEPER/Community+Meetings
>
> Meeting is at 8am-9am PST  (17.00 for Central Europe, like Italy)
>
> Currently the link is:
> https://goo.gl/iyRA6G
>
> Cheers
>
> -- Enrico
>



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


Re: [GitHub] sijie commented on issue #210: BOOKKEEPER-1100: Add Http Server for Bookkeeper

2017-07-12 Thread Venkateswara Rao Jujjuri
Please add this to the agenda tomorrow. Dustin please plan on attending.

On Wed, Jul 12, 2017 at 7:18 PM  wrote:

> sijie commented on issue #210: BOOKKEEPER-1100: Add Http Server for
> Bookkeeper
> URL: https://github.com/apache/bookkeeper/pull/210#issuecomment-314950842
>
>
>@jvrao @eolivelli @yzang @jiazhai do any of you want to start an email
> thread in dev@ to carry this discussion forward? It doesn't seem that we
> achieve any conclusion on this pull request. It will be good any of you can
> drive this forward otherwise the effort was just dismissed.
>
> 
> This is an automated message from the Apache Git Service.
> To respond to the message, please log on GitHub and use the
> URL above to go to the specific comment.
>
> For queries about this service, please contact Infrastructure at:
> us...@infra.apache.org
>
>
> With regards,
> Apache Git Services
>
-- 
Sent from iPhone


Re: Git protected branches

2017-07-12 Thread Venkateswara Rao Jujjuri
Awesome. !!

On Tue, Jul 11, 2017 at 6:06 AM, Enrico Olivelli 
wrote:

> The ticket has been closed.
> The protection is on
>
> -- Enrico
>
> 2017-07-07 11:22 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > 2017-07-07 1:51 GMT+02:00 Sijie Guo :
> >
> >> +1 for disable force push to master
> >>
> >
> >
> > this is the issue:
> > https://issues.apache.org/jira/browse/INFRA-14535
> >
> > I will send updates
> >
> > - Enrico
> >
> >
> >>
> >> On Thu, Jul 6, 2017 at 1:16 PM, Enrico Olivelli 
> >> wrote:
> >>
> >> > Other thoughts?
> >> > If there is no objection I would like to enable the force push block.
> >> Next
> >> > week
> >> >
> >> > Enrico
> >> >
> >> > Il mar 4 lug 2017, 10:28 Enrico Olivelli  ha
> >> scritto:
> >> >
> >> > > 2017-07-04 5:51 GMT+02:00 Jia Zhai :
> >> > > > Agree to have them protected.
> >> > > > Currently, seems these branches are not protected, at least master
> >> > branch
> >> > > > is not. The merge button is still available even "Jenkins: Maven
> >> clean
> >> > > > install — Build finished." in some of PRs
> >> > > > 
> >> > > > We may first need to make the test stable.
> >> > >
> >> > > I think that is it enough to only protect against "force push",
> which
> >> > > makes impossible for anyone to 'rewrite' the "public history" of the
> >> > > project
> >> > > Other checks are performed by QA test and by the committer who takes
> >> > > responsibility for pushing the patch in the repo
> >> > >
> >> > >
> >> > > -- Enrico
> >> > >
> >> > >
> >> > > >
> >> > > > On Tue, Jul 4, 2017 at 5:10 AM, Enrico Olivelli <
> >> eolive...@gmail.com>
> >> > > wrote:
> >> > > >
> >> > > >> Hi,
> >> > > >> I don't know current configuration but I would like to ensure
> that
> >> our
> >> > > >> release branches, that is master and branch-xx are 'protected'.
> >> > > >> I mean at least that 'force push' must be forbidden to everyone.
> >> > > >>
> >> > > >> In github it is simple to enable such protection, see
> >> > > >> https://help.github.com/articles/about-protected-branches/
> >> > > >>
> >> > > >> I don't want to try btyy performing a force push. Maybe we can
> ask
> >> > infra
> >> > > >> about current configuration
> >> > > >>
> >> > > >> Thoughts?
> >> > > >>
> >> > > >> Enrico
> >> > > >> --
> >> > > >>
> >> > > >>
> >> > > >> -- Enrico Olivelli
> >> > > >>
> >> > >
> >> > --
> >> >
> >> >
> >> > -- Enrico Olivelli
> >> >
> >>
> >
> >
>



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


Re: BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Venkateswara Rao Jujjuri
With Netty changes, lack of native epoll() has huge perf impact as per
Kishore.
Are you sure you are using epoll()?

On Mon, Jul 10, 2017 at 1:49 AM, Enrico Olivelli 
wrote:

> 2017-07-10 10:40 GMT+02:00 Sijie Guo :
>
> > Also one other thing to check is the JVM settings. Do you mind sharing
> that
> > as well?
> >
> >
> this is the surefire config, I am using oracle jdk 8
>
>  
> maven-surefire-plugin
> 2.20
> 
> 1
> false
>
> 300
> -Xmx2G
> -Djava.io.tmpdir=${basedir}/target
> 
> 
>
> -- Enrico
>
>
>
> > Sijie
> >
> > On Jul 10, 2017 1:17 AM, "Sijie Guo"  wrote:
> >
> > > I am not sure if there is any default values changed for journal
> > settings.
> > > I would suggest you testing by setting specifically the journal
> settings.
> > >
> > > Also if you can share your benchmark, that would be good for other
> people
> > > to verify.
> > >
> > > Sijie
> > >
> > > On Jul 10, 2017 12:32 AM, "Enrico Olivelli" 
> wrote:
> > >
> > >> Hi,
> > >> I am doing some benchmarks on BK, I see that from 4.4.0 to 4.5.0 there
> > is
> > >> something "slow" but I cannot understand what. I really hope that I am
> > >> wrong.
> > >>
> > >> I am working with writes, I will pass to reads once writes will be ok.
> > >> My problem is both on latency (time for AddComplete callback to
> > complete)
> > >> and on overall throuput.
> > >>
> > >> Actually I have two distinct problems, but working on the first
> problem
> > I
> > >> found a performance regression.
> > >> I know that talking about "slow" things it is an hard matter, so I
> will
> > >> try
> > >> do describe as much as possible all the aspects that I think are
> > relevant.
> > >>
> > >> First problem: under certain load performance (latency+throughput)
> > degrade
> > >> too much
> > >> Second problem: the first problem is more evident in 4.5.0
> > >>
> > >> Let's describe my testcase and why I am worried.
> > >> The bench issues a batch of asyncAddEntry and prints the average time
> > for
> > >> AddComplete to complete and the overall clock time.
> > >>
> > >> This is the code
> > >>
> > >> private static final byte[] TEST_DATA = new byte[35 * 1024];
> > >> private static final int testsize = 1000;
> > >>
> > >> .. (start 1 bookie, see below)
> > >> ClientConfiguration clientConfiguration = new
> > >> ClientConfiguration();
> > >> clientConfiguration.setZkServers(env.getAddress());
> > >> try (BookKeeper bk = new BookKeeper(clientConfiguration);
> > >> LedgerHandle lh = bk.createLedger(1, 1, 1,
> > >> BookKeeper.DigestType.CRC32, new byte[0])) {
> > >> LongAdder totalTime = new LongAdder();
> > >> long _start = System.currentTimeMillis();
> > >> Collection batch = new
> > >> ConcurrentLinkedQueue<>();
> > >> for (int i = 0; i < testsize; i++) {
> > >> CompletableFuture cf = new CompletableFuture();
> > >> batch.add(cf);
> > >> lh.asyncAddEntry(TEST_DATA, new
> > >> AsyncCallback.AddCallback() {
> > >>
> > >> long start = System.currentTimeMillis();
> > >>
> > >> @Override
> > >> public void addComplete(int rc, LedgerHandle
> lh,
> > >> long entryId, Object ctx) {
> > >> long now =
> > >> System.currentTimeMillis();
> > >> CompletableFuture _cf =
> (CompletableFuture)
> > >> ctx;
> > >> if (rc == BKException.Code.OK) {
> > >> _cf.complete("");
> > >> } else {
> > >>
> > >> _cf.completeExceptionally(BKException.create(rc));
> > >> }
> > >> totalTime.add(now - start);
> > >> }
> > >> }, cf);
> > >> //Thread.sleep(1);  // this is the tirgger!!!
> > >> }
> > >> assertEquals(testsize, batch.size());
> > >> for (CompletableFuture f : batch) {
> > >> f.get();
> > >> }
> > >> long _stop = System.currentTimeMillis();
> > >> long delta = _stop - _start;
> > >> System.out.println("Total time: " + delta + " ms");
> > >> System.out.println("Total real time: " +
> > totalTime.sum() +
> > >> " ms -> "+(totalTime.sum()/testsize)+" ms per entry");
> > >> }
> > >>
> > >> Bookie config:
> > >> ServerConfiguration conf = new ServerConfiguration();
> > >> conf.setBookiePort(5621);
> > >> conf.setUseHostNameAsBookieID(true);
> > >>
> > >> Path targetDir = 

Re: [ANNOUNCE] JV Jujjuri joins the Apache BookKeeper PMC

2017-07-07 Thread Venkateswara Rao Jujjuri
Thank you all!!

On Fri, Jul 7, 2017 at 5:22 AM, Edward Ribeiro 
wrote:

> Congrats, JV!
>
> Ed
>
> On Fri, Jul 7, 2017 at 7:39 AM, Jia Zhai  wrote:
>
> > Congrats, JV!
> >
> > On Fri, Jul 7, 2017 at 6:18 PM, Flavio Junqueira  wrote:
> >
> >> Congrats, JV!
> >>
> >> -F
> >>
> >> On 07 Jul 2017, at 00:11, Charan Reddy G 
> wrote:
> >>
> >> Congrats JV!
> >>
> >> On Thu, Jul 6, 2017 at 9:29 AM, Enrico Olivelli 
> >> wrote:
> >>
> >>> Congrats JV !
> >>>
> >>> Il gio 6 lug 2017, 18:24 Sijie Guo  ha scritto:
> >>>
> >>> > On behalf of the Apache BookKeeper PMC I am pleased to announce that
> JV
> >>> > Jujjuri has accepted our invitation to become a PMC member on the
> >>> Apache
> >>> > BookKeeper project.
> >>> >
> >>> > JV has been an active contributor in many areas, including driving
> >>> > community meetings, reviewing patches, helping with triaging bugs and
> >>> > organizing the bookkeeper meetups. He has the ability and passion to
> >>> move
> >>> > the development of bookkeeper forward and has been speaking of Apache
> >>> > BookKeeper in different conferences and meetups.
> >>> >
> >>> > Please join me in thanking JV for his contributions to date and
> >>> > anticipation of many more contributions.
> >>> >
> >>> > Welcome to the PMC, JV!
> >>> >
> >>> > - Sijie
> >>> >
> >>> --
> >>>
> >>>
> >>> -- Enrico Olivelli
> >>>
> >>
> >>
> >>
> >
>



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


Re: [DISCUSS] BookKeeper - A High Performance and Low Latency Storage Service

2017-07-03 Thread Venkateswara Rao Jujjuri
Everything said on this thread is important and accurate. The description
on the website must be a story rather than a blurb.
We should talk about BK's strengths as Enrico pointed out, and because of
its versatility it became fundamental building block
for various other technologies and usecases. IMO, the entire story is very
powerful and appealing for BK.

On Mon, Jul 3, 2017 at 7:55 AM, Sijie Guo  wrote:

> On Mon, Jul 3, 2017 at 1:35 AM, Enrico Olivelli 
> wrote:
>
> > 2017-07-03 7:00 GMT+02:00 Sijie Guo :
> > > Hi all,
> > >
> > > It has been almost 6-7 years since Apache BookKeeper was born. Apache
> > > BookKeeper has already grown beyond a WAL system. Both Twitter and
> Yahoo
> > > have used it as their storage foundation for their messaging systems,
> > > Salesforce is using it for storage service. We also started talking
> > Apache
> > > BookKeeper as a storage service since 2016 ([1][2]).
> > >
> > > I am thinking of changing the description of Apache BookKeeper from a
> WAL
> > > system to "a High Performance and Low Latency Storage Service (that
> > > optimized for immutable/append-only data)" in the new website that we
> are
> > > building for BP-11
> > >  > action?pageId=71012301>.
> > > This will help us to bring more use cases/adoptions to the project and
> > help
> > > grow the community.
> > >
> > > Any thoughts?
> >
> > My two cents
> >
> > Honestly when I found BookKeeper I was very happy because I found an
> > "original" building block to build replicated state machines.
> > I think that the main soul of BK is exactly to be a WAL and this is
> > really "original".
> >
> > From my point of view the "key features" of BookKeeper are "Fencing"
> > and "Last-Add-Confirmed protocol"
> >
> > BookKeeper is really good at storing data, but IMHO it is because it
> > has been designed and implemented by very skilled engineers,
> > BookKeeper needs to be "fast", because in order to provide a fast WAL
> > you have to give an ultra-fast storage, because the essence of a WAL
> > is  "durability" and usually "durable" comes together with 'sync' and
> > so with 'slow' .
> >
> > I am not a "marketing expert" but IMHO we should stress on the
> > distinctive features of BK in respect to other softwares.
> >
> > I am not against the proposed change but as an user I wanted to point
> > that I happy with BK because it is the most powerful distributed WAL
> > (and maybe it is the unique in the opensource/free world)
> >
> > I would like to write in the website that BookKeeper is the real
> > answer to whom who are looking for a distributed WAL.
>
>
> Agree, we should make a clear case for distributed WAL.
>
> It is worth just putting down all the use cases that BookKeeper has
> supported.
>
> - WAL (e.g. HDFS NameNode)
> - Message Store (e.g. Apache Pulsar, Twitter Pub/Sub via DistributedLog)
> - Offset/Cursor Store (e.g. Apache Pulsar stores cursors in ledgers)
> - Object/Blob Store (e.g. in replicated state machine, storing state
> machine snapshots in ledgers. We used this pattern at distributedlog based
> replicated state machines.)
> - ...
>
> They are not all typical WAL use cases. But the common thing on all these
> use cases - they are using bookkeeper as an append-only/immutable store.
>
> - Sijie
>
>
>
>
> >
> >
> > -- Enrico
> >
> >
> >
> > >
> > > [1]
> > > https://www.slideshare.net/hustlmsp/apache-bookkeeper-a-
> > high-performance-and-low-latency-storage-service
> > > [2] https://www.slideshare.net/jujjuri/apache-con2016final
> >
>



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


  1   2   >