You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@bookkeeper.apache.org by Enrico Olivelli - Diennea <en...@diennea.com> on 2015/09/14 14:02:46 UTC

Fastest way to write to bookkeeper

Hi,
What is the fastest way to write to BookKeeper a batch of entries ?

I'm using a sequence of asyncAddEntry, some thing like the code below:

List<Long> res= new ArrayList<>(); // holds entry sequence numbers
CountDownLatch latch = new CountDownLatch(size);
for (int i = 0; i < size; i++) {
.....
   this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {

    public void addComplete(int rc, LedgerHandle lh, long entryId, Object i) {
                            int index = (Integer) i;
                            if (rc != BKException.Code.OK) {
                                BKException error = BKException.create(rc);
                                exception.value = error;
                                res.set(index, null);
                                for (int j = 0; j < size; j++) {
                                    // early exit
                                    latch.countDown();
                                }
                            } else {
                                res.set(index, entryId);
                                latch.countDown();
                            }
                        }
                    }, i);
    }
latch.await();

Would it be faster to group all the entries in one "large" entry ? This may alter application semantics but if it would be faster I will do the refactor

Can I file an issue in order to implement a "batchAddEntries" which implements the write of a batch of entries within the native Bookkeeper client ?





Enrico Olivelli
Software Development Manager @Diennea
Tel.: (+39) 0546 066100 - Int. 925
Viale G.Marconi 30/14 - 48018 Faenza (RA)

MagNews - E-mail Marketing Solutions
http://www.magnews.it<http://www.magnews.it/>
Diennea - Digital Marketing Solutions
http://www.diennea.com<http://www.diennea.com/>


________________________________
Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/

R: Fastest way to write to bookkeeper

Posted by Enrico Olivelli - Diennea <en...@diennea.com>.
Thank you for this detailed clarification.

I changed journalMaxGroupWaitMSec to 10 ms and this did the trick in staging environment.

- Enrico



Da: Sijie Guo [mailto:sijie@apache.org]
Inviato: martedì 15 settembre 2015 03:35
A: user@bookkeeper.apache.org
Oggetto: Re: Fastest way to write to bookkeeper

Hmm. This calculation is a bit misleading. Throughput is a matter of the bandwidth, it is dominated by your disk or network. Latency is a matter of bookie group commit interval, how fast that the bookie flushes (syncs) data to disk. High latency doesn't necessary mean low throughput when your request pipeline is purely asynchronous. It is the other way, latency will be impacted when the disk/network is saturated by high throughput.
200ms comes from the default group commit interval in ServerConfiguration, which isn't good for latency-sensitive user cases.
Back to the question, why writing a batch of 100 (500bytes) entries is as fast as writing only one entry. network latency within a data center is usually around millisecond. A hundred of entries couldn't saturate the network, so they would arrive at bookie around millisecond and wait in the remaining 200ms for bookie to committing them. So there wouldn't be any difference between writing 100 entries and 1 entry.
How to make it fast? Are u looking for 'under X MB/second (requests/second), p999 request latency is around Y milliseconds'? or something else?
The dominated factor for bookkeeper add latency is how often a bookie flushes/syncs data to disk, while the dominated factor for bookkeeper throughput is typically disk/network bandwidth.
Improving latency (which I assume people talking about latency when using 'fast'), there are couple of settings in bookie journal that you could tune:

journalMaxGroupWaitMSec (default 200ms): the maximum latency to wait for incoming adds before flushing to disks
journalBufferedWritesThreshold (default 512KB): the maximum bytes to buffer for incoming adds before flushing to disks
journalFlushWhenQueueEmpty (default false): flush the buffered byes when there isn't data in the queue.
Ideally, setting journalFlushWhenQueueEmpty to true will get pretty decent low latency. But since if only flushes data when there isn't incoming data in the queue. But with traffic increased, it introduce varieties on flushing data, which you couldn't predict your latency.
A typical setup would be turn journalFlushWhenQueueEmpty to off and tune journalMaxGroupWaitMSec based on your latency requirement. lower journalMaxGroupWaitMSec would improve your request latency, but it means more filesystem syncs which would limit the throughput (as disk would become the bottleneck); increasing journalMaxGroupWaitMSec would increase your request latency but it also means reducing filesystem syncs, which essentially improve the throughput that a bookie could take.
4-6ms as journalMaxGroupWaitMSec is a good balance between latency and throughput. In additional to journal setting, you could increase numJournalCallbackThreads to support higher throughput.
Does this make sense?
- Sijie



