Re: BookKeeper 4.5.0 performance regression ?

2017-07-28 Thread Enrico Olivelli
Another piece of this story
I have mocked a dummy LedgerStorage (like DummyLedgerStorage in BK tests)
which does nothing (no real disk storage, non memory, no index), no
checkpoint
The problems happens anyway, after a bunch opf seconds (20-30s) the system
slows down
So the  storage is not the problem

-- Enrico


2017-07-26 22:18 GMT+02:00 Enrico Olivelli :

> Any idea?
> What happens after the first checkpoint?
> It seems that the bookies starts to behave in a different way.
> I image that in a first phase all is in memory + journal and then in a
> second phase we start using the disk because the amount of data (index +
> ledger data) cannot be retained in memory.
> I cannot find good metrics, statslogger counter or code points...to
> demonstrate this.
> I would like to understand deeply and to see if the performances I get
> after the first checkpoint are as expected or if this behaviour is weird
> and there is some bug or knob to tune.
>
> Thank you all for your support
> Enrico
>
> Il mar 25 lug 2017, 16:03 Enrico Olivelli  ha
> scritto:
>
>> 2017-07-25 15:36 GMT+02:00 Enrico Olivelli :
>>
>>>
>>>
>>> 2017-07-25 13:58 GMT+02:00 Enrico Olivelli :
>>>
 I noticed that the "performance" drop in my bench happens contextually
 to the opening of several "RandomAccessFile", to .dx files.

 In my bench I continue to perform writes and after some time the
 overall performance (latency and throughtput) "degrade"
 while the bench is running I see that the overall number of open files
 (with "lsof") and the number of open RandomAccessFile (using YourKit
 profile) continue to grow.

 The mechanics in FileInfo are quite complex

>>>
>>>
>>> I have changed setOpenFileLimit to 10 and the Bookie seems respects this
>>> value.
>>> Even if only 10 index files are kept open performances drop anyway
>>>
>>
>>
>> It seems that after the first checkpoint the bookie becomes "slower" but
>> I cannot find the reason.
>>
>> Does anyone else ever noticed this fact ?
>> Are the 'real' bookie performances the ones recorded after the first
>> checkpoint?
>>
>> Enrico
>>
>>
>>
>>
>>>
>>>
>>>
>>>
>>>

 can this be a clue on my problem ?

 This is the code of the bench
 https://github.com/eolivelli/bookkeepers-benchs/blob/
 master/src/test/java/BookKeeperWriteSynchClientsTest.java

 just clone from GitHub and run the test
 on my laptop it starts with 80,0 MB/s throughput and when the "slow
 down" occours it drops to 9 MB/s

 Any suggestion ?

 -- Enrico


 2017-07-24 22:31 GMT+02:00 Enrico Olivelli :

>
>
> Il lun 24 lug 2017, 19:54 Venkateswara Rao Jujjuri 
> ha scritto:
>
>> 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?
>>
>
> Yes this is why linux is better then other OSs, like Windows.
>
>>
>>
>> > 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
>> > flushRotatedLo

Re: BookKeeper 4.5.0 performance regression ?

2017-07-26 Thread Enrico Olivelli
Any idea?
What happens after the first checkpoint?
It seems that the bookies starts to behave in a different way.
I image that in a first phase all is in memory + journal and then in a
second phase we start using the disk because the amount of data (index +
ledger data) cannot be retained in memory.
I cannot find good metrics, statslogger counter or code points...to
demonstrate this.
I would like to understand deeply and to see if the performances I get
after the first checkpoint are as expected or if this behaviour is weird
and there is some bug or knob to tune.

Thank you all for your support
Enrico

Il mar 25 lug 2017, 16:03 Enrico Olivelli  ha scritto:

> 2017-07-25 15:36 GMT+02:00 Enrico Olivelli :
>
>>
>>
>> 2017-07-25 13:58 GMT+02:00 Enrico Olivelli :
>>
>>> I noticed that the "performance" drop in my bench happens contextually
>>> to the opening of several "RandomAccessFile", to .dx files.
>>>
>>> In my bench I continue to perform writes and after some time the overall
>>> performance (latency and throughtput) "degrade"
>>> while the bench is running I see that the overall number of open files
>>> (with "lsof") and the number of open RandomAccessFile (using YourKit
>>> profile) continue to grow.
>>>
>>> The mechanics in FileInfo are quite complex
>>>
>>
>>
>> I have changed setOpenFileLimit to 10 and the Bookie seems respects this
>> value.
>> Even if only 10 index files are kept open performances drop anyway
>>
>
>
> It seems that after the first checkpoint the bookie becomes "slower" but I
> cannot find the reason.
>
> Does anyone else ever noticed this fact ?
> Are the 'real' bookie performances the ones recorded after the first
> checkpoint?
>
> Enrico
>
>
>
>
>>
>>
>>
>>
>>
>>>
>>> can this be a clue on my problem ?
>>>
>>> This is the code of the bench
>>>
>>> https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/BookKeeperWriteSynchClientsTest.java
>>>
>>> just clone from GitHub and run the test
>>> on my laptop it starts with 80,0 MB/s throughput and when the "slow
>>> down" occours it drops to 9 MB/s
>>>
>>> Any suggestion ?
>>>
>>> -- Enrico
>>>
>>>
>>> 2017-07-24 22:31 GMT+02:00 Enrico Olivelli :
>>>


 Il lun 24 lug 2017, 19:54 Venkateswara Rao Jujjuri 
 ha scritto:

> 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?
>

 Yes this is why linux is better then other OSs, like Windows.

>
>
> > 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 <
> eolive...@gmail.com>
> > >> wrote:
>>>

Re: BookKeeper 4.5.0 performance regression ?

2017-07-25 Thread Enrico Olivelli
2017-07-25 15:36 GMT+02:00 Enrico Olivelli :

>
>
> 2017-07-25 13:58 GMT+02:00 Enrico Olivelli :
>
>> I noticed that the "performance" drop in my bench happens contextually to
>> the opening of several "RandomAccessFile", to .dx files.
>>
>> In my bench I continue to perform writes and after some time the overall
>> performance (latency and throughtput) "degrade"
>> while the bench is running I see that the overall number of open files
>> (with "lsof") and the number of open RandomAccessFile (using YourKit
>> profile) continue to grow.
>>
>> The mechanics in FileInfo are quite complex
>>
>
>
> I have changed setOpenFileLimit to 10 and the Bookie seems respects this
> value.
> Even if only 10 index files are kept open performances drop anyway
>


It seems that after the first checkpoint the bookie becomes "slower" but I
cannot find the reason.

Does anyone else ever noticed this fact ?
Are the 'real' bookie performances the ones recorded after the first
checkpoint?

Enrico




>
>
>
>
>
>>
>> can this be a clue on my problem ?
>>
>> This is the code of the bench
>> https://github.com/eolivelli/bookkeepers-benchs/blob/master/
>> src/test/java/BookKeeperWriteSynchClientsTest.java
>>
>> just clone from GitHub and run the test
>> on my laptop it starts with 80,0 MB/s throughput and when the "slow down"
>> occours it drops to 9 MB/s
>>
>> Any suggestion ?
>>
>> -- Enrico
>>
>>
>> 2017-07-24 22:31 GMT+02:00 Enrico Olivelli :
>>
>>>
>>>
>>> Il lun 24 lug 2017, 19:54 Venkateswara Rao Jujjuri 
>>> ha scritto:
>>>
 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?

>>>
>>> Yes this is why linux is better then other OSs, like Windows.
>>>


 > 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 <
 eolive...@gmail.com>
 > >> 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.

Re: BookKeeper 4.5.0 performance regression ?

2017-07-25 Thread Enrico Olivelli
2017-07-25 13:58 GMT+02:00 Enrico Olivelli :

> I noticed that the "performance" drop in my bench happens contextually to
> the opening of several "RandomAccessFile", to .dx files.
>
> In my bench I continue to perform writes and after some time the overall
> performance (latency and throughtput) "degrade"
> while the bench is running I see that the overall number of open files
> (with "lsof") and the number of open RandomAccessFile (using YourKit
> profile) continue to grow.
>
> The mechanics in FileInfo are quite complex
>


I have changed setOpenFileLimit to 10 and the Bookie seems respects this
value.
Even if only 10 index files are kept open performances drop anyway





>
> can this be a clue on my problem ?
>
> This is the code of the bench
> https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/
> BookKeeperWriteSynchClientsTest.java
>
> just clone from GitHub and run the test
> on my laptop it starts with 80,0 MB/s throughput and when the "slow down"
> occours it drops to 9 MB/s
>
> Any suggestion ?
>
> -- Enrico
>
>
> 2017-07-24 22:31 GMT+02:00 Enrico Olivelli :
>
>>
>>
>> Il lun 24 lug 2017, 19:54 Venkateswara Rao Jujjuri 
>> ha scritto:
>>
>>> 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?
>>>
>>
>> Yes this is why linux is better then other OSs, like Windows.
>>
>>>
>>>
>>> > 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 <
>>> eolive...@gmail.com>
>>> > >> 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

