cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jonathan Ellis (Updated) (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-3333) remove more copies from read/write network path
Date Fri, 07 Oct 2011 19:22:29 GMT

     [ https://issues.apache.org/jira/browse/CASSANDRA-3333?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Jonathan Ellis updated CASSANDRA-3333:
--------------------------------------

    Attachment: 0002-reduce-copies.patch
                0001-clean-up-Serializer-mess.patch

01 cleans up the serializer snafu we've been living with
02 actually does the optimization

Prior to this patch we had three generic serializers:

ICompactSerializer, which is used by Messages, and whose de/serialize methods have version
parameters

ICompactSerializer2, which does not have versions and whose methods have DataInput/Output
parameters instead of Input/OutputStreams. (The former is a superset of the later.)

ICompactSerializer3, which extends ICS2 and adds serializedSize.

To do this optimization I'd need ICS4, since I want serializedSize AND versioning.

Instead, I've replaced the three old interfaces with just ISerializer and IVersionedSerializer.
 Both use DataInput/Output, and both have a serializedSize method; the difference is that
the former does not have a version parameter. Implementers that don't care about precomputing
size simply implement as UnsupportedOperation.

A few serializers (in LegacyBloomFilter and MerkleTree) still needed a Stream parameter because
they are doing JDK serialization under the hood. None of that code actually cares about using
a generic Serializer interface, so I just made them one-off classes and everything was happy.
                
> remove more copies from read/write network path
> -----------------------------------------------
>
>                 Key: CASSANDRA-3333
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3333
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Jonathan Ellis
>            Assignee: Jonathan Ellis
>            Priority: Minor
>             Fix For: 1.0.1
>
>         Attachments: 0001-clean-up-Serializer-mess.patch, 0002-reduce-copies.patch
>
>
> RowMutation.serializedBuffer and ReadVerbHandler both do an extra copy of the serialized
data. We can avoid this be pre-computing the serialized size and allocating an appropriate
buffer.

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