[
https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17113873#comment-17113873
]
Sam Tunnicliffe commented on CASSANDRA-15299:
---------------------------------------------
[~omichallat], that's great, thanks!
I've fixed the server side to switch to framing only after sending the
{{READY/AUTHENTICATE}} response and tested with your branch. I'd also already
fixed the off-by-one error that was affecting compression, so that's working
for v5 now too. It appears to be broken for v4 though, which is also the case
running the 2772 branch against an unpatched trunk C*, but not a vanilla 3.6.0.
There's a few client facing things still missing from the server side:
* Backpressure when overloaded
* Closing connection on corrupt frames
* Handling frames where compression is skipped
There's also still quite a bit of internal stuff to be done, but this is pretty
encouraging. Thanks again!
> CASSANDRA-13304 follow-up: improve checksumming and compression in protocol
> v5-beta
> -----------------------------------------------------------------------------------
>
> Key: CASSANDRA-15299
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15299
> Project: Cassandra
> Issue Type: Improvement
> Components: Messaging/Client
> Reporter: Aleksey Yeschenko
> Assignee: Sam Tunnicliffe
> Priority: Normal
> Labels: protocolv5
> Fix For: 4.0-alpha
>
>
> CASSANDRA-13304 made an important improvement to our native protocol: it
> introduced checksumming/CRC32 to request and response bodies. It’s an
> important step forward, but it doesn’t cover the entire stream. In
> particular, the message header is not covered by a checksum or a crc, which
> poses a correctness issue if, for example, {{streamId}} gets corrupted.
> Additionally, we aren’t quite using CRC32 correctly, in two ways:
> 1. We are calculating the CRC32 of the *decompressed* value instead of
> computing the CRC32 on the bytes written on the wire - losing the properties
> of the CRC32. In some cases, due to this sequencing, attempting to decompress
> a corrupt stream can cause a segfault by LZ4.
> 2. When using CRC32, the CRC32 value is written in the incorrect byte order,
> also losing some of the protections.
> See https://users.ece.cmu.edu/~koopman/pubs/KoopmanCRCWebinar9May2012.pdf for
> explanation for the two points above.
> Separately, there are some long-standing issues with the protocol - since
> *way* before CASSANDRA-13304. Importantly, both checksumming and compression
> operate on individual message bodies rather than frames of multiple complete
> messages. In reality, this has several important additional downsides. To
> name a couple:
> # For compression, we are getting poor compression ratios for smaller
> messages - when operating on tiny sequences of bytes. In reality, for most
> small requests and responses we are discarding the compressed value as it’d
> be smaller than the uncompressed one - incurring both redundant allocations
> and compressions.
> # For checksumming and CRC32 we pay a high overhead price for small messages.
> 4 bytes extra is *a lot* for an empty write response, for example.
> To address the correctness issue of {{streamId}} not being covered by the
> checksum/CRC32 and the inefficiency in compression and checksumming/CRC32, we
> should switch to a framing protocol with multiple messages in a single frame.
> I suggest we reuse the framing protocol recently implemented for internode
> messaging in CASSANDRA-15066 to the extent that its logic can be borrowed,
> and that we do it before native protocol v5 graduates from beta. See
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/FrameDecoderCrc.java
> and
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/FrameDecoderLZ4.java.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]