Re: BookKeeper 4.5.0 performance regression ?

2017-07-25 Thread Enrico Olivelli
I noticed that the "performance" drop in my bench happens contextually to
the opening of several "RandomAccessFile", to .dx files.

In my bench I continue to perform writes and after some time the overall
performance (latency and throughtput) "degrade"
while the bench is running I see that the overall number of open files
(with "lsof") and the number of open RandomAccessFile (using YourKit
profile) continue to grow.

The mechanics in FileInfo are quite complex

can this be a clue on my problem ?

This is the code of the bench
https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/BookKeeperWriteSynchClientsTest.java

just clone from GitHub and run the test
on my laptop it starts with 80,0 MB/s throughput and when the "slow down"
occours it drops to 9 MB/s

Any suggestion ?

-- Enrico


2017-07-24 22:31 GMT+02:00 Enrico Olivelli :

>
>
> Il lun 24 lug 2017, 19:54 Venkateswara Rao Jujjuri  ha
> scritto:
>
>> 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?
>>
>
> Yes this is why linux is better then other OSs, like Windows.
>
>>
>>
>> > 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 <
>> eolive...@gmail.com>
>> > >> 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 protoc

Re: BookKeeper 4.5.0 performance regression ?

2017-07-24 Thread Enrico Olivelli
Il lun 24 lug 2017, 19:54 Venkateswara Rao Jujjuri  ha
scritto:

> 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?
>

Yes this is why linux is better then other OSs, like Windows.

>
>
> > 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

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 (latency),0,1 ms per entry
> >> > (throughput)
> >> > >> 307,9 MB/s throughput
> >> > >> #6 Total wall clock time: 114.0 ms, total callbacks time: 8

Re: BookKeeper 4.5.0 performance regression ?

