You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Anoop Sam John (JIRA)" <ji...@apache.org> on 2016/10/28 08:36:58 UTC

[jira] [Comment Edited] (HBASE-16891) Try copying to the Netty ByteBuf directly from the WALEdit

    [ https://issues.apache.org/jira/browse/HBASE-16891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15614756#comment-15614756 ] 

Anoop Sam John edited comment on HBASE-16891 at 10/28/16 8:36 AM:
------------------------------------------------------------------

Good that we can avoid 2 copy now.
But the wrapper is just normal OS only.. We need to make the wrapper object implements, ByteBufferSupportOutputStream.   We have write(BB) API there.   So any way we can directly write DBB to AsyncOuput.  Ya any way this interface will have writeInt() support as well.

Initially when the work was done, it was done as java ByteArrayOS and then as per my comment only it was changed to make hbase implemented ByteArrayOS as that supports BB based APIs.  So here the new wrapper also do that, we are good.. Ya AsyncOuput also need to support BB write.


was (Author: anoop.hbase):
Good that we can avoid 2 copy now.
But the wrapper is just normal OS only.. We need to make the wrapper object implements, ByteBufferSupportOutputStream.   We have write(BB) API there.   So any way we can directly write DBB to AsyncOuput.  Ya any way this interface will have writeInt() support as well.

> Try copying to the Netty ByteBuf directly from the WALEdit
> ----------------------------------------------------------
>
>                 Key: HBASE-16891
>                 URL: https://issues.apache.org/jira/browse/HBASE-16891
>             Project: HBase
>          Issue Type: Sub-task
>          Components: wal
>    Affects Versions: 2.0.0
>            Reporter: ramkrishna.s.vasudevan
>            Assignee: ramkrishna.s.vasudevan
>             Fix For: 2.0.0
>
>         Attachments: HBASE-16891-v1.patch, HBASE-16891.patch
>
>
> -> The FanOutOneBlockAsyncDFSOutput is a much sophisticated dfs client model that works with Netty ByteBuf. Here we hold on connection to the datanodes using
> Netty Channels. And the idea is to write data direclty to these channels.
> AsyncHLog gets an append call. The AysncWAL uses the HBase's ByteArrayOutputSTream and so the content of the cell is written to this BAOS and that is again
> copied to the netty Bytebuf in the FanOutOneBlockAsyncDFSOutput.
> So when the sync call happens this FanoutDFSoutput does the checksum calcualtion itself and then writes the content of this buffer direclty to the DN channel.
> -> In case of FSHLOg this is different. When an append call comes we direclty write the content to the FSDataOutputStream (it is copied to this stream).
> Then here internally there is a checkSum calculation that happens. when a sync call happens there is noth ing to do except to notify the NN to flush the latest
> data.
> AS we can see from the above that there are two copies in AsyncWAL
> -> From the Cell to the BAOS 
> -> From the BAOS to the Netty byte buf
> -> On sync() call, do check sum and finally flush the netty byte buf to the DN channel
> In case of FSHLog
> -> From cell to the FSDataoutputstream. data is copied. Check sum happens here.
> -> Sync call just tries to notify the NN.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)