On Mon, Sep 14, 2015 at 10:24 AM, Flavio Junqueira <fp...@apache.org>> wrote:
Ok, so that's 50k per write and you seem to be getting 250 kbytes per second. That's low, you should be able to get higher throughput. We used to get over 20k adds/s of 1kbytes each, which is more like 20Mbytes/s.

-Flavio

On 14 Sep 2015, at 06:49, Enrico Olivelli - Diennea <en...@diennea.com>> wrote:

I did some benchmark and actually writing a batch of 100 (little, 500 bytes) entries using asyncAddEntry is as fast as writing only one entry, that is 200 ms.

I will resolve my problems trying to avoid to single entry writes

Thanks

Enrico Olivelli

Da: Ivan Kelly [mailto:ivank@apache.org]
Inviato: lunedì 14 settembre 2015 15:46
A: user@bookkeeper.apache.org<ma...@bookkeeper.apache.org>
Oggetto: Re: Fastest way to write to bookkeeper

May I suggest, before making any code changes, you measure the difference in MB/s between large writes and small writes? I do recall there was some advantage to using large entries in the past, more than 1k, but I don't remember what it was, and it may not still be true. A lot of code has changed since then.
In theory, anything greater than the MTU shouldn't give too much of a boost.
-Ivan

On Mon, Sep 14, 2015 at 3:35 PM Flavio Junqueira <fp...@apache.org>> wrote:
Hi Enrico,

What's the size of each entry? If they are small say just a few bytes, then you're indeed better off grouping them. If they are 1k or more, then the benefit of grouping shouldn't be much.

About extending the API, the only disadvantage I can see of grouping writes into an entry rather than writing a batch of entries is that a read request will have to read them all. I personally don't like so much the idea of a batch call because it makes the code a bit messier. You need to start a batch, add a bunch of stuff, flush the batch, start a new batch, add a bunch of stuff, and so on. With addEntry, you just invoke it every time you have a new message.

-Flavio

On 14 Sep 2015, at 05:02, Enrico Olivelli - Diennea <en...@diennea.com>> wrote:

Hi,
What is the fastest way to write to BookKeeper a batch of entries ?

I’m using a sequence of asyncAddEntry, some thing like the code below:

List<Long> res= new ArrayList<>(); // holds entry sequence numbers
CountDownLatch latch = new CountDownLatch(size);
for (int i = 0; i < size; i++) {
…..
   this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {

    public void addComplete(int rc, LedgerHandle lh, long entryId, Object i) {
                            int index = (Integer) i;
                            if (rc != BKException.Code.OK) {
                                BKException error = BKException.create(rc);
                                exception.value = error;
                                res.set(index, null);
                                for (int j = 0; j < size; j++) {
                                    // early exit
                                    latch.countDown();
                                }
                            } else {
                                res.set(index, entryId);
                                latch.countDown();
                            }
                        }
                    }, i);
    }
latch.await();

Would it be faster to group all the entries in one “large” entry ? This may alter application semantics but if it would be faster I will do the refactor

Can I file an issue in order to implement a “batchAddEntries” which implements the write of a batch of entries within the native Bookkeeper client ?





Enrico Olivelli
Software Development Manager @Diennea
Tel.: (+39) 0546 066100<tel:%28%2B39%29%C2%A00546%C2%A0066100> - Int. 925
Viale G.Marconi 30/14 - 48018 Faenza (RA)