2017-07-24 Thread Enrico Olivelli
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.
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


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 (latency),0,1 ms per entry
>> > (throughput)
>> > >> 307,9 MB/s throughput
>> > >> #6 Total wall clock time: 114.0 ms, total callbacks time: 82562 ms,
>> > entry
>> > >> size 0,034 MB -> 82,56 ms per entry (latency),0,1 ms per entry
>> > (throughput)
>> > >> 299,8 MB/s throughput
>> > >> #7 Total wall clock time: 104.0 ms, total callbacks time: 74356 ms,
>> > entry
>> > >> size 0,034 MB -> 74,36 ms per entry (latency),0,1 ms per entry
>> > (throughput)
>> > >> 328,7 MB/s throughput
>> > >> #8 Total wall clock time: 110.0 ms, total callbacks time: 83488 ms,
>> > entry
>> > >> size 0,034 MB -> 83,49 ms per entry (laten

Re: BookKeeper 4.5.0 performance regression ?

2017-07-21 Thread 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

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 (latency),0,1 ms per entry
> > (throughput)
> > >> 307,9 MB/s throughput
> > >> #6 Total wall clock time: 114.0 ms, total callbacks time: 82562 ms,
> > entry
> > >> size 0,034 MB -> 82,56 ms per entry (latency),0,1 ms per entry
> > (throughput)
> > >> 299,8 MB/s throughput
> > >> #7 Total wall clock time: 104.0 ms, total callbacks time: 74356 ms,
> > entry
> > >> size 0,034 MB -> 74,36 ms per entry (latency),0,1 ms per entry
> > (throughput)
> > >> 328,7 MB/s throughput
> > >> #8 Total wall clock time: 110.0 ms, total callbacks time: 83488 ms,
> > entry
> > >> size 0,034 MB -> 83,49 ms per entry (latency),0,1 ms per entry
> > (throughput)
> > >> 310,7 MB/s throughput
> > >> #9 Total wall clock time: 102.0 ms, total callbacks time: 68787 ms,
> > entry
> > >> size 0,034 MB -> 68,79 ms per entry (latency),0,1 ms per entry
> > (throughput)
> > >> 335,1 MB/s throughput
> > >>
> > >> gc.log
> > >> Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
> > >> (1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc
> > 4.3.0
> > >> 20080428 (Red Hat 4.3.0-8)
> > >> Memory: 4k page, physical 32870636k(3276148k free), swap
> > >> 15622140k(15460604k free)
> > >> CommandLine flags: -XX:+CMSIncrementalMode
> > -XX:InitialHeapSize=2147483648
> > >> <(214)%20748-3648> -XX:MaxHeapSize=2147483648 <(214)%20748-3648>
> > >> -XX:MaxNewSize=697933824 -XX:MaxTenuringThreshold=6
> > -XX:NewSize=697933824
> > >> -XX:OldPLABSize=16 -XX:OldSize=1395867648 -XX:+PrintGC
> > >> -XX:+PrintGCTimeStamps -XX:+UseCompressedClassPointers
> > >> -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC
> > >> 0,630: [GC (Allocation Failu

Re: BookKeeper 4.5.0 performance regression ?

2017-07-21 Thread Sijie Guo
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.

- 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  >
> >>> 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 (latency),0,1 ms per entry
> (throughput)
> >> 307,9 MB/s throughput
> >> #6 Total wall clock time: 114.0 ms, total callbacks time: 82562 ms,
> entry
> >> size 0,034 MB -> 82,56 ms per entry (latency),0,1 ms per entry
> (throughput)
> >> 299,8 MB/s throughput
> >> #7 Total wall clock time: 104.0 ms, total callbacks time: 74356 ms,
> entry
> >> size 0,034 MB -> 74,36 ms per entry (latency),0,1 ms per entry
> (throughput)
> >> 328,7 MB/s throughput
> >> #8 Total wall clock time: 110.0 ms, total callbacks time: 83488 ms,
> entry
> >> size 0,034 MB -> 83,49 ms per entry (latency),0,1 ms per entry
> (throughput)
> >> 310,7 MB/s throughput
> >> #9 Total wall clock time: 102.0 ms, total callbacks time: 68787 ms,
> entry
> >> size 0,034 MB -> 68,79 ms per entry (latency),0,1 ms per entry
> (throughput)
> >> 335,1 MB/s throughput
> >>
> >> gc.log
> >> Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
> >> (1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc
> 4.3.0
> >> 20080428 (Red Hat 4.3.0-8)
> >> Memory: 4k page, physical 32870636k(3276148k free), swap
> >> 15622140k(15460604k free)
> >> CommandLine flags: -XX:+CMSIncrementalMode
> -XX:InitialHeapSize=2147483648
> >> <(214)%20748-3648> -XX:MaxHeapSize=2147483648 <(214)%20748-3648>
> >> -XX:MaxNewSize=697933824 -XX:MaxTenuringThreshold=6
> -XX:NewSize=697933824
> >> -XX:OldPLABSize=16 -XX:OldSize=1395867648 -XX:+PrintGC
> >> -XX:+PrintGCTimeStamps -XX:+UseCompressedClassPointers
> >> -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC
> >> 0,630: [GC (Allocation Failure)  545344K->8508K(2029056K), 0,0065802
> secs]
> >> 1,196: [GC (Allocation Failure)  553852K->117940K(2029056K), 0,0385576
> >> secs]
> >> 1,235: [GC (CMS Initial Mark)  119486K(2029056K), 0,0022661 secs]
> >> 1,544: [GC (Allocation Failure)  663284K->207833K(2029056K), 0,0330904
> >> secs]
> >> 1,731: [GC (CMS Final Remark)  560308K(2029056K), 0,0105624 secs]
> >> 1,837: [GC (Allocation Failure)  753177K->214147K(2029056K), 0,0105574
> >> secs]
> >> 2,115: [GC (Allocation Failure)  759491K->241263K(2029056K), 0,0166134
> >> secs]
> >>
> >>
> >> BK 4.5 + v2 protocol + CMS + throttle = 0
> >>
> >> #0 Total wall clock time: 198.0 ms, total callbacks time: 169954 ms,
> >> entry size 0,034 MB 

Re: BookKeeper 4.5.0 performance regression ?

2017-07-20 Thread Enrico Olivelli
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 
>>> 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 (latency),0,1 ms per entry (throughput)
>> 307,9 MB/s throughput
>> #6 Total wall clock time: 114.0 ms, total callbacks time: 82562 ms, entry
>> size 0,034 MB -> 82,56 ms per entry (latency),0,1 ms per entry (throughput)
>> 299,8 MB/s throughput
>> #7 Total wall clock time: 104.0 ms, total callbacks time: 74356 ms, entry
>> size 0,034 MB -> 74,36 ms per entry (latency),0,1 ms per entry (throughput)
>> 328,7 MB/s throughput
>> #8 Total wall clock time: 110.0 ms, total callbacks time: 83488 ms, entry
>> size 0,034 MB -> 83,49 ms per entry (latency),0,1 ms per entry (throughput)
>> 310,7 MB/s throughput
>> #9 Total wall clock time: 102.0 ms, total callbacks time: 68787 ms, entry
>> size 0,034 MB -> 68,79 ms per entry (latency),0,1 ms per entry (throughput)
>> 335,1 MB/s throughput
>>
>> gc.log
>> Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
>> (1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc 4.3.0
>> 20080428 (Red Hat 4.3.0-8)
>> Memory: 4k page, physical 32870636k(3276148k free), swap
>> 15622140k(15460604k free)
>> CommandLine flags: -XX:+CMSIncrementalMode -XX:InitialHeapSize=2147483648
>> <(214)%20748-3648> -XX:MaxHeapSize=2147483648 <(214)%20748-3648>
>> -XX:MaxNewSize=697933824 -XX:MaxTenuringThreshold=6 -XX:NewSize=697933824
>> -XX:OldPLABSize=16 -XX:OldSize=1395867648 -XX:+PrintGC
>> -XX:+PrintGCTimeStamps -XX:+UseCompressedClassPointers
>> -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC
>> 0,630: [GC (Allocation Failure)  545344K->8508K(2029056K), 0,0065802 secs]
>> 1,196: [GC (Allocation Failure)  553852K->117940K(2029056K), 0,0385576
>> secs]
>> 1,235: [GC (CMS Initial Mark)  119486K(2029056K), 0,0022661 secs]
>> 1,544: [GC (Allocation Failure)  663284K->207833K(2029056K), 0,0330904
>> secs]
>> 1,731: [GC (CMS Final Remark)  560308K(2029056K), 0,0105624 secs]
>> 1,837: [GC (Allocation Failure)  753177K->214147K(2029056K), 0,0105574
>> secs]
>> 2,115: [GC (Allocation Failure)  759491K->241263K(2029056K), 0,0166134
>> secs]
>>
>>
>> BK 4.5 + v2 protocol + CMS + throttle = 0
>>
>> #0 Total wall clock time: 198.0 ms, total callbacks time: 169954 ms,
>> entry size 0,034 MB -> 169,95 ms per entry (latency),0,2 ms per entry
>> (throughput) 172,6 MB/s throughput
>> #1 Total wall clock time: 106.0 ms, total callbacks time: 70885 ms, entry
>> size 0,034 MB -> 70,89 ms per entry (latency),0,1 ms per entry (throughput)
>> 322,4 MB/s throughput
>> #2 Total wall clock time: 109.0 ms, total callbacks time: 71848 ms, entry
>> size 0,034 MB -> 71,85 ms per entry (latency),0,1 ms per entry (throughput)
>> 313,6 MB/s throughput
>> #3 Total wall clock time: 97.0 ms, total callbacks time: 73583 ms, entry
>> size 0,034 MB -> 73,58 ms per entry (latency),0,1 ms per entry (throughput)
>> 35

Re: BookKeeper 4.5.0 performance regression ?

2017-07-14 Thread Enrico Olivelli
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 
>> 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 (latency),0,1 ms per entry (throughput)
> 307,9 MB/s throughput
> #6 Total wall clock time: 114.0 ms, total callbacks time: 82562 ms, entry
> size 0,034 MB -> 82,56 ms per entry (latency),0,1 ms per entry (throughput)
> 299,8 MB/s throughput
> #7 Total wall clock time: 104.0 ms, total callbacks time: 74356 ms, entry
> size 0,034 MB -> 74,36 ms per entry (latency),0,1 ms per entry (throughput)
> 328,7 MB/s throughput
> #8 Total wall clock time: 110.0 ms, total callbacks time: 83488 ms, entry
> size 0,034 MB -> 83,49 ms per entry (latency),0,1 ms per entry (throughput)
> 310,7 MB/s throughput
> #9 Total wall clock time: 102.0 ms, total callbacks time: 68787 ms, entry
> size 0,034 MB -> 68,79 ms per entry (latency),0,1 ms per entry (throughput)
> 335,1 MB/s throughput
>
> gc.log
> Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
> (1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc 4.3.0
> 20080428 (Red Hat 4.3.0-8)
> Memory: 4k page, physical 32870636k(3276148k free), swap
> 15622140k(15460604k free)
> CommandLine flags: -XX:+CMSIncrementalMode -XX:InitialHeapSize=2147483648
> <(214)%20748-3648> -XX:MaxHeapSize=2147483648 <(214)%20748-3648>
> -XX:MaxNewSize=697933824 -XX:MaxTenuringThreshold=6 -XX:NewSize=697933824
> -XX:OldPLABSize=16 -XX:OldSize=1395867648 -XX:+PrintGC
> -XX:+PrintGCTimeStamps -XX:+UseCompressedClassPointers
> -XX:+UseCompressedOops -XX:+UseConcMarkSweepGC -XX:+UseParNewGC
> 0,630: [GC (Allocation Failure)  545344K->8508K(2029056K), 0,0065802 secs]
> 1,196: [GC (Allocation Failure)  553852K->117940K(2029056K), 0,0385576
> secs]
> 1,235: [GC (CMS Initial Mark)  119486K(2029056K), 0,0022661 secs]
> 1,544: [GC (Allocation Failure)  663284K->207833K(2029056K), 0,0330904
> secs]
> 1,731: [GC (CMS Final Remark)  560308K(2029056K), 0,0105624 secs]
> 1,837: [GC (Allocation Failure)  753177K->214147K(2029056K), 0,0105574
> secs]
> 2,115: [GC (Allocation Failure)  759491K->241263K(2029056K), 0,0166134
> secs]
>
>
> BK 4.5 + v2 protocol + CMS + throttle = 0
>
> #0 Total wall clock time: 198.0 ms, total callbacks time: 169954 ms, entry
> size 0,034 MB -> 169,95 ms per entry (latency),0,2 ms per entry
> (throughput) 172,6 MB/s throughput
> #1 Total wall clock time: 106.0 ms, total callbacks time: 70885 ms, entry
> size 0,034 MB -> 70,89 ms per entry (latency),0,1 ms per entry (throughput)
> 322,4 MB/s throughput
> #2 Total wall clock time: 109.0 ms, total callbacks time: 71848 ms, entry
> size 0,034 MB -> 71,85 ms per entry (latency),0,1 ms per entry (throughput)
> 313,6 MB/s throughput
> #3 Total wall clock time: 97.0 ms, total callbacks time: 73583 ms, entry
> size 0,034 MB -> 73,58 ms per entry (latency),0,1 ms per entry (throughput)
> 352,4 MB/s throughput
> #4 Total wall clock time: 91.0 ms, total callbacks time: 61293 ms, entry
> size 0,034 MB -> 61,29 ms per entry (latency),0,1 ms per entry (throughput)
> 375,6 MB/s throughput
> #5 Tot

Re: BookKeeper 4.5.0 performance regression ?

2017-07-13 Thread Enrico Olivelli
2017-07-13 4:11 GMT+02:00 Sijie Guo :

> On Wed, Jul 12, 2017 at 10:35 PM, Enrico Olivelli 
> 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 (latency),0,1 ms per entry (throughput)
307,9 MB/s throughput
#6 Total wall clock time: 114.0 ms, total callbacks time: 82562 ms, entry
size 0,034 MB -> 82,56 ms per entry (latency),0,1 ms per entry (throughput)
299,8 MB/s throughput
#7 Total wall clock time: 104.0 ms, total callbacks time: 74356 ms, entry
size 0,034 MB -> 74,36 ms per entry (latency),0,1 ms per entry (throughput)
328,7 MB/s throughput
#8 Total wall clock time: 110.0 ms, total callbacks time: 83488 ms, entry
size 0,034 MB -> 83,49 ms per entry (latency),0,1 ms per entry (throughput)
310,7 MB/s throughput
#9 Total wall clock time: 102.0 ms, total callbacks time: 68787 ms, entry
size 0,034 MB -> 68,79 ms per entry (latency),0,1 ms per entry (throughput)
335,1 MB/s throughput

gc.log
Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
(1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc 4.3.0
20080428 (Red Hat 4.3.0-8)
Memory: 4k page, physical 32870636k(3276148k free), swap
15622140k(15460604k free)
CommandLine flags: -XX:+CMSIncrementalMode -XX:InitialHeapSize=2147483648
-XX:MaxHeapSize=2147483648 -XX:MaxNewSize=697933824
-XX:MaxTenuringThreshold=6 -XX:NewSize=697933824 -XX:OldPLABSize=16
-XX:OldSize=1395867648 -XX:+PrintGC -XX:+PrintGCTimeStamps
-XX:+UseCompressedClassPointers -XX:+UseCompressedOops
-XX:+UseConcMarkSweepGC -XX:+UseParNewGC
0,630: [GC (Allocation Failure)  545344K->8508K(2029056K), 0,0065802 secs]
1,196: [GC (Allocation Failure)  553852K->117940K(2029056K), 0,0385576 secs]
1,235: [GC (CMS Initial Mark)  119486K(2029056K), 0,0022661 secs]
1,544: [GC (Allocation Failure)  663284K->207833K(2029056K), 0,0330904 secs]
1,731: [GC (CMS Final Remark)  560308K(2029056K), 0,0105624 secs]
1,837: [GC (Allocation Failure)  753177K->214147K(2029056K), 0,0105574 secs]
2,115: [GC (Allocation Failure)  759491K->241263K(2029056K), 0,0166134 secs]


BK 4.5 + v2 protocol + CMS + throttle = 0

#0 Total wall clock time: 198.0 ms, total callbacks time: 169954 ms, entry
size 0,034 MB -> 169,95 ms per entry (latency),0,2 ms per entry
(throughput) 172,6 MB/s throughput
#1 Total wall clock time: 106.0 ms, total callbacks time: 70885 ms, entry
size 0,034 MB -> 70,89 ms per entry (latency),0,1 ms per entry (throughput)
322,4 MB/s throughput
#2 Total wall clock time: 109.0 ms, total callbacks time: 71848 ms, entry
size 0,034 MB -> 71,85 ms per entry (latency),0,1 ms per entry (throughput)
313,6 MB/s throughput
#3 Total wall clock time: 97.0 ms, total callbacks time: 73583 ms, entry
size 0,034 MB -> 73,58 ms per entry (latency),0,1 ms per entry (throughput)
352,4 MB/s throughput
#4 Total wall clock time: 91.0 ms, total callbacks time: 61293 ms, entry
size 0,034 MB -> 61,29 ms per entry (latency),0,1 ms per entry (throughput)
375,6 MB/s throughput
#5 Total wall clock time: 98.0 ms, total callbacks time: 73006 ms, entry
size 0,034 MB -> 73,01 ms per entry (latency),0,1 ms per entry (throughput)
348,8 MB/s throughput
#6 Total wall clock time: 104.0 ms, total callbacks time: 75696 ms, entry
size 0,034 MB -> 75,70 ms per entry (latency),0,1 ms per entry (throughput)
328,7 MB/s throughput
#7 Total wall clock time: 103.0 ms, total callbacks time: 79260 ms, entry
size 0,034 MB -> 79,26 ms per entry (latency),0,1 ms per entry (throughput)
331,8 MB/s throughput
#8 Total wall clock time: 92.0 ms, total callbacks time: 62232 ms, entry
size 0,034 MB -> 62,23 ms per entry (latency),0,1 ms per entry (throughput)
371,5 MB/s throughput
#9 Total wall clock time: 98.0 ms, total callbacks time: 

Re: BookKeeper 4.5.0 performance regression ?

2017-07-12 Thread Sijie Guo
On Wed, Jul 12, 2017 at 10:35 PM, Enrico Olivelli 
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?


> - using the v2 protocol brings huge improvements in the usage of memory
> - if you want good "latency" for each entry you have to set at least
> throttle to 1000, but the throughput will be very lower (from 30 MB/s vs
> 300 MB/s on my disk)
>

What type of disk do you have? What is your filesystem mount options?

When you did the test, how did GC activities look like when setting
throttle to be more than 1000?

I think the latency looked "bad" when throttle was more than 1000, is
because of GC. Did you have that test results?


> below the results of my simple bench
>
> Enrico
>
> Bench code:
> https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/
> BookKeeperWriteTest.java
>
> Results:
> with 4.5.0-SNAPSHOT and v2 protocol - G1 GC
>
> #0 Total wall clock time: 1000.0 ms, total callbacks time: 8460 ms, entry
> size 0,034 MB -> 8,46 ms per entry (latency),1,0 ms per entry (throughput)
> 34,2 MB/s throughput
> #1 Total wall clock time: 1005.0 ms, total callbacks time: 4529 ms, entry
> size 0,034 MB -> 4,53 ms per entry (latency),1,0 ms per entry (throughput)
> 34,0 MB/s throughput
> #2 Total wall clock time: 1002.0 ms, total callbacks time: 4293 ms, entry
> size 0,034 MB -> 4,29 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #3 Total wall clock time: 1003.0 ms, total callbacks time: 4317 ms, entry
> size 0,034 MB -> 4,32 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #4 Total wall clock time: 1002.0 ms, total callbacks time: 4553 ms, entry
> size 0,034 MB -> 4,55 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #5 Total wall clock time: 1001.0 ms, total callbacks time: 4448 ms, entry
> size 0,034 MB -> 4,45 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #6 Total wall clock time: 1004.0 ms, total callbacks time: 4449 ms, entry
> size 0,034 MB -> 4,45 ms per entry (latency),1,0 ms per entry (throughput)
> 34,0 MB/s throughput
> #7 Total wall clock time: 1003.0 ms, total callbacks time: 4504 ms, entry
> size 0,034 MB -> 4,50 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #8 Total wall clock time: 1002.0 ms, total callbacks time: 4290 ms, entry
> size 0,034 MB -> 4,29 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #9 Total wall clock time: 1003.0 ms, total callbacks time: 4441 ms, entry
> size 0,034 MB -> 4,44 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
>
> gc.log
> Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
> (1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc 4.3.0
> 20080428 (Red Hat 4.3.0-8)
> Memory: 4k page, physical 32870636k(9153868k free), swap
> 15622140k(15464700k free)
> CommandLine flags: -XX:InitialHeapSize=2147483648
> -XX:MaxHeapSize=2147483648 -XX:+PrintGC -XX:+PrintGCTimeStamps
> -XX:+UseCompressedClassPointers -XX:+UseCompressedOops -XX:+UseG1GC
> 1,988: [GC pause (G1 Evacuation Pause) (young) 146M->59M(2048M), 0,0104300
> secs]
>
>
> Results, with 4.5.0-SNAPSHOT and v3 protocol - G1 GC
> #0 Total wall clock time: 1001.0 ms, total callbacks time: 11800 ms, entry
> size 0,034 MB -> 11,80 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #1 Total wall clock time: 1001.0 ms, total callbacks time: 4585 ms, entry
> size 0,034 MB -> 4,59 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #2 Total wall clock time: 1002.0 ms, total callbacks time: 4458 ms, entry
> size 0,034 MB -> 4,46 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #3 Total wall clock time: 1002.0 ms, total callbacks time: 4471 ms, entry
> size 0,034 MB -> 4,47 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #4 Total wall clock time: 1004.0 ms, total callbacks time: 4473 ms, entry
> size 0,034 MB -> 4,47 ms per entry (latency),1,0 ms per entry (throughput)
> 34,0 MB/s throughput
> #5 Total wall clock time: 1003.0 ms, total callbacks time: 4477 ms, entry
> size 0,034 MB -> 4,48 ms per entry (latency),1,0 ms per entry (throughput)
> 34,1 MB/s throughput
> #6 Total wall clock time: 1007.0 ms, total callbacks time: 4424 ms, entry
> size 0,034 MB -> 4,42 ms per entry (latency),1,0 ms per entry (throughput)
> 33,9 MB/s throughput
> #7 Total wall clock time: 1005.0 ms, total callbacks time: 4576 ms, entry
> size 0,034 MB -> 4,58 ms per entry (latency),1,0 ms per entry (throughput)
> 34,0 MB/s throughput
> #8 Total wall clock time: 1002.0 ms, total callbacks time: 4

Re: BookKeeper 4.5.0 performance regression ?

2017-07-12 Thread Enrico Olivelli
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
- using the v2 protocol brings huge improvements in the usage of memory
- if you want good "latency" for each entry you have to set at least
throttle to 1000, but the throughput will be very lower (from 30 MB/s vs
300 MB/s on my disk)

below the results of my simple bench

Enrico

Bench code:
https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/BookKeeperWriteTest.java

Results:
with 4.5.0-SNAPSHOT and v2 protocol - G1 GC

#0 Total wall clock time: 1000.0 ms, total callbacks time: 8460 ms, entry
size 0,034 MB -> 8,46 ms per entry (latency),1,0 ms per entry (throughput)
34,2 MB/s throughput
#1 Total wall clock time: 1005.0 ms, total callbacks time: 4529 ms, entry
size 0,034 MB -> 4,53 ms per entry (latency),1,0 ms per entry (throughput)
34,0 MB/s throughput
#2 Total wall clock time: 1002.0 ms, total callbacks time: 4293 ms, entry
size 0,034 MB -> 4,29 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#3 Total wall clock time: 1003.0 ms, total callbacks time: 4317 ms, entry
size 0,034 MB -> 4,32 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#4 Total wall clock time: 1002.0 ms, total callbacks time: 4553 ms, entry
size 0,034 MB -> 4,55 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#5 Total wall clock time: 1001.0 ms, total callbacks time: 4448 ms, entry
size 0,034 MB -> 4,45 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#6 Total wall clock time: 1004.0 ms, total callbacks time: 4449 ms, entry
size 0,034 MB -> 4,45 ms per entry (latency),1,0 ms per entry (throughput)
34,0 MB/s throughput
#7 Total wall clock time: 1003.0 ms, total callbacks time: 4504 ms, entry
size 0,034 MB -> 4,50 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#8 Total wall clock time: 1002.0 ms, total callbacks time: 4290 ms, entry
size 0,034 MB -> 4,29 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#9 Total wall clock time: 1003.0 ms, total callbacks time: 4441 ms, entry
size 0,034 MB -> 4,44 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput

gc.log
Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
(1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc 4.3.0
20080428 (Red Hat 4.3.0-8)
Memory: 4k page, physical 32870636k(9153868k free), swap
15622140k(15464700k free)
CommandLine flags: -XX:InitialHeapSize=2147483648
-XX:MaxHeapSize=2147483648 -XX:+PrintGC -XX:+PrintGCTimeStamps
-XX:+UseCompressedClassPointers -XX:+UseCompressedOops -XX:+UseG1GC
1,988: [GC pause (G1 Evacuation Pause) (young) 146M->59M(2048M), 0,0104300
secs]


Results, with 4.5.0-SNAPSHOT and v3 protocol - G1 GC
#0 Total wall clock time: 1001.0 ms, total callbacks time: 11800 ms, entry
size 0,034 MB -> 11,80 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#1 Total wall clock time: 1001.0 ms, total callbacks time: 4585 ms, entry
size 0,034 MB -> 4,59 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#2 Total wall clock time: 1002.0 ms, total callbacks time: 4458 ms, entry
size 0,034 MB -> 4,46 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#3 Total wall clock time: 1002.0 ms, total callbacks time: 4471 ms, entry
size 0,034 MB -> 4,47 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#4 Total wall clock time: 1004.0 ms, total callbacks time: 4473 ms, entry
size 0,034 MB -> 4,47 ms per entry (latency),1,0 ms per entry (throughput)
34,0 MB/s throughput
#5 Total wall clock time: 1003.0 ms, total callbacks time: 4477 ms, entry
size 0,034 MB -> 4,48 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#6 Total wall clock time: 1007.0 ms, total callbacks time: 4424 ms, entry
size 0,034 MB -> 4,42 ms per entry (latency),1,0 ms per entry (throughput)
33,9 MB/s throughput
#7 Total wall clock time: 1005.0 ms, total callbacks time: 4576 ms, entry
size 0,034 MB -> 4,58 ms per entry (latency),1,0 ms per entry (throughput)
34,0 MB/s throughput
#8 Total wall clock time: 1002.0 ms, total callbacks time: 4447 ms, entry
size 0,034 MB -> 4,45 ms per entry (latency),1,0 ms per entry (throughput)
34,1 MB/s throughput
#9 Total wall clock time: 1007.0 ms, total callbacks time: 4501 ms, entry
size 0,034 MB -> 4,50 ms per entry (latency),1,0 ms per entry (throughput)
33,9 MB/s throughput

gc.log
Java HotSpot(TM) 64-Bit Server VM (25.121-b13) for linux-amd64 JRE
(1.8.0_121-b13), built on Dec 12 2016 16:36:53 by "java_re" with gcc 4.3.0
20080428 (Red Hat 4.3.0-8)
Memory: 4k page, physical 32870636k(9042512k free), swap
15622140k(15464700k free)
CommandLine flags: -XX:In

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Sijie Guo
On Wed, Jul 12, 2017 at 2:04 AM, Venkateswara Rao Jujjuri  wrote:

> Enrico, let me try to paraphrase the issue.
>
> - With G1GC + Netty 4.1 is giving you roughly same perf as prev release. Is
> that accurate statement?
> But you are still seeing latency spikes with Netty 4.1??





> I did not fully
> understand your sleep usecase.
> How sleep is yielding better latency?
>

I think the "sleep(1)" in Enrico's benchmark is for throttling. It is same
as setting 'throttle' value to 1000 in his test.

- Sijie



>
> Thanks,
> JV
>
> On Tue, Jul 11, 2017 at 8:27 AM, Enrico Olivelli 
> wrote:
>
> > Another interesting thing...during my profiling activity I gave a chance
> to
> > the old v2 protocol and activated the gc logs, as expected the result is
> > that with v2 protocol there is almost no GC activity during the benchmark
> > -- Enrico
> >
> > 2017-07-11 12:07 GMT+02:00 Enrico Olivelli :
> >
> > >
> > >
> > > 2017-07-11 11:04 GMT+02:00 Sijie Guo :
> > >
> > >> I think Netty4 requires more offheap memory. you might need to tune
> the
> > >> JVM
> > >> settings. I doubt that latency diff coming from the JVM gc.
> > >>
> > >> A simple thing to verify that is to dump the gc log by adding "
> > -Xloggc:"
> > >> setting and compare the gc logs between versions.
> > >>
> > >
> > >
> > > Enabling G1 GC (-XX:+UseG1GC) apparently solves the difference between
> > 4.4
> > > and 4.5.
> > >
> > > And I a loop the bench (in order to have a warm JVM) numbers become
> more
> > > stable and similar to each other
> > >
> > > I have the "other issue" pending, the fact that latency (time for
> > > asynchAddEntry to complete) is really bad and unpredictable (from 100
> ms
> > to
> > > 2000ms)
> > >
> > > and if I introduce a Thread.sleep(1) all the callbacks complete always
> > > with success in < 10ms. This happens even with G1 GC.
> > >
> > > Without the "sleep(1)" my machine uses a lot of CPU (I have 8 "CPUs")
> and
> > > with the "sleep(1)" the load is slightly lower
> > >
> > > Honestly for me this is still a problem and I hope that with your help
> I
> > > will be able to find the problem, wherever it is (in BK code or in the
> > way
> > > I am doing the bench)
> > >
> > > I will try to create a new more complete bench
> > >
> > > -- Enrico
> > >
> > >
> > >
> > >
> > >>
> > >> - Sijie
> > >>
> > >> On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli <
> eolive...@gmail.com>
> > >> wrote:
> > >>
> > >> > a did a bisect and the culprit (in my opinion) is the switch to
> netty
> > 4
> > >> for
> > >> > the performance regression from 4.5 and 4.4
> > >> >
> > >> > at commit:
> > >> > commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058:
> > >> Ignore
> > >> > already deleted ledger on replication audit
> > >> >
> > >> > Total time: 204 ms
> > >> > Total real time: 79 ms per entry
> > >> >
> > >> > at commit:
> > >> > commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008:
> > Netty
> > >> 4.1
> > >> >
> > >> > Total time: 308 ms
> > >> > Total real time: 189 ms per entry
> > >> >
> > >> > I have tried with epoll and with local transport, results does not
> > >> change.
> > >> > I tried to upgrade to netty 4.1.13 too, but no change
> > >> >
> > >> > Could it be  the memory allocator of netty which is overwhelmed with
> > >> sudden
> > >> > bursts of allocation ?
> > >> > I did some trial with UnpooledByteBufAllocator.DEFAULT and it
> helps a
> > >> > little, we get to "110 ms per entry" vs "189 ms per entry"
> > >> >
> > >> > the bench is here:
> > >> > https://github.com/eolivelli/bookkeepers-benchs/blob/master/
> > >> src/test/java/
> > >> > BookKeeperWriteTest.java
> > >> >
> > >> >
> > >> > -- Enrico
> > >> >
> > >> >
> > >> >
> > >> > 2017-07-10 19:46 GMT+02:00 Enrico Olivelli :
> > >> >
> > >> > >
> > >> > >
> > >> > > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri <
> > jujj...@gmail.com
> > >> >
> > >> > ha
> > >> > > scritto:
> > >> > >
> > >> > >> With Netty changes, lack of native epoll() has huge perf impact
> as
> > >> per
> > >> > >> Kishore.
> > >> > >> Are you sure you are using epoll()?
> > >> > >>
> > >> > >
> > >> > > Yes. I tried with netty local transport too. It seems not related
> to
> > >> > netty
> > >> > > to me.
> > >> > > I will double check, tomorrow
> > >> > > Enrico
> > >> > >
> > >> > >
> > >> > >> On Mon, Jul 10, 2017 at 1:49 AM, Enrico Olivelli <
> > >> eolive...@gmail.com>
> > >> > >> 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
> > >> > >> >
> > >> > >> > 30

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Sijie Guo
Enrico,

Do you mind share your gc log between your different runs?

- Sijie

On Tue, Jul 11, 2017 at 11:27 PM, Enrico Olivelli 
wrote:

> Another interesting thing...during my profiling activity I gave a chance to
> the old v2 protocol and activated the gc logs, as expected the result is
> that with v2 protocol there is almost no GC activity during the benchmark
> -- Enrico
>
> 2017-07-11 12:07 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > 2017-07-11 11:04 GMT+02:00 Sijie Guo :
> >
> >> I think Netty4 requires more offheap memory. you might need to tune the
> >> JVM
> >> settings. I doubt that latency diff coming from the JVM gc.
> >>
> >> A simple thing to verify that is to dump the gc log by adding "
> -Xloggc:"
> >> setting and compare the gc logs between versions.
> >>
> >
> >
> > Enabling G1 GC (-XX:+UseG1GC) apparently solves the difference between
> 4.4
> > and 4.5.
> >
> > And I a loop the bench (in order to have a warm JVM) numbers become more
> > stable and similar to each other
> >
> > I have the "other issue" pending, the fact that latency (time for
> > asynchAddEntry to complete) is really bad and unpredictable (from 100 ms
> to
> > 2000ms)
> >
> > and if I introduce a Thread.sleep(1) all the callbacks complete always
> > with success in < 10ms. This happens even with G1 GC.
> >
> > Without the "sleep(1)" my machine uses a lot of CPU (I have 8 "CPUs") and
> > with the "sleep(1)" the load is slightly lower
> >
> > Honestly for me this is still a problem and I hope that with your help I
> > will be able to find the problem, wherever it is (in BK code or in the
> way
> > I am doing the bench)
> >
> > I will try to create a new more complete bench
> >
> > -- Enrico
> >
> >
> >
> >
> >>
> >> - Sijie
> >>
> >> On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli 
> >> wrote:
> >>
> >> > a did a bisect and the culprit (in my opinion) is the switch to netty
> 4
> >> for
> >> > the performance regression from 4.5 and 4.4
> >> >
> >> > at commit:
> >> > commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058:
> >> Ignore
> >> > already deleted ledger on replication audit
> >> >
> >> > Total time: 204 ms
> >> > Total real time: 79 ms per entry
> >> >
> >> > at commit:
> >> > commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008:
> Netty
> >> 4.1
> >> >
> >> > Total time: 308 ms
> >> > Total real time: 189 ms per entry
> >> >
> >> > I have tried with epoll and with local transport, results does not
> >> change.
> >> > I tried to upgrade to netty 4.1.13 too, but no change
> >> >
> >> > Could it be  the memory allocator of netty which is overwhelmed with
> >> sudden
> >> > bursts of allocation ?
> >> > I did some trial with UnpooledByteBufAllocator.DEFAULT and it helps a
> >> > little, we get to "110 ms per entry" vs "189 ms per entry"
> >> >
> >> > the bench is here:
> >> > https://github.com/eolivelli/bookkeepers-benchs/blob/master/
> >> src/test/java/
> >> > BookKeeperWriteTest.java
> >> >
> >> >
> >> > -- Enrico
> >> >
> >> >
> >> >
> >> > 2017-07-10 19:46 GMT+02:00 Enrico Olivelli :
> >> >
> >> > >
> >> > >
> >> > > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri <
> jujj...@gmail.com
> >> >
> >> > ha
> >> > > scritto:
> >> > >
> >> > >> With Netty changes, lack of native epoll() has huge perf impact as
> >> per
> >> > >> Kishore.
> >> > >> Are you sure you are using epoll()?
> >> > >>
> >> > >
> >> > > Yes. I tried with netty local transport too. It seems not related to
> >> > netty
> >> > > to me.
> >> > > I will double check, tomorrow
> >> > > Enrico
> >> > >
> >> > >
> >> > >> On Mon, Jul 10, 2017 at 1:49 AM, Enrico Olivelli <
> >> eolive...@gmail.com>
> >> > >> 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 forkedProcessTimeoutInSeconds>
> >> > >> > -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 Ju

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Venkateswara Rao Jujjuri
Enrico, let me try to paraphrase the issue.

- With G1GC + Netty 4.1 is giving you roughly same perf as prev release. Is
that accurate statement?
But you are still seeing latency spikes with Netty 4.1?? I did not fully
understand your sleep usecase.
How sleep is yielding better latency?

Thanks,
JV

On Tue, Jul 11, 2017 at 8:27 AM, Enrico Olivelli 
wrote:

> Another interesting thing...during my profiling activity I gave a chance to
> the old v2 protocol and activated the gc logs, as expected the result is
> that with v2 protocol there is almost no GC activity during the benchmark
> -- Enrico
>
> 2017-07-11 12:07 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > 2017-07-11 11:04 GMT+02:00 Sijie Guo :
> >
> >> I think Netty4 requires more offheap memory. you might need to tune the
> >> JVM
> >> settings. I doubt that latency diff coming from the JVM gc.
> >>
> >> A simple thing to verify that is to dump the gc log by adding "
> -Xloggc:"
> >> setting and compare the gc logs between versions.
> >>
> >
> >
> > Enabling G1 GC (-XX:+UseG1GC) apparently solves the difference between
> 4.4
> > and 4.5.
> >
> > And I a loop the bench (in order to have a warm JVM) numbers become more
> > stable and similar to each other
> >
> > I have the "other issue" pending, the fact that latency (time for
> > asynchAddEntry to complete) is really bad and unpredictable (from 100 ms
> to
> > 2000ms)
> >
> > and if I introduce a Thread.sleep(1) all the callbacks complete always
> > with success in < 10ms. This happens even with G1 GC.
> >
> > Without the "sleep(1)" my machine uses a lot of CPU (I have 8 "CPUs") and
> > with the "sleep(1)" the load is slightly lower
> >
> > Honestly for me this is still a problem and I hope that with your help I
> > will be able to find the problem, wherever it is (in BK code or in the
> way
> > I am doing the bench)
> >
> > I will try to create a new more complete bench
> >
> > -- Enrico
> >
> >
> >
> >
> >>
> >> - Sijie
> >>
> >> On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli 
> >> wrote:
> >>
> >> > a did a bisect and the culprit (in my opinion) is the switch to netty
> 4
> >> for
> >> > the performance regression from 4.5 and 4.4
> >> >
> >> > at commit:
> >> > commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058:
> >> Ignore
> >> > already deleted ledger on replication audit
> >> >
> >> > Total time: 204 ms
> >> > Total real time: 79 ms per entry
> >> >
> >> > at commit:
> >> > commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008:
> Netty
> >> 4.1
> >> >
> >> > Total time: 308 ms
> >> > Total real time: 189 ms per entry
> >> >
> >> > I have tried with epoll and with local transport, results does not
> >> change.
> >> > I tried to upgrade to netty 4.1.13 too, but no change
> >> >
> >> > Could it be  the memory allocator of netty which is overwhelmed with
> >> sudden
> >> > bursts of allocation ?
> >> > I did some trial with UnpooledByteBufAllocator.DEFAULT and it helps a
> >> > little, we get to "110 ms per entry" vs "189 ms per entry"
> >> >
> >> > the bench is here:
> >> > https://github.com/eolivelli/bookkeepers-benchs/blob/master/
> >> src/test/java/
> >> > BookKeeperWriteTest.java
> >> >
> >> >
> >> > -- Enrico
> >> >
> >> >
> >> >
> >> > 2017-07-10 19:46 GMT+02:00 Enrico Olivelli :
> >> >
> >> > >
> >> > >
> >> > > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri <
> jujj...@gmail.com
> >> >
> >> > ha
> >> > > scritto:
> >> > >
> >> > >> With Netty changes, lack of native epoll() has huge perf impact as
> >> per
> >> > >> Kishore.
> >> > >> Are you sure you are using epoll()?
> >> > >>
> >> > >
> >> > > Yes. I tried with netty local transport too. It seems not related to
> >> > netty
> >> > > to me.
> >> > > I will double check, tomorrow
> >> > > Enrico
> >> > >
> >> > >
> >> > >> On Mon, Jul 10, 2017 at 1:49 AM, Enrico Olivelli <
> >> eolive...@gmail.com>
> >> > >> 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 forkedProcessTimeoutInSeconds>
> >> > >> > -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.
> >

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Enrico Olivelli
Another interesting thing...during my profiling activity I gave a chance to
the old v2 protocol and activated the gc logs, as expected the result is
that with v2 protocol there is almost no GC activity during the benchmark
-- Enrico

2017-07-11 12:07 GMT+02:00 Enrico Olivelli :

>
>
> 2017-07-11 11:04 GMT+02:00 Sijie Guo :
>
>> I think Netty4 requires more offheap memory. you might need to tune the
>> JVM
>> settings. I doubt that latency diff coming from the JVM gc.
>>
>> A simple thing to verify that is to dump the gc log by adding " -Xloggc:"
>> setting and compare the gc logs between versions.
>>
>
>
> Enabling G1 GC (-XX:+UseG1GC) apparently solves the difference between 4.4
> and 4.5.
>
> And I a loop the bench (in order to have a warm JVM) numbers become more
> stable and similar to each other
>
> I have the "other issue" pending, the fact that latency (time for
> asynchAddEntry to complete) is really bad and unpredictable (from 100 ms to
> 2000ms)
>
> and if I introduce a Thread.sleep(1) all the callbacks complete always
> with success in < 10ms. This happens even with G1 GC.
>
> Without the "sleep(1)" my machine uses a lot of CPU (I have 8 "CPUs") and
> with the "sleep(1)" the load is slightly lower
>
> Honestly for me this is still a problem and I hope that with your help I
> will be able to find the problem, wherever it is (in BK code or in the way
> I am doing the bench)
>
> I will try to create a new more complete bench
>
> -- Enrico
>
>
>
>
>>
>> - Sijie
>>
>> On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli 
>> wrote:
>>
>> > a did a bisect and the culprit (in my opinion) is the switch to netty 4
>> for
>> > the performance regression from 4.5 and 4.4
>> >
>> > at commit:
>> > commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058:
>> Ignore
>> > already deleted ledger on replication audit
>> >
>> > Total time: 204 ms
>> > Total real time: 79 ms per entry
>> >
>> > at commit:
>> > commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008: Netty
>> 4.1
>> >
>> > Total time: 308 ms
>> > Total real time: 189 ms per entry
>> >
>> > I have tried with epoll and with local transport, results does not
>> change.
>> > I tried to upgrade to netty 4.1.13 too, but no change
>> >
>> > Could it be  the memory allocator of netty which is overwhelmed with
>> sudden
>> > bursts of allocation ?
>> > I did some trial with UnpooledByteBufAllocator.DEFAULT and it helps a
>> > little, we get to "110 ms per entry" vs "189 ms per entry"
>> >
>> > the bench is here:
>> > https://github.com/eolivelli/bookkeepers-benchs/blob/master/
>> src/test/java/
>> > BookKeeperWriteTest.java
>> >
>> >
>> > -- Enrico
>> >
>> >
>> >
>> > 2017-07-10 19:46 GMT+02:00 Enrico Olivelli :
>> >
>> > >
>> > >
>> > > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri > >
>> > ha
>> > > scritto:
>> > >
>> > >> With Netty changes, lack of native epoll() has huge perf impact as
>> per
>> > >> Kishore.
>> > >> Are you sure you are using epoll()?
>> > >>
>> > >
>> > > Yes. I tried with netty local transport too. It seems not related to
>> > netty
>> > > to me.
>> > > I will double check, tomorrow
>> > > Enrico
>> > >
>> > >
>> > >> On Mon, Jul 10, 2017 at 1:49 AM, Enrico Olivelli <
>> eolive...@gmail.com>
>> > >> 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" <
>> eolive...@gmail.com>
>> > >> > 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 c

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Enrico Olivelli
2017-07-11 11:04 GMT+02:00 Sijie Guo :

> I think Netty4 requires more offheap memory. you might need to tune the JVM
> settings. I doubt that latency diff coming from the JVM gc.
>
> A simple thing to verify that is to dump the gc log by adding " -Xloggc:"
> setting and compare the gc logs between versions.
>


Enabling G1 GC (-XX:+UseG1GC) apparently solves the difference between 4.4
and 4.5.

And I a loop the bench (in order to have a warm JVM) numbers become more
stable and similar to each other

I have the "other issue" pending, the fact that latency (time for
asynchAddEntry to complete) is really bad and unpredictable (from 100 ms to
2000ms)

and if I introduce a Thread.sleep(1) all the callbacks complete always with
success in < 10ms. This happens even with G1 GC.

Without the "sleep(1)" my machine uses a lot of CPU (I have 8 "CPUs") and
with the "sleep(1)" the load is slightly lower

Honestly for me this is still a problem and I hope that with your help I
will be able to find the problem, wherever it is (in BK code or in the way
I am doing the bench)

I will try to create a new more complete bench

-- Enrico




>
> - Sijie
>
> On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli 
> wrote:
>
> > a did a bisect and the culprit (in my opinion) is the switch to netty 4
> for
> > the performance regression from 4.5 and 4.4
> >
> > at commit:
> > commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058:
> Ignore
> > already deleted ledger on replication audit
> >
> > Total time: 204 ms
> > Total real time: 79 ms per entry
> >
> > at commit:
> > commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008: Netty
> 4.1
> >
> > Total time: 308 ms
> > Total real time: 189 ms per entry
> >
> > I have tried with epoll and with local transport, results does not
> change.
> > I tried to upgrade to netty 4.1.13 too, but no change
> >
> > Could it be  the memory allocator of netty which is overwhelmed with
> sudden
> > bursts of allocation ?
> > I did some trial with UnpooledByteBufAllocator.DEFAULT and it helps a
> > little, we get to "110 ms per entry" vs "189 ms per entry"
> >
> > the bench is here:
> > https://github.com/eolivelli/bookkeepers-benchs/blob/
> master/src/test/java/
> > BookKeeperWriteTest.java
> >
> >
> > -- Enrico
> >
> >
> >
> > 2017-07-10 19:46 GMT+02:00 Enrico Olivelli :
> >
> > >
> > >
> > > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri 
> > ha
> > > scritto:
> > >
> > >> With Netty changes, lack of native epoll() has huge perf impact as per
> > >> Kishore.
> > >> Are you sure you are using epoll()?
> > >>
> > >
> > > Yes. I tried with netty local transport too. It seems not related to
> > netty
> > > to me.
> > > I will double check, tomorrow
> > > Enrico
> > >
> > >
> > >> 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" <
> eolive...@gmail.com>
> > >> > 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: un

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Sijie Guo
I think Netty4 requires more offheap memory. you might need to tune the JVM
settings. I doubt that latency diff coming from the JVM gc.

A simple thing to verify that is to dump the gc log by adding " -Xloggc:"
setting and compare the gc logs between versions.

- Sijie

On Tue, Jul 11, 2017 at 12:16 AM, Enrico Olivelli 
wrote:

> a did a bisect and the culprit (in my opinion) is the switch to netty 4 for
> the performance regression from 4.5 and 4.4
>
> at commit:
> commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058: Ignore
> already deleted ledger on replication audit
>
> Total time: 204 ms
> Total real time: 79 ms per entry
>
> at commit:
> commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008: Netty 4.1
>
> Total time: 308 ms
> Total real time: 189 ms per entry
>
> I have tried with epoll and with local transport, results does not change.
> I tried to upgrade to netty 4.1.13 too, but no change
>
> Could it be  the memory allocator of netty which is overwhelmed with sudden
> bursts of allocation ?
> I did some trial with UnpooledByteBufAllocator.DEFAULT and it helps a
> little, we get to "110 ms per entry" vs "189 ms per entry"
>
> the bench is here:
> https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/
> BookKeeperWriteTest.java
>
>
> -- Enrico
>
>
>
> 2017-07-10 19:46 GMT+02:00 Enrico Olivelli :
>
> >
> >
> > Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri 
> ha
> > scritto:
> >
> >> With Netty changes, lack of native epoll() has huge perf impact as per
> >> Kishore.
> >> Are you sure you are using epoll()?
> >>
> >
> > Yes. I tried with netty local transport too. It seems not related to
> netty
> > to me.
> > I will double check, tomorrow
> > Enrico
> >
> >
> >> 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();
> >> > > >> 

Re: BookKeeper 4.5.0 performance regression ?

2017-07-11 Thread Enrico Olivelli
a did a bisect and the culprit (in my opinion) is the switch to netty 4 for
the performance regression from 4.5 and 4.4

at commit:
commit 811ece53a1c975c4e768422f3d622ac9de6b3e41BOOKKEEPER-1058: Ignore
already deleted ledger on replication audit

Total time: 204 ms
Total real time: 79 ms per entry

at commit:
commit 74f795136c1fff3badb29fc982d0cc2d43096b45 BOOKKEEPER-1008: Netty 4.1

Total time: 308 ms
Total real time: 189 ms per entry

I have tried with epoll and with local transport, results does not change.
I tried to upgrade to netty 4.1.13 too, but no change

Could it be  the memory allocator of netty which is overwhelmed with sudden
bursts of allocation ?
I did some trial with UnpooledByteBufAllocator.DEFAULT and it helps a
little, we get to "110 ms per entry" vs "189 ms per entry"

the bench is here:
https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/BookKeeperWriteTest.java


-- Enrico



2017-07-10 19:46 GMT+02:00 Enrico Olivelli :

>
>
> Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri  ha
> scritto:
>
>> With Netty changes, lack of native epoll() has huge perf impact as per
>> Kishore.
>> Are you sure you are using epoll()?
>>
>
> Yes. I tried with netty local transport too. It seems not related to netty
> to me.
> I will double check, tomorrow
> Enrico
>
>
>> 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) {
>> >

Re: BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Enrico Olivelli
Il lun 10 lug 2017, 18:21 Venkateswara Rao Jujjuri  ha
scritto:

> With Netty changes, lack of native epoll() has huge perf impact as per
> Kishore.
> Are you sure you are using epoll()?
>

Yes. I tried with netty local transport too. It seems not related to netty
to me.
I will double check, tomorrow
Enrico


> 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");
> > > >> Syst

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 = path.resolve("bookie_data");
> > >> conf.setZkServers("localhost:1282");
> > >>  

Re: BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Enrico Olivelli
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 = path.resolve("bookie_data");
> >> conf.setZkServers("localhost:1282");
> >> conf.setLedgerDirNames(new
> >> String[]{targetDir.toAbsolutePath().toString()});
> >> conf.setJournalDirName(targetDir.toAbsolutePath().toString());
> >> conf.setFlushInterval(1000);
> >> conf.setJournalFlushWhenQueueEmpty(true);
> >> conf.setProperty("journalMaxGroupWaitMSec", 0);
> >> conf.setProperty("journalBufferedWritesThreshold", 1024);
> >> conf.setAutoRecoveryDaemonEnabled(false);
>

Re: BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Sijie Guo
Also one other thing to check is the JVM settings. Do you mind sharing that
as well?

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 = path.resolve("bookie_data");
>> conf.setZkServers("localhost:1282");
>> conf.setLedgerDirNames(new
>> String[]{targetDir.toAbsolutePath().toString()});
>> conf.setJournalDirName(targetDir.toAbsolutePath().toString());
>> conf.setFlushInterval(1000);
>> conf.setJournalFlushWhenQueueEmpty(true);
>> conf.setProperty("journalMaxGroupWaitMSec", 0);
>> conf.setProperty("journalBufferedWritesThreshold", 1024);
>> conf.setAutoRecoveryDaemonEnabled(false);
>> conf.setEnableLocalTransport(true);
>> conf.setAllowLoopback(true);
>>
>> The tests starts one ZK server + 1 Bookie + the testcase in a JUnit test
>>
>>
>> Results:
>> A - BK-4.4.0:
>> Total time: 209 ms
>> Total real time: 194337 ms -> 194 ms per entry
>>
>> B - BK-4.5.0-SNAPSHOT:
>> Total time: 269 ms
>> Total real time: 239918 ms -> 239 ms per entry
>>
>> C - BK-4.4,0 with sleep(1):
>> Total time: 1113 ms (1000 ms sleep time)
>> Total real time: 4238 ms  -> 4 ms per entry
>>
>> D - BK-4.5,0-SNAPSHOT with sleep(1):
>> Total time: 1121 ms (1000 ms s

Re: BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Enrico Olivelli
2017-07-10 10:17 GMT+02:00 Sijie Guo :

> 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.
>


Sure
this is the test case
https://github.com/eolivelli/bookkeepers-benchs/blob/master/src/test/java/BookKeeperWriteTest.java

just checkout
https://github.com/eolivelli/bookkeepers-benchs

in order to use 4.4.0 just change the pom

Thank you
-- Enrico



>
> 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 = path.resolve("bookie_data");
> > conf.setZkServers("localhost:1282");
> > conf.setLedgerDirNames(new
> > String[]{targetDir.toAbsolutePath().toString()});
> > conf.setJournalDirName(targetDir.toAbsolutePath().toString());
> > conf.setFlushInterval(1000);
> > conf.setJournalFlushWhenQueueEmpty(true);
> > conf.setProperty("journalMaxGroupWaitMSec", 0);
> > conf.setProperty("journalBufferedWritesThreshold", 1024);
> > conf.setAutoRecoveryDaemonEnabled(false);
> > conf.setEnableLocalTransport(true);
> > conf.setAllowLoopback(true);
> >
> > The tests starts one ZK server + 1 Bookie + the testcase in a JUnit test
> >
> >
> > Results:
> > A - BK-4.4.0:
> > Total time: 209 ms
> > Total real time: 194337 ms -> 194 ms per entry
> >
> > B - BK-4.5.0-SNAPSHOT:
> > T

Re: BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Sijie Guo
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 = path.resolve("bookie_data");
> conf.setZkServers("localhost:1282");
> conf.setLedgerDirNames(new
> String[]{targetDir.toAbsolutePath().toString()});
> conf.setJournalDirName(targetDir.toAbsolutePath().toString());
> conf.setFlushInterval(1000);
> conf.setJournalFlushWhenQueueEmpty(true);
> conf.setProperty("journalMaxGroupWaitMSec", 0);
> conf.setProperty("journalBufferedWritesThreshold", 1024);
> conf.setAutoRecoveryDaemonEnabled(false);
> conf.setEnableLocalTransport(true);
> conf.setAllowLoopback(true);
>
> The tests starts one ZK server + 1 Bookie + the testcase in a JUnit test
>
>
> Results:
> A - BK-4.4.0:
> Total time: 209 ms
> Total real time: 194337 ms -> 194 ms per entry
>
> B - BK-4.5.0-SNAPSHOT:
> Total time: 269 ms
> Total real time: 239918 ms -> 239 ms per entry
>
> C - BK-4.4,0 with sleep(1):
> Total time: 1113 ms (1000 ms sleep time)
> Total real time: 4238 ms  -> 4 ms per entry
>
> D - BK-4.5,0-SNAPSHOT with sleep(1):
> Total time: 1121 ms (1000 ms sleep time)
> Total real time: 8018 ms -> 8 ms per entry
>
> Problem 1 (unexpected performance degradation):
> Times per entry (latency) are incredibly slow in cases A and B.
> If I add a sleep(1) between one call of asyncAddEntry and the next
> "latency" is around 4 ms

BookKeeper 4.5.0 performance regression ?

2017-07-10 Thread Enrico Olivelli
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 = path.resolve("bookie_data");
conf.setZkServers("localhost:1282");
conf.setLedgerDirNames(new
String[]{targetDir.toAbsolutePath().toString()});
conf.setJournalDirName(targetDir.toAbsolutePath().toString());
conf.setFlushInterval(1000);
conf.setJournalFlushWhenQueueEmpty(true);
conf.setProperty("journalMaxGroupWaitMSec", 0);
conf.setProperty("journalBufferedWritesThreshold", 1024);
conf.setAutoRecoveryDaemonEnabled(false);
conf.setEnableLocalTransport(true);
conf.setAllowLoopback(true);

The tests starts one ZK server + 1 Bookie + the testcase in a JUnit test


Results:
A - BK-4.4.0:
Total time: 209 ms
Total real time: 194337 ms -> 194 ms per entry

B - BK-4.5.0-SNAPSHOT:
Total time: 269 ms
Total real time: 239918 ms -> 239 ms per entry

C - BK-4.4,0 with sleep(1):
Total time: 1113 ms (1000 ms sleep time)
Total real time: 4238 ms  -> 4 ms per entry

D - BK-4.5,0-SNAPSHOT with sleep(1):
Total time: 1121 ms (1000 ms sleep time)
Total real time: 8018 ms -> 8 ms per entry

Problem 1 (unexpected performance degradation):
Times per entry (latency) are incredibly slow in cases A and B.
If I add a sleep(1) between one call of asyncAddEntry and the next
"latency" is around 4 ms per entry.

Problem 2: worse performance on 4.5.0
Compare A vs B and C vs D, it is self-explaining.

I am running the test on my laptop, with linux 64bit (Fedora), 12 GB RAM,
no swap, on an SSD disk. The results are similar on other computers.

It seems that if I issue too many addEntry the systems slows down.

Please note this fact:
numbers for case A and B (without sleep) mean that all the adds got
completed almost together

for the 4.5 vs 4.4 case:
I tried to disable all of the threadpool enhancements