hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Todd Lipcon (Commented) (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-5355) Compressed RPC's for HBase
Date Wed, 08 Feb 2012 22:04:59 GMT

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

Todd Lipcon commented on HBASE-5355:
------------------------------------

I've been mulling this over in the back of my head recently with regards to the work just
getting started on adding extensible RPC. Here are a few thoughts:

A lot of our current lack of efficiency can be dealt with by simply avoiding multiple copies
of the same data. The most egregious example: when we serialize columns, we serialize each
KeyValue indepedendently, even though they all share the same row key.
One potential solution I've been thinking about:
- introduce the concept of a "constant pool" which is associated with an RPC request or response.
This pool would be serialized on the wire before the actual request/response and might be
encoded like:
{code}
<total byte length of constant pool> <number of constants> <constant 1 len>
<constant 1 val> <constant 2 len> <constant 2 val>
{code}
Then in the actual serialization of KeyValues, etc, we would not write out the data, but rather
indexes into the constant pool.
The advantages to this kind of technique would be:
- pushing all of the data near each other in the packet would make any compression more beneficial
(rather than interleaving compressible user data with less compressible encoded information)
- allows multiple parts of a request/response to reference the same byte arrays (eg multiple
columns referring to the same row key)
- allows zero-copy implementations even if we use protobufs to encode the actual call/response

This idea might be orthogonal to the compression discussed above, but may be a cheaper (CPU-wise)
way of getting a similar effect.
                
> Compressed RPC's for HBase
> --------------------------
>
>                 Key: HBASE-5355
>                 URL: https://issues.apache.org/jira/browse/HBASE-5355
>             Project: HBase
>          Issue Type: Improvement
>          Components: ipc
>    Affects Versions: 0.89.20100924
>            Reporter: Karthik Ranganathan
>            Assignee: Karthik Ranganathan
>
> Some application need ability to do large batched writes and reads from a remote MR cluster.
These eventually get bottlenecked on the network. These results are also pretty compressible
sometimes.
> The aim here is to add the ability to do compressed calls to the server on both the send
and receive paths.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message