MagNews - E-mail Marketing Solutions
http://www.magnews.it<http://www.magnews.it/>
Diennea - Digital Marketing Solutions
http://www.diennea.com<http://www.diennea.com/>


________________________________
Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/


________________________________
Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/



________________________________
Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/

Re: Fastest way to write to bookkeeper

Posted by Sijie Guo <si...@apache.org>.
Hmm. This calculation is a bit misleading. Throughput is a matter of the
bandwidth, it is dominated by your disk or network. Latency is a matter of
bookie group commit interval, how fast that the bookie flushes (syncs) data
to disk. High latency doesn't necessary mean low throughput when your
request pipeline is purely asynchronous. It is the other way, latency will
be impacted when the disk/network is saturated by high throughput.

200ms comes from the default group commit interval in ServerConfiguration,
which isn't good for latency-sensitive user cases.

Back to the question, why writing a batch of 100 (500bytes) entries is as
fast as writing only one entry. network latency within a data center is
usually around millisecond. A hundred of entries couldn't saturate the
network, so they would arrive at bookie around millisecond and wait in the
remaining 200ms for bookie to committing them. So there wouldn't be any
difference between writing 100 entries and 1 entry.

How to make it fast? Are u looking for 'under X MB/second
(requests/second), p999 request latency is around Y milliseconds'? or
something else?

The dominated factor for bookkeeper add latency is how often a bookie
flushes/syncs data to disk, while the dominated factor for bookkeeper
throughput is typically disk/network bandwidth.

Improving latency (which I assume people talking about latency when using
'fast'), there are couple of settings in bookie journal that you could tune:

journalMaxGroupWaitMSec (default 200ms): the maximum latency to wait for
incoming adds before flushing to disks
journalBufferedWritesThreshold (default 512KB): the maximum bytes to buffer
for incoming adds before flushing to disks
journalFlushWhenQueueEmpty (default false): flush the buffered byes when
there isn't data in the queue.

Ideally, setting journalFlushWhenQueueEmpty to true will get pretty decent
low latency. But since if only flushes data when there isn't incoming data
in the queue. But with traffic increased, it introduce varieties on
flushing data, which you couldn't predict your latency.

A typical setup would be turn journalFlushWhenQueueEmpty to off and tune
journalMaxGroupWaitMSec based on your latency requirement. lower
journalMaxGroupWaitMSec would improve your request latency, but it means
more filesystem syncs which would limit the throughput (as disk would
become the bottleneck); increasing journalMaxGroupWaitMSec would increase
your request latency but it also means reducing filesystem syncs, which
essentially improve the throughput that a bookie could take.

4-6ms as journalMaxGroupWaitMSec is a good balance between latency and
throughput. In additional to journal setting, you could increase
numJournalCallbackThreads to support higher throughput.

Does this make sense?

- Sijie




On Mon, Sep 14, 2015 at 10:24 AM, Flavio Junqueira <fp...@apache.org> wrote:

> Ok, so that's 50k per write and you seem to be getting 250 kbytes per
> second. That's low, you should be able to get higher throughput. We used to
> get over 20k adds/s of 1kbytes each, which is more like 20Mbytes/s.
>
> -Flavio
>
>
> On 14 Sep 2015, at 06:49, Enrico Olivelli - Diennea <
> enrico.olivelli@diennea.com> wrote:
>
> I did some benchmark and actually writing a batch of 100 (little, 500
> bytes) entries using asyncAddEntry is as fast as writing only one entry,
> that is 200 ms.
>
> I will resolve my problems trying to avoid to single entry writes
>
> Thanks
>
> Enrico Olivelli
>
> *Da:* Ivan Kelly [mailto:ivank@apache.org <iv...@apache.org>]
> *Inviato:* lunedì 14 settembre 2015 15:46
> *A:* user@bookkeeper.apache.org
> *Oggetto:* Re: Fastest way to write to bookkeeper
>
>
> May I suggest, before making any code changes, you measure the difference
> in MB/s between large writes and small writes? I do recall there was some
> advantage to using large entries in the past, more than 1k, but I don't
> remember what it was, and it may not still be true. A lot of code has
> changed since then.
>
> In theory, anything greater than the MTU shouldn't give too much of a
> boost.
> -Ivan
>
> On Mon, Sep 14, 2015 at 3:35 PM Flavio Junqueira <fp...@apache.org> wrote:
>
> Hi Enrico,
>
> What's the size of each entry? If they are small say just a few bytes,
> then you're indeed better off grouping them. If they are 1k or more, then
> the benefit of grouping shouldn't be much.
>
> About extending the API, the only disadvantage I can see of grouping
> writes into an entry rather than writing a batch of entries is that a read
> request will have to read them all. I personally don't like so much the
> idea of a batch call because it makes the code a bit messier. You need to
> start a batch, add a bunch of stuff, flush the batch, start a new batch,
> add a bunch of stuff, and so on. With addEntry, you just invoke it every
> time you have a new message.
>
> -Flavio
>
>
> On 14 Sep 2015, at 05:02, Enrico Olivelli - Diennea <
> enrico.olivelli@diennea.com> wrote:
>
> Hi,
> What is the fastest way to write to BookKeeper a batch of entries ?
>
> I’m using a sequence of asyncAddEntry, some thing like the code below:
>
> List<Long> res= new ArrayList<>(); // holds entry sequence numbers
> CountDownLatch latch = new CountDownLatch(size);
> for (int i = 0; i < size; i++) {
> …..
>    this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {
>
>     public void addComplete(int rc, LedgerHandle lh, long entryId, Object
> i) {
>                             int index = (Integer) i;
>                             if (rc != BKException.Code.OK) {
>                                 BKException error = BKException.create(rc);
>                                 exception.value = error;
>                                 res.set(index, null);
>                                 for (int j = 0; j < size; j++) {
>                                     // early exit
>                                     latch.countDown();
>                                 }
>                             } else {
>                                 res.set(index, entryId);
>                                 latch.countDown();
>                             }
>                         }
>                     }, i);
>     }
> latch.await();
>
> Would it be faster to group all the entries in one “large” entry ? This
> may alter application semantics but if it would be faster I will do the
> refactor
>
> Can I file an issue in order to implement a “batchAddEntries” which
> implements the write of a batch of entries within the native Bookkeeper
> client ?
>
>
>
>
>
> *Enrico Olivelli*
> Software Development Manager @Diennea
> Tel.: (+39) 0546 066100 - Int. 925
> Viale G.Marconi 30/14 - 48018 Faenza (RA)
>
> MagNews - E-mail Marketing Solutions
> http://www.magnews.it
> Diennea - Digital Marketing Solutions
> http://www.diennea.com
>
>
> ------------------------------
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed
> email marketing! http://www.magnews.it/newsletter/
>
>
>
>
> ------------------------------
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed
> email marketing! http://www.magnews.it/newsletter/
>
>
>

Re: Fastest way to write to bookkeeper

Posted by Flavio Junqueira <fp...@apache.org>.
Ok, so that's 50k per write and you seem to be getting 250 kbytes per second. That's low, you should be able to get higher throughput. We used to get over 20k adds/s of 1kbytes each, which is more like 20Mbytes/s.

-Flavio
  
