cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Brandon Williams (JIRA)" <j...@apache.org>
Subject [jira] [Issue Comment Edited] (CASSANDRA-1788) reduce copies on read, write paths
Date Wed, 20 Jul 2011 20:59:58 GMT

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

Brandon Williams edited comment on CASSANDRA-1788 at 7/20/11 8:58 PM:
----------------------------------------------------------------------

So, rebased v6 is even rarer now.  I now have to do reads/writes to the cluster to trigger
it, where before it looked like the occasional gossip message would cause it.  And even doing
reads/writes, it's still quite rare: out of the 333k (three node cluster, rf=1, 1M total)
inserts/reads to the patched node, only 16 occurrences.  When the patched node is the only
coordinator, it never produces an exception on reads, however for writes it increases the
amount of exceptions, nearly 60 out of 1M inserts.  I suspect there is a problem in ITC or
Message where it's not reading something correctly, but difficult to trigger. I confirmed
with wireshark the other nodes are sending correct messages.

      was (Author: brandon.williams):
    So, rebased v6 is even rarer now.  I now have to do reads/writes to the cluster to trigger
it, where before it looked like the occasional gossip message would cause it.  And even doing
reads/writes, it's still quite rare: out of the 333k (three node cluster, rf=1, 1M total)
inserts/reads to the patched node, only 16 occurrences.  When the patched node is the only
coordinator, it never produces an exception on reads, however for writes it increases the
amount of exceptions, nearly 60 out of 1M inserts.  I suspect there is a problem in ITC or
Message where it's not reading something correctly, but difficult to trigger.
  
> reduce copies on read, write paths
> ----------------------------------
>
>                 Key: CASSANDRA-1788
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1788
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Core
>            Reporter: Jonathan Ellis
>            Assignee: Jonathan Ellis
>            Priority: Minor
>             Fix For: 1.0
>
>         Attachments: 0001-setup.txt, 0002-remove-copies-from-network-path.txt, 1788-v2.txt,
1788-v3.txt, 1788-v4.txt, 1788-v6.txt, 1788.txt
>
>   Original Estimate: 24h
>  Remaining Estimate: 24h
>
> Currently, we do _three_ unnecessary copies (that is, writing to the socket is necessary;
any other copies made are overhead) for each message:
> - constructing the Message body byte[] (this is typically a call to a ICompactSerializer[2]
serialize method, but sometimes we cheat e.g. in SchemaCheckVerbHandler's reply)
> - which is copied to a buffer containing the entire Message (i.e. including Header) when
sendOneWay calls Message.serializer.serialize()
> - which is copied to a newly-allocated ByteBuffer when sendOneWay calls packIt
> - which is what we write to the socket
> For deserialize we perform a similar orgy of copies:
> - IncomingTcpConnection reads the Message length, allocates a byte[], and reads the serialized
Message into it
> - ITcpC then calls Message.serializer().deserialize, which allocates a new byte[] for
the body and copies that part
> - finally, the verbHandler (determined by the now-deserialized Message header) deserializes
the actual object represented by the body
> Most of these are out of scope for 0.7 but I think we can at least elide the last copy
on the write path and the first on the read.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message