[ https://issues.apache.org/jira/browse/BOOKKEEPER-1068?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Enrico Olivelli resolved BOOKKEEPER-1068. ----------------------------------------- Resolution: Fixed Issue resolved by merging pull request 155 [https://github.com/apache/bookkeeper/pull/155] {noformat} commit 7b1eec47092d0de6776c5a89575dbfc678165ee7 Author: Matteo Merli <mme...@yahoo-inc.com> AuthorDate: Thu May 25 14:23:03 2017 +0200 Commit: eolivelli <eolive...@apache.org> CommitDate: Thu May 25 14:23:03 2017 +0200 BOOKKEEPER-1068: Expose ByteBuf in LedgerEntry to avoid data copy To avoid copying the entries payloads when writing/reading on a ledger and having to allocate a lot of `byte[]` on the JVM heap, we need to accept Netty ByteBuf buffer. By passing a ByteBuf, an application can use a pooled buffer, pointing to direct memory, to the `LedgerHandle.addEntry()` and have the same buffer forwarded on the connection sockets to the bookies. The same thing on the read side, `LedgerEntry` exposes an additional `getEntryBuffer()` method that can be used to get the underlying buffer and possibly forward that to some other connection, with zero-copy behavior (excluding getting data in-out of the kernel). Author: Matteo Merli <mme...@yahoo-inc.com> Reviewers: Jia Zhai, Sijie Guo, Enrico Olivelli Closes #155 from merlimat/byte-buf-ledger-entry {noformat} > Expose ByteBuf in LedgerEntry to avoid data copy > ------------------------------------------------ > > Key: BOOKKEEPER-1068 > URL: https://issues.apache.org/jira/browse/BOOKKEEPER-1068 > Project: Bookkeeper > Issue Type: Improvement > Reporter: Matteo Merli > Assignee: Matteo Merli > Fix For: 4.5.0 > > > To avoid copying the entries payloads when writing/reading on a ledger and > having to allocate a lot of byte[] on the JVM heap, we need to accept Netty > ByteBuf buffer. > By passing a ByteBuf, an application can use a pooled buffer, pointing to > direct memory, to the {{LedgerHandle.addEntry()}} and have the same buffer > forwarded on the connection sockets to the bookies. > The same thing on the read side, {{LedgerEntry}} exposes an additional > {{getEntryBuffer()}} method that can be used to get the underlying buffer and > possibly forward that to some other connection, with zero-copy behavior > (excluding getting data in-out of the kernel). -- This message was sent by Atlassian JIRA (v6.3.15#6346)