> On 14 Sep 2015, at 06:49, Enrico Olivelli - Diennea <en...@diennea.com> wrote:
> 
> I did some benchmark and actually writing a batch of 100 (little, 500 bytes) entries using asyncAddEntry is as fast as writing only one entry, that is 200 ms.
>  
> I will resolve my problems trying to avoid to single entry writes
>  
> Thanks
>  
> Enrico Olivelli
>  
> Da: Ivan Kelly [mailto:ivank@apache.org] 
> Inviato: lunedì 14 settembre 2015 15:46
> A: user@bookkeeper.apache.org
> Oggetto: Re: Fastest way to write to bookkeeper
>  
> May I suggest, before making any code changes, you measure the difference in MB/s between large writes and small writes? I do recall there was some advantage to using large entries in the past, more than 1k, but I don't remember what it was, and it may not still be true. A lot of code has changed since then.
> 
> In theory, anything greater than the MTU shouldn't give too much of a boost.
> 
> -Ivan
>  
> On Mon, Sep 14, 2015 at 3:35 PM Flavio Junqueira <fpj@apache.org <ma...@apache.org>> wrote:
> Hi Enrico,
>  
> What's the size of each entry? If they are small say just a few bytes, then you're indeed better off grouping them. If they are 1k or more, then the benefit of grouping shouldn't be much.
>  
> About extending the API, the only disadvantage I can see of grouping writes into an entry rather than writing a batch of entries is that a read request will have to read them all. I personally don't like so much the idea of a batch call because it makes the code a bit messier. You need to start a batch, add a bunch of stuff, flush the batch, start a new batch, add a bunch of stuff, and so on. With addEntry, you just invoke it every time you have a new message.
>  
> -Flavio
>  
> On 14 Sep 2015, at 05:02, Enrico Olivelli - Diennea <enrico.olivelli@diennea.com <ma...@diennea.com>> wrote:
>  
> Hi,
> What is the fastest way to write to BookKeeper a batch of entries ?
>  
> I’m using a sequence of asyncAddEntry, some thing like the code below:
>  
> List<Long> res= new ArrayList<>(); // holds entry sequence numbers
> CountDownLatch latch = new CountDownLatch(size);
> for (int i = 0; i < size; i++) {
> …..
>    this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {
>  
>     public void addComplete(int rc, LedgerHandle lh, long entryId, Object i) {
>                             int index = (Integer) i;
>                             if (rc != BKException.Code.OK) {
>                                 BKException error = BKException.create(rc);
>                                 exception.value = error;
>                                 res.set(index, null);
>                                 for (int j = 0; j < size; j++) {
>                                     // early exit
>                                     latch.countDown();
>                                 }
>                             } else {
>                                 res.set(index, entryId);
>                                 latch.countDown();
>                             }
>                         }
>                     }, i);
>     }
> latch.await();
>  
> Would it be faster to group all the entries in one “large” entry ? This may alter application semantics but if it would be faster I will do the refactor
>  
> Can I file an issue in order to implement a “batchAddEntries” which implements the write of a batch of entries within the native Bookkeeper client ?
>  
>  
>  
>  
>  
> Enrico Olivelli 
> Software Development Manager @Diennea 
> Tel.: (+39) 0546 066100 - Int. 925 
> Viale G.Marconi 30/14 - 48018 Faenza (RA) 
> 
> MagNews - E-mail Marketing Solutions
> http://www.magnews.it <http://www.magnews.it/>
> Diennea - Digital Marketing Solutions
> http://www.diennea.com <http://www.diennea.com/>
>  
>  
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/ <http://www.magnews.it/newsletter/>
>  
> 
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/ <http://www.magnews.it/newsletter/>

R: Fastest way to write to bookkeeper

Posted by Enrico Olivelli - Diennea <en...@diennea.com>.
I did some benchmark and actually writing a batch of 100 (little, 500 bytes) entries using asyncAddEntry is as fast as writing only one entry, that is 200 ms.

I will resolve my problems trying to avoid to single entry writes

Thanks

Enrico Olivelli

Da: Ivan Kelly [mailto:ivank@apache.org]
Inviato: lunedì 14 settembre 2015 15:46
A: user@bookkeeper.apache.org
Oggetto: Re: Fastest way to write to bookkeeper

May I suggest, before making any code changes, you measure the difference in MB/s between large writes and small writes? I do recall there was some advantage to using large entries in the past, more than 1k, but I don't remember what it was, and it may not still be true. A lot of code has changed since then.
In theory, anything greater than the MTU shouldn't give too much of a boost.
-Ivan

On Mon, Sep 14, 2015 at 3:35 PM Flavio Junqueira <fp...@apache.org>> wrote:
Hi Enrico,

What's the size of each entry? If they are small say just a few bytes, then you're indeed better off grouping them. If they are 1k or more, then the benefit of grouping shouldn't be much.

About extending the API, the only disadvantage I can see of grouping writes into an entry rather than writing a batch of entries is that a read request will have to read them all. I personally don't like so much the idea of a batch call because it makes the code a bit messier. You need to start a batch, add a bunch of stuff, flush the batch, start a new batch, add a bunch of stuff, and so on. With addEntry, you just invoke it every time you have a new message.

-Flavio

On 14 Sep 2015, at 05:02, Enrico Olivelli - Diennea <en...@diennea.com>> wrote:

Hi,
What is the fastest way to write to BookKeeper a batch of entries ?

I’m using a sequence of asyncAddEntry, some thing like the code below:

List<Long> res= new ArrayList<>(); // holds entry sequence numbers
CountDownLatch latch = new CountDownLatch(size);
for (int i = 0; i < size; i++) {
…..
   this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {

    public void addComplete(int rc, LedgerHandle lh, long entryId, Object i) {
                            int index = (Integer) i;
                            if (rc != BKException.Code.OK) {
                                BKException error = BKException.create(rc);
                                exception.value = error;
                                res.set(index, null);
                                for (int j = 0; j < size; j++) {
                                    // early exit
                                    latch.countDown();
                                }
                            } else {
                                res.set(index, entryId);
                                latch.countDown();
                            }
                        }
                    }, i);
    }
latch.await();

Would it be faster to group all the entries in one “large” entry ? This may alter application semantics but if it would be faster I will do the refactor

Can I file an issue in order to implement a “batchAddEntries” which implements the write of a batch of entries within the native Bookkeeper client ?





Enrico Olivelli
Software Development Manager @Diennea
Tel.: (+39) 0546 066100 - Int. 925
Viale G.Marconi 30/14 - 48018 Faenza (RA)

MagNews - E-mail Marketing Solutions
http://www.magnews.it<http://www.magnews.it/>
Diennea - Digital Marketing Solutions
http://www.diennea.com<http://www.diennea.com/>


________________________________
Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/


________________________________
Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/

Re: Fastest way to write to bookkeeper

Posted by Ivan Kelly <iv...@apache.org>.
May I suggest, before making any code changes, you measure the difference
in MB/s between large writes and small writes? I do recall there was some
advantage to using large entries in the past, more than 1k, but I don't
remember what it was, and it may not still be true. A lot of code has
changed since then.

In theory, anything greater than the MTU shouldn't give too much of a boost.

-Ivan

On Mon, Sep 14, 2015 at 3:35 PM Flavio Junqueira <fp...@apache.org> wrote:

> Hi Enrico,
>
> What's the size of each entry? If they are small say just a few bytes,
> then you're indeed better off grouping them. If they are 1k or more, then
> the benefit of grouping shouldn't be much.
>
> About extending the API, the only disadvantage I can see of grouping
> writes into an entry rather than writing a batch of entries is that a read
> request will have to read them all. I personally don't like so much the
> idea of a batch call because it makes the code a bit messier. You need to
> start a batch, add a bunch of stuff, flush the batch, start a new batch,
> add a bunch of stuff, and so on. With addEntry, you just invoke it every
> time you have a new message.
>
> -Flavio
>
> On 14 Sep 2015, at 05:02, Enrico Olivelli - Diennea <
> enrico.olivelli@diennea.com> wrote:
>
> Hi,
> What is the fastest way to write to BookKeeper a batch of entries ?
>
> I’m using a sequence of asyncAddEntry, some thing like the code below:
>
> List<Long> res= new ArrayList<>(); // holds entry sequence numbers
> CountDownLatch latch = new CountDownLatch(size);
> for (int i = 0; i < size; i++) {
> …..
>    this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {
>
>     public void addComplete(int rc, LedgerHandle lh, long entryId, Object
> i) {
>                             int index = (Integer) i;
>                             if (rc != BKException.Code.OK) {
>                                 BKException error = BKException.create(rc);
>                                 exception.value = error;
>                                 res.set(index, null);
>                                 for (int j = 0; j < size; j++) {
>                                     // early exit
>                                     latch.countDown();
>                                 }
>                             } else {
>                                 res.set(index, entryId);
>                                 latch.countDown();
>                             }
>                         }
>                     }, i);
>     }
> latch.await();
>
> Would it be faster to group all the entries in one “large” entry ? This
> may alter application semantics but if it would be faster I will do the
> refactor
>
> Can I file an issue in order to implement a “batchAddEntries” which
> implements the write of a batch of entries within the native Bookkeeper
> client ?
>
>
>
>
>
> *Enrico Olivelli*
> Software Development Manager @Diennea
> Tel.: (+39) 0546 066100 - Int. 925
> Viale G.Marconi 30/14 - 48018 Faenza (RA)
>
> MagNews - E-mail Marketing Solutions
> http://www.magnews.it
> Diennea - Digital Marketing Solutions
> http://www.diennea.com
>
>
> ------------------------------
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed
> email marketing! http://www.magnews.it/newsletter/
>
>
>

Re: Fastest way to write to bookkeeper

Posted by Flavio Junqueira <fp...@apache.org>.
Hi Enrico,

What's the size of each entry? If they are small say just a few bytes, then you're indeed better off grouping them. If they are 1k or more, then the benefit of grouping shouldn't be much.

About extending the API, the only disadvantage I can see of grouping writes into an entry rather than writing a batch of entries is that a read request will have to read them all. I personally don't like so much the idea of a batch call because it makes the code a bit messier. You need to start a batch, add a bunch of stuff, flush the batch, start a new batch, add a bunch of stuff, and so on. With addEntry, you just invoke it every time you have a new message.

-Flavio

> On 14 Sep 2015, at 05:02, Enrico Olivelli - Diennea <en...@diennea.com> wrote:
> 
> Hi,
> What is the fastest way to write to BookKeeper a batch of entries ?
>  
> I’m using a sequence of asyncAddEntry, some thing like the code below:
>  
> List<Long> res= new ArrayList<>(); // holds entry sequence numbers
> CountDownLatch latch = new CountDownLatch(size);
> for (int i = 0; i < size; i++) {
> …..
>    this.out.asyncAddEntry(entry, new AsyncCallback.AddCallback() {
>  
>     public void addComplete(int rc, LedgerHandle lh, long entryId, Object i) {
>                             int index = (Integer) i;
>                             if (rc != BKException.Code.OK) {
>                                 BKException error = BKException.create(rc);
>                                 exception.value = error;
>                                 res.set(index, null);
>                                 for (int j = 0; j < size; j++) {
>                                     // early exit
>                                     latch.countDown();
>                                 }
>                             } else {
>                                 res.set(index, entryId);
>                                 latch.countDown();
>                             }
>                         }
>                     }, i);
>     }
> latch.await();
>  
> Would it be faster to group all the entries in one “large” entry ? This may alter application semantics but if it would be faster I will do the refactor
>  
> Can I file an issue in order to implement a “batchAddEntries” which implements the write of a batch of entries within the native Bookkeeper client ?
>  
>  
>  
>  
>  
> Enrico Olivelli 
> Software Development Manager @Diennea 
> Tel.: (+39) 0546 066100 - Int. 925 
> Viale G.Marconi 30/14 - 48018 Faenza (RA) 
> 
> MagNews - E-mail Marketing Solutions
> http://www.magnews.it <http://www.magnews.it/>
> Diennea - Digital Marketing Solutions
> http://www.diennea.com <http://www.diennea.com/>
>  
> 
> Iscriviti alla nostra newsletter per rimanere aggiornato su digital ed email marketing! http://www.magnews.it/newsletter/ <http://www.magnews.it/newsletter/>