[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17239313#comment-17239313 ] Alex Petrov commented on CASSANDRA-15299: - [~beobal] I've reviewed the latest changes (renaming and addressing comments) and the patch looks good to me with a single discardable comment in the patch (disambiguate Compressor/Decompressor class names), please feel free to ignore it. +1, LGTM and thank you for the patch! > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17231691#comment-17231691 ] Sam Tunnicliffe commented on CASSANDRA-15299: - I found 1 dtest which broke when updating the driver to current master (e1fc528a0d), and opened CASSANDRA-16275 to fix it. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17231559#comment-17231559 ] Sam Tunnicliffe commented on CASSANDRA-15299: - [~aholmber] that's great, thanks! I'll make sure everything's passing against trunk with the latest driver and get back to you. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17231545#comment-17231545 ] Adam Holmberg commented on CASSANDRA-15299: --- {quote}This is the thing I'm unsure about - the python driver is not an asf project (yet), so it's not really up to me/us whether we update that branch ... TBH, I don't recall the full reasoning for using the cassandra-test branch in the first place ...{quote} The {{cassandra-test}} branch was created explicitly to allow server-side testing of merged client-impacting features, independent of driver releases. It was born of a time when there were multiple client-impacting changes in-flight so a single commit would not suffice. Normally we would coordinate a merge into that driver branch with a PR for the server, so CI could be tested with it. Updating the branch should not be an issue. I, or [~aboudreault] can facilitate. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17231525#comment-17231525 ] Michael Semb Wever commented on CASSANDRA-15299: bq. Absolutely, I published them myself to test, but I definitely think we should have them under /apache. Done, ref: INFRA-21103. Let's see how that goes. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17231489#comment-17231489 ] Sam Tunnicliffe commented on CASSANDRA-15299: - {quote}If we're updating to use a new updated version of the driver, does that mean the cassandra-test branch is being sync'd up to master in the progress?{quote} This is the thing I'm unsure about - the python driver is not an asf project (yet), so it's not really up to me/us whether we update that branch (of course I can open a PR to the driver to do that). Binding cassandra-dtest to a specific published commit is something wholly within our wheelhouse though, so that was the route I took. TBH, I don't recall the full reasoning for using the cassandra-test branch in the first place (backports I suppose). {quote}Is it time to start deploying these images under apache/ ? If agreed, I can open an infra ticket to set up deployment of docker images.{quote} Absolutely, I published them myself to test, but I definitely think we should have them under /apache. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17231462#comment-17231462 ] Michael Semb Wever commented on CASSANDRA-15299: bq. One thing that's a bit concerning is that the cassandra-test branch of the driver, which is what dtests are currently using, is currently 693 commits behind the master branch. If we're updating to use a new updated version of the driver, does that mean the {{cassandra-test}} branch being sync'd up to master in the progress? {quote}Docker images: - beobal/cassandra-testing-ubuntu1910-java11:2020 - beobal/cassandra-testing-ubuntu1910-java11-w-dependencies:2020{quote} Is it time to start deploying these images under [{{apache/}}|https://hub.docker.com/u/apache] ? If agreed, I can open an infra ticket to set up deployment of docker images. bq. I'll open PRs to cassandra-builds and cassandra-dtest before going any further here. Go for it! :-) > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17230079#comment-17230079 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Something we'll need to do in order to merge this is to update the python dtests to use a driver which supports the new protocol. Initial support has been committed, but not yet released, so I propose updating dtests to pull a specific commit from the driver repo. I thought it should be possible to do this just by updating {{requirements.txt}} in the dtest repo, but unfortunately not. While this does cause the container to pull an updated version, but it only wipes the source files, not the compiled {{.c}} and {{.so}} objects from the installation and there are incompatibilities which cause {{ImportError}} when trying to run the dtests. I spoke to [~eduard.tudenhoefner] about this and published new docker images with the updated driver installed ([e1fc528a0|https://github.com/datastax/python-driver/commit/e1fc528a0deec67f725f066204e8c1ea8b26bbde]). Docker images: * [beobal/cassandra-testing-ubuntu1910-java11:2020|https://hub.docker.com/layers/beobal/cassandra-testing-ubuntu1910-java11/2020/images/sha256-6c4bdcb82c6b25ec30495f3a49188206fbad99d5c8083217d576a8a84a610bf4?context=repo] * [beobal/cassandra-testing-ubuntu1910-java11-w-dependencies:2020|https://hub.docker.com/layers/beobal/cassandra-testing-ubuntu1910-java11-w-dependencies/2020/images/sha256-beda323a6fbba94dc6c2953f5cd5f95d71dfc3dafbd605470ecb0d4282fb0fa8?context=repo] This works as expected, but there is one failing dtest, which looks to be related so I'd like to open a follow up JIRA to fix that. One thing that's a bit concerning is that the {{cassandra-test}} branch of the driver, which is what dtests are currently using, is currently 693 commits behind the master branch. But, if there are no objections to updating in this way, I'll open PRs to {{cassandra-builds}} and {{cassandra-dtest}} before going any further here. cc: [~mck] [~aboudreault] [~brandon.williams] > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17229401#comment-17229401 ] Caleb Rackliffe commented on CASSANDRA-15299: - The rename looks great. (Just dropped a [couple|https://github.com/apache/cassandra/commit/f29c4120477de1c322a84a2cf00aace17a0da263#r44051143] [tiny|https://github.com/apache/cassandra/commit/f29c4120477de1c322a84a2cf00aace17a0da263#r44050926] typo comments.) > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17227575#comment-17227575 ] Sam Tunnicliffe commented on CASSANDRA-15299: - I just pushed a commit which renames {{o.a.c.t.Frame}} to {{Envelope}}, which IMO this greatly reduces the cognitive friction here. There are no client facing changes involved, the renaming is purely internal (aside from docs, I've updated the WIP asciidoc on V5 framing, but will get the main protocol spec in CASSANDRA-14688, asap) > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17226898#comment-17226898 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Thanks, [~ifesdjeen], the flow chart is definitely useful (and looks correct to me, aside from the missing labels that Caleb mentioned). Re: your comments... {quote}in CQLMessageHeader#processLargeMessage, we seem to be creating a LargeMessage object to call processCqlFrame() over a frame that is assembled from a single byte buffer. Maybe we can just call processCqlFrame directly?{quote} thanks, removed in {{ce6223cb}} {quote}this is probably something that we should address outside this ticket, but still: "corrupt frame recovered" seems to be slightly misleading wording, same as Frame#recoverable. We can not recover the frame itself, we just can skip/drop it. Maybe we can rename this, along with metrics in Messaging, before they become public in 4.0.{quote} I tend to agree, maybe something which conveys that the frames are skipped/dropped rather than recovered would be better? We can open a new JIRA for this. {quote}in CQLMessageHeader#processCqlFrame, we only call handleErrorAndRelease. However, it may theoretically happen that we fail before we finish messageDecoder.decode(channel, frame). Maybe we can do something like the [1], to make it consistent with what we do in ProtocolDecoder#decode?{quote} Good call, added in {{8dbf7e42}}. {quote}in CQLMessageHeader$LargeMessage#onComplete, we wrap processCqlFrame(assembleFrame() call in try/catch which is identical to try/catch block from processCqlFrame itself. Just checking if this is intended.{quote} no, it was a redundancy left over from earlier, removed it in {{8dbf7e42}}. {quote}in Dispatcher#processRequest, we can slightly simplify code by declaring FlushItem flushItem variable outside try/catch block and assigning a corresponding value in try or catch, and only calling flush once.{quote} done in {{8dbf7e42}} {quote}during sever bootstrap initialization, we're using deprecated low/high watermark child options, probably we should use .childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(8 * 1024, 32 * 1024)) instead.{quote} Thanks, fixed in {{8dbf7e42}} {quote}SimpleClientBurnTest#random is unused{quote} thanks, removed. [~maedhroz], thanks, those are decent improvements and seem to me worth making. {quote}testChangingLimitsAtRuntime has a bunch of places where it checks the same value twice, for instance from ClientResourceLimits.getGlobalLimit() and then DatabaseDescriptor.getNativeTransportMaxConcurrentRequestsInBytes(). Seems like we could do away with the second?{quote} Thanks, it is redundant for the global limit, as the {{ClientResourceLimits}} method just delegates to {{DatabaseDescriptor}}, which was always the case I think. For the per-endpoint limits, it's definitely worth checking both places to ensure existing and new limits observe the same cap. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17226492#comment-17226492 ] Caleb Rackliffe commented on CASSANDRA-15299: - [~ifesdjeen] I think the flow chart is certainly helpful. A couple of very minor things... * The arrows from "Is first frame?" look like they are missing "Yes"/"No" labels. * Once the frame naming decisions are finalized, there might be a couple labels that should be updated. > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17226048#comment-17226048 ] Alex Petrov commented on CASSANDRA-15299: - Patch looks good to me, I have only several comments: * in {{CQLMessageHeader#processLargeMessage}}, we seem to be creating a {{LargeMessage}} object to call {{processCqlFrame()}} over a frame that is assembled from a single byte buffer. Maybe we can just call {{processCqlFrame}} directly? * this is probably something that we should address outside this ticket, but still: "corrupt frame recovered" seems to be slightly misleading wording, same as {{Frame#recoverable}}. We can not recover the frame itself, we just can skip/drop it. Maybe we can rename this, along with metrics in Messaging, before they become public in 4.0. * in {{CQLMessageHeader#processCqlFrame}}, we only call {{handleErrorAndRelease}}. However, it may theoretically happen that we fail before we finish {{messageDecoder.decode(channel, frame)}}. Maybe we can do something like the [1], to make it consistent with what we do in {{ProtocolDecoder#decode}}? * in {{CQLMessageHeader$LargeMessage#onComplete}}, we wrap {{processCqlFrame(assembleFrame()}} call in try/catch which is identical to try/catch block from {{processCqlFrame}} itself. Just checking if this is intended. * in {{Dispatcher#processRequest}}, we can slightly simplify code by declaring {{FlushItem flushItem}} variable outside try/catch block and assigning a corresponding value in try or catch, and only calling {{flush}} once. * during sever bootstrap initialization, we're using deprecated low/high watermark child options, probably we should use {{.childOption(ChannelOption.WRITE_BUFFER_WATER_MARK, new WriteBufferWaterMark(8 * 1024, 32 * 1024))}} instead. * {{SimpleClientBurnTest#random}} is unused If this is helpful, I've also put nits mentioned above (and a couple cleanups) in a commit [here|https://github.com/apache/cassandra/commit/76ee6e00f42446d94679e9c9001c81ebfa9418ab]. Not sure if this is helpful, but I've also put together a v5 flow chart, which might be helpful if anyone wants a quick overview of what's going on in v5. [1] {code} protected void processCqlFrame(Frame frame) { M message = null; try { message = messageDecoder.decode(channel, frame); dispatcher.accept(channel, message, this::toFlushItem); } catch (Exception e) { if (message == null) frame.release(); handleErrorAndRelease(e, frame.header); } } {code} > 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 > > Attachments: Process CQL Frame.png, V5 Flow Chart.png > > > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17223241#comment-17223241 ] Caleb Rackliffe commented on CASSANDRA-15299: - Alright, LGTM (assuming nothing crazy happens around the perf test reruns). Minor nits (that may or may not be worth fixing) around {{ClientResourceLimitsTest}}: - {{testQueryExecutionWithThrowOnOverload}} and {{testQueryExecutionWithoutThrowOnOverload}} are identical with the exception of the {{throwOnOverload}} argument to the new {{SimpleClient}}. - {{testOverloadedExceptionWhenGlobalLimitExceeded}} and the three tests that follow it are also largely duplicates and we could probably factor out a helper below the initial limit setting. - {{testOverloadedExceptionWhenGlobalLimitByMultiFrameMessage}} and its endpoint limit analog might want to use a large message threshold relative to {{LOW_LIMIT}} rather than something hard-coded (in this case 100). {{testChangingLimitsAtRuntime}} has a bunch of places where it checks the same value twice, for instance from {{ClientResourceLimits.getGlobalLimit()}} and then {{DatabaseDescriptor.getNativeTransportMaxConcurrentRequestsInBytes()}}. Seems like we could do away with the second? > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17222843#comment-17222843 ] Sam Tunnicliffe commented on CASSANDRA-15299: - {quote}p99 latency spike between 4.0-beta2 V4 and 15299 V5 w/ LZ4 {quote} my bad sorry, I realise I'd missed the labels indicating which columns were for which operation (added now). So I guess you're talking about the spike in write latency in the r/w/d workload here, I'll re-run the tests and see if it reproduces. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17222476#comment-17222476 ] Caleb Rackliffe commented on CASSANDRA-15299: - Reading through the overallocation commit, it seems we might be able to factor a little helper out of {{Decoder#extractHeader()}} and {{decodeFrame()}} that decodes and validates the header flags. ex. {noformat} private EnumSet decodeFlags(ProtocolVersion version, int flags) { EnumSet decodedFlags = Header.Flag.deserialize(flags); if (version.isBeta() && !decodedFlags.contains(Header.Flag.USE_BETA)) throw new ProtocolException(String.format("Beta version of the protocol used (%s), but USE_BETA flag is unset", version), version); return decodedFlags; } {noformat} > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17222394#comment-17222394 ] Caleb Rackliffe commented on CASSANDRA-15299: - bq. I don't think anything here indicates a serious regression in either v4 or v5 performance. Generally, I agree. The only bit that looks sort of interesting is the p99 latency spike between 4.0-beta2 V4 and 15299 V5 w/ LZ4. Write latencies are lower though, so maybe prioritization effects help explain that. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17220921#comment-17220921 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Here are some numbers from running tlp-stress against 4.0-beta2 vs the 15299 branch {{70923bae4}} with protocol v4 & v5. The dataset was sized to be mostly in-memory and reading/writing at {{CL.ONE}}. Each workload ran for 10 minutes, with a 1m warmup, on a 3 node cluster. Both latency and throughput are pretty much on a par, with v5 showing a bit of an improvement where the workload is amenable to compression. Memory usage and GC were pretty much the same too. If anything, during the v5 runs the servers spent less time in GC, but this was so close as not be significant. There are definitely some places we can improve the performance of v5, and I don't think anything here indicates a serious regression in either v4 or v5 performance. Given the additional integrity checks in v5, I think not regressing meets the perf bar here, at least in the first instance. h4. 100% read workload, reads a single row at a time. {code} Count Latency (p99) 1min (req/s) 4.0-beta2 V4135449878 72.43 224222.86 15299 V4138424112 68.3 229765.45 15299 V5137618437 70.35 231602.36 4.0-beta2 V4 LZ4103348953 66.68 173437.38 15299 V4 LZ4105114560 68.83 176192.36 15299 V5 LZ4131833462 70.19 222092.99 {code} h4. Mixed r/w workload (50/50). K/V with blobs upto 65k {code} Count Latency (p99) 1min (req/s) | Count Latency (p99) 1min (req/s) 4.0-beta2 V434455009 76.59 56557.78 | 34464217 74.85 56546.40 15299 V433368171 74.90 54859.94 | 33361940 67.77 54848.57 15299 V532991815 76.00 54780.74 | 33001153 76.72 54856.99 4.0-beta2 V4 LZ432152220 83.41 53306.03 | 32147206 83.22 53334.00 15299 V4 LZ431158895 71.01 51106.60 | 31153453 72.75 51087.01 15299 V5 LZ432634296 75.73 54370.71 | 32644765 76.70 54396.15 {code} h4. Mixed r/w/d workload (60/30/10). Wide rows with values up to 200b and slicing on both the inserts and deletions. {code} Count Latency (p99) 1min (req/s) | Count Latency (p99) 1min (req/s) | Count Latency (p99) 1min (req/s) 4.0-beta2 V418725688 197.47 25377.16 | 9357971 193.86 12687.94 | 3117394 178.44 4221.90 15299 V417975636 185.88 24160.78 | 8986125 184.97 12087.13 | 2995562 179.99 4023.88 15299 V518429252 192.91 25349.35 | 9223277 188.15 12678.46 | 3073312 184.24 4232.23 4.0-beta2 V4 LZ418407719 179.94 25160.16 | 9197664 178.25 12575.03 | 3068134 180.90 4195.38 15299 V4 LZ417678994 171.39 24073.09 | 8842952 196.06 12064.18 | 2947344 170.53 4026.37 15299 V5 LZ418274085 208.57 25127.02 | 9138491 163.23 12558.28 | 3045264 203.54 4188.88 {code} > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17219275#comment-17219275 ] Sam Tunnicliffe commented on CASSANDRA-15299: - [~maedhroz], thanks for the initial comments and apologies for the delay in responding. I've pushed an update to the branch with some rework to the backpressure mechanism and resource management/book keeping that popped up during some perf testing (explanations in the commit messages) I've also made a start on your review comments, covering the *Correctness*, *Testing*, *Zombies* and *Questions* sections (see below). I'll have more patches for the remaining sections tomorrow. {quote}Are we incrementing receivedCount both on the first frame of a large message (in processFirstFrameOfLargeMessage()) and on the last frame (in processSubsequentFrameOfLargeMessage())?{quote} Good catch, removed. {quote}forceOverAllocation() only allocates enough to hit the limit, but is it possible for the subsequent release to de-allocate the full frame size?{quote} Removed forceOverAllocation in b8251bf3b157f29dbed258aa2795728964db7804 {quote}Flusher uses a PayloadAllocator from either the CRC or LZ4 encoder. CRC includes the header and trailer size when it hits the BufferPool, while LZ4 uses the Payload constructor that doesn't take those into account. Is this correct?{quote} It is correct. Using the CRC encoder, you know the exact size of the payload in advance, so we allocate a buffer exactly the right size (payload + header + trailer). The producer then writes directly to that buffer and the encoder updates it in place with the header/trailer when done. For LZ4, this isn’t possible as we don’t know the compressed size up front. So in this case, the payload is sized for the frame content and the encoder allocates another buffer once it does know the compressed size. {quote}PostV5ExceptionHandler - in exceptionCaught(), does payload.release() need to be in a finally block?{quote} Added a finally block {quote}CQLMessageHandler - should processOneContainedMessage() release the frame in a finally block?{quote} Not applicable anymore as we only construct the frame if we’re processing it {quote}It isn't really a correctness issue per-se, but FrameSet would feel safer if we used composition rather than inheritance. (Was the motivator not creating the additional object?){quote} As far as I remember, there was no major motivation for it other than we needed a List with some mildly specialized behaviour (I think I had recently been doing something in OutboundConnection, where Deliver extends AtomicInteger). I don’t really mind changing it to use composition, but why do you say that would feel safer? (Totally agreed we should rename to XList though, but I will hold off until actually renaming Frame -> X) {quote}CQLTester - It might be a little awkward to follow the semantics of "require" -> "prepare" -> "initialize". Maybe we should just inline prepareNetwork().{quote} Yep, I don’t recall exactly why I broke it out into 3, rather than 2 pieces. Inlined prepare in require. {quote}The large frame accumulation state machine might be a good unit test target. Lowest level might be a test-only subclass of AbstractMessageHandler.LargeMessage that simulates the cases we're interested in. Another would be to go one level up and test the CQLMessageHandler.LargeMessage, but that pulls all of CQLMessageHandler in, as it's not a static class. (We could make it static, create a little interface to abstract CQLMessageHandler away, etc.){quote} I’ll follow up on using a test-only subclass to exercise the accumulation. {quote}It might be worth pulling up WaitQueue (which is already static) as a top-level class, then throw a few unit-level tests around it it.{quote} I agree that it might be nice to unit test this, but I’m not sure I would prioritize it, given that the class is exercised by existing unit and burn tests, plus has been used during extensive perf testing of the messaging subsystem when validating CASSANDRA-15066 {quote}StartupMessage - There's enough logic that it might be nice to unit test execute(). (Mocking Connection and checking replays, etc.){quote} Similarly, I agree that we could add direct test coverage for this but I don’t see it as super high priority as all the functionality should be covered by other unit or dtests (I will check that though). {quote}transport.Frame.Header.BODY_LENGTH_SIZE is unused{quote} It’s used again now that I reverted some of the changes to Frame.Decoder in b8251bf3b157f29dbed258aa2795728964db7804 {quote}PasswordAuthenticatorTest - import java.net.InetSocketAddress is unused{quote} Removed, thanks {quote}OutboundConnection - import java.util.stream.Stream is unused{quote} I don’t think I’ve touched that file {quote}StressSettings - import com.datastax.driver.core.Metadata and
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17211482#comment-17211482 ] Caleb Rackliffe commented on CASSANDRA-15299: - [~samt] Here are the detailed/tactical points from my first pass at review. Some of these items, especially the testing bits, are things I could potentially branch and work on a bit myself, so let me know if you'd like a hand. I still want to make one more top-down pass at things Monday now that the smaller details are clearer. *Correctness* - {{AbstractMessageHandler}} -- Are we incrementing {{receivedCount}} both on the first frame of a large message (in {{processFirstFrameOfLargeMessage()}}) and on the last frame (in {{processSubsequentFrameOfLargeMessage()}})? -- {{forceOverAllocation()}} only allocates enough to hit the limit, but is it possible for the subsequent release to de-allocate the full frame size? - {{Flusher}} uses a {{PayloadAllocator}} from either the CRC or LZ4 encoder. CRC includes the header and trailer size when it hits the {{BufferPool}}, while LZ4 uses the {{Payload}} constructor that doesn't take those into account. Is this correct? - {{PostV5ExceptionHandler}} - in {{exceptionCaught()}}, does {{payload.release()}} need to be in a {{finally}} block? - {{CQLMessageHandler}} - should {{processOneContainedMessage()}} release the frame in a {{finally}} block? - It isn't really a correctness issue per-se, but {{FrameSet}} would feel safer if we used composition rather than inheritance. (Was the motivator not creating the additional object?) *Testing* - {{CQLTester}} - It might be a little awkward to follow the semantics of "require" -> "prepare" -> "initialize". Maybe we should just inline {{prepareNetwork()}}. - The large frame accumulation state machine might be a good unit test target. Lowest level might be a test-only subclass of {{AbstractMessageHandler.LargeMessage}} that simulates the cases we're interested in. Another would be to go one level up and test the {{CQLMessageHandler.LargeMessage}, but that pulls all of {{CQLMessageHandler}} in, as it's not a static class. (We could make it static, create a little interface to abstract {{CQLMessageHandler}} away, etc.) - It might be worth pulling up {{WaitQueue}} (which is already static) as a top-level class, then throw a few unit-level tests around it it. - {{StartupMessage}} - There's enough logic that it might be nice to unit test execute(). (Mocking {{Connection}} and checking replays, etc.) *Documentation* - Should describe {{native_transport_receive_queue_capacity_in_bytes}} in {{cassandra.yaml}} and mention it in the Jira docs section? - Is the plan to just merge the {{ql_protocol_V5_framing.asc}} content into {{native_protocol_v5.spec}} when things solidify? - {{cql_protocol_V5_framing.asc}} - worth mentioning somewhere there (or if this ends up in the official V5 spec) that non-self-contained frames still only hold part of one message and can't contain the end of one large message and the start of another one - {{OutboundMessageQueue}} - The race from CASSANDRA-15958 is fixed? (Noticed the comment was removed...) - {{InboundMessageHandler}} - The class-level JavaDoc is now almost identical to {{AbstractMessageHandler}}, so we might want to narrow down to the places where the former specializes. (ex. Only it uses, {{ProcessMessage}}.) The same thing goes for the {{LargeMessage}} abstract class and its two sub-classes. - {{CQLMessageHandler}} - could use brief class-level JavaDoc on how it extends {{AbstractMessageHandler}} - {{transport.Frame}} - brief class-level JavaDoc might help clarify its scope/usage (even if it's renamed) - Would class-level JavaDoc for {{transport.Dispatcher}} and {{transport.Flusher}} be helpful. (ex. How do they interact with each other?) - {{Payload#finish()}} - It might be useful to have a bit of detail in the method JavaDoc explaining when we're expected to call it in the {{Payload}} lifecycle. - Do we need a deprecation plan for {{native_transport_frame_block_size_in_kb}} ...or not so much, because it was only added for 4.0 anyway? *Zombies* - {{transport.Frame.Header.BODY_LENGTH_SIZE}} is unused - {{PasswordAuthenticatorTest}} - {{import java.net.InetSocketAddress}} is unused - {{OutboundConnection}} - {{import java.util.stream.Stream}} is unused - {{StressSettings}} - {{import com.datastax.driver.core.Metadata}} and {{import com.google.common.collect.ImmutableMap}} are unused - {{FrameCompressor.LZ4Compressor}} - {{compress()}} doesn't need the {{throws}} clause, and {{outputOffset + 0}} could be simplified - {{ProtocolNegotiationTest}} - can remove the dead {{throws}} clauses from a few tests - {{ProtocolErrorTest}} - {{testErrorMessageWithNullString()}} doesn't need throws clause - {{CQLMessageHandler}} - {{UNKNOWN_STREAM_ID}} and {{import org.apache.cassandra.net.Crc} are unused -
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17211259#comment-17211259 ] Adam Holmberg commented on CASSANDRA-15299: --- Thanks for the additional detail. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17211167#comment-17211167 ] Sam Tunnicliffe commented on CASSANDRA-15299: - [~aholmber] review, plus I'm running some stress tests, which should be done soon. After those two things are finished, there's just some internal renaming to do, which I held off doing until now to avoid polluting the changeset. Not to preempt [~maedhroz] 's review, but I think the wire format is pretty stable now, and the drivers are working well. I've used the python driver branch that [~aboudreault] referenced to re-enable the v5 dtests, which seems fine (there are a handful of failures, but I think those are unrelated). > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17211165#comment-17211165 ] Adam Holmberg commented on CASSANDRA-15299: --- Great to hear. Thanks for checking in. Not trying to hassle, just looking for assurance we're not stuck. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17211093#comment-17211093 ] Caleb Rackliffe commented on CASSANDRA-15299: - [~aholmber] I'm _trying_ to finalize my review today. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17211090#comment-17211090 ] Adam Holmberg commented on CASSANDRA-15299: --- Is this waiting on further work from you Sam, or more review? Just wanted to touch and make sure it's not deadlocked somehow by missed assumptions. There are drivers and server stuff that can be broken loose when this is finalized. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17198465#comment-17198465 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Sorry it's been a while without any visible movement here, but I've just pushed some more commits to address the latest comments from [~ifesdjeen] and [~omichallat]. I've added some tests for protocol negotiation, correct handling of corrupt messages and resource management. {quote} * In CQLMessageHandler#processOneContainedMessage, when we can't acquire capacity and, subsequently, we're not passing the frame further down the line. Shouold we release the frame in this case, since usually we're releasing the source frame after flush. {quote} Done, though we only need to do this when {{throwOnOverload == true}} as otherwise we process the inflight request before applying backpressure. {quote} * ReusableBuffer is unused. {quote} Ah yes, removed {quote} * Server has a few unused imports and eventExecutorGroup which is unused.{quote} Cleaned up the imports and removed eventExecutorGroup {quote} * I'm not sure if we currently handle releasing corrupted frames.{quote} For self-contained frames, there's nothing to do here as no resources have been acquired before the corruption is detected, hence {{CorruptFrame::release}} is a no-op. For frames which are part of a large message, there may be some resource allocated before we discover corruption. This is ok though, as we consume the frame, supplying it to the large message state machine, which handles releasing the bufffers of the previous frames (if any). I've added a test for this scenario which includes a check that everything allocated has been freed. {quote} Shouold we maybe make FrameSet auto-closeable and make sure we always release buffers in finally? I've also made a similar change to processItem which would add item to flushed to make sure it's released. That makes flushed variable name not quite right though. {quote} I've pulled in some of your change to {{processItem}} as it removes some duplication around polling the queue. I've removed the condition in the {{finally}} of {{LegacyFlusher}} though, since if we throw from {{processQueue}} then {{doneWork}} will be false anyway, but there may have been some items processed and waiting to flush. The trade off is calling {{flushWrittenChannels}} even if there's no work to do, but that seems both cheap and unlikely, what do you think? As far as making {{FrameSet}} autoclosable, I don't think that's feasible, given how they are created and accessed. Instead, I've moved the release of source frame buffers out of {{finish}} and added a {{try..finally}} to ensure we release them in the case that {{finish}} fails. I've also tried to address one of your comment re: the memory management here by adding some comments. They're probably not yet enough, but let me know if they are helpful at all. {quote} We can (and probably should) open a separate ticket that could aim at performance improvements around native protocol. {quote} Agreed, I'd like to do some further perf testing, but the results from your initial tests makes a follow-up ticket seem a reasonable option. {quote} I've noticed an issue when the client starts protocol negotiation with an unsupported version. {quote} Fixed, thanks. [~ifesdjeen], I haven't pulled in your burn test or changes to {{SimpleClient}} yet, I'll try to do that next week. I also haven't done any automated renaming yet, I'll hold off on that so as not to add to the cognitive burden until we're pretty much done with review. ||branch||CI|| |[15299-trunk|https://github.com/beobal/cassandra/tree/15299-trunk]|[circle|https://app.circleci.com/pipelines/github/beobal/cassandra?branch=15299-trunk]| > 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: Alex Petrov >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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17188051#comment-17188051 ] Alan Boudreault commented on CASSANDRA-15299: - The python driver implementation is almost done. As I mentioned, it´s a full support of the checksumming feature but this iteration simply wraps every message (no write coalescer). I plan to release the checksumming in the next version of the driver, which is planned in 2-3 weeks. However, this change makes the protocol V5 unusable with current 4.0 beta releases. If you think it´s a problem for the python driver, let me know. The PR is here for testing: [https://github.com/datastax/python-driver/pull/1091] Let me know if I can help in any other way. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17170358#comment-17170358 ] Alan Boudreault commented on CASSANDRA-15299: - FYI, we plan to start working on the python driver implementation soon. Since this driver does not have a write coalescer, we want to provide an _experimental_ checksumming support that simply wrap every message. We will evaluate the performance and the addition of a write coalescer in a second iteration. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17167119#comment-17167119 ] Sam Tunnicliffe commented on CASSANDRA-15299: - > we thought that having supporting drivers available publicly from Maven > Central would facilitate and encourage testing this feature Absolutely, thanks a lot [~adutra]! > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17167104#comment-17167104 ] Alexandre Dutra commented on CASSANDRA-15299: - FYI Java drivers 4.8.0 and 3.10.0 were released yesterday with a preview of protocol checksumming, as implemented currently. We understand that the implementation may evolve, especially around naming issues, but we thought that having supporting drivers available publicly from Maven Central would facilitate and encourage testing this feature. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17167102#comment-17167102 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Thanks [~omichallat], I found a similar issue in the V5 exception handler and reworked V5 side in the previous commit, but hadn't gotten to the pre-V5 version yet. Thanks for confirming, and also for merging JAVA-2772 & JAVA-2773. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17165987#comment-17165987 ] Olivier Michallat commented on CASSANDRA-15299: --- I've noticed an issue when the client starts protocol negotiation with an unsupported version. For example at some point in the future we might have a newer, v6-enabled driver trying to negotiate with a now legacy Cassandra 4.0.0. This causes {{Frame.Decoder}} to throw a {{ProtocolException}} while decoding the first OPTIONS frame. The exception bubbles up the "initial" pipeline to reach {{PreV5Handlers.ExceptionHandler}}, which returns an error response: {code} ErrorMessage errorMessage = ErrorMessage.fromException(cause, handler); ... ChannelFuture future = ctx.writeAndFlush(errorMessage); {code} But looking at {{PipelineConfigurator.configureInitialPipeline}}, the next outbound handler is {{Frame.Encoder}}. It processes frames, not messages. {{PreV5Handlers.ExceptionHandler}} needs to wrap the response. I'm not sure which protocol version to use, but if I'm reading things correctly I think trunk uses {{ProtocolVersion.CURRENT}} in this situation: {code} ChannelFuture future = ctx.writeAndFlush(errorMessage.encode(ProtocolVersion.CURRENT)); {code} With this change the negotiation works as expected. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17163743#comment-17163743 ] Olivier Michallat commented on CASSANDRA-15299: --- Nice to see the performance boost. Is compression enabled in any of those tests? {quote}Speaking of {{Segment}} in driver, we probably want to use naming that is consistent with server. {quote} Yes. Once we've agreed on the definitive names here, I'll do a global refactoring. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17163372#comment-17163372 ] Alex Petrov commented on CASSANDRA-15299: - Thanks Olivier, you're right this one is fixed. I was using other branch to hunt down the leaks and never switched back. I did some basic performance tests, and it looks like new v5 outperforms always outperforms old v4 and v5, and performs either on par with a new v4, or better than it, in most of scenarios. Scenarios where v4 seems to perform better that I was able to produce were with a single connected client. There, with a small number of concurrent async operations, v4 was performing better. However, this might not be such an important scenario. In general, the more we can use batching, the better results seem to be. At the same time, checksumming doesn't seem to negatively contribute to V5 performance. This is a very basic benchmark, since server and client run on the same machine, so there's definitely some interference. What might be good about this test, Cassandra read/write path is mocked out to avoid further interference. Scenario 1: requestSize = 1197, responseSize = 234, 10 connected clients, 1 async operation per batch: {code:java} V5 Variance: 135333.13542032443 Median: 155.0 90p: 159.0 95p: 160.0 99p: 160.0 V4 Variance: 107837.257938666 Median: 153.0 90p: 158.0 95p: 158.0 99p: 158.0 {code} Scenario 2: requestSize = 1197, responseSize = 234, 10 connected clients, 5 async operation per batch: {code:java} Variance: 2.3154951046883754E7 Median: 183.0 90p: 190.0 95p: 191.0 99p: 192.0 Variance: 9145882.574275833 Median: 196.0 90p: 203.0 95p: 204.0 99p: 204.0 {code} Scenario 3: requestSize = 1197, responseSize = 234, 10 connected clients, 10 async operation per batch: {code:java} V5 Variance: 6.080661284308405E7 Median: 182.0 90p: 190.0 95p: 191.0 99p: 191.0 Variance: 7.45228262686024E7 Median: 218.0 90p: 229.0 95p: 230.0 99p: 231.0 {code} Scenario 4: requestSize = 1515743, responseSize = 415145, 10 connected clients, 1 async operation per batch: {code:java} V5 Variance: 3.809988628626683E7 Median: 2442.0 90p: 2482.0 95p: 2487.0 99p: 2489.2995 V4 Variance: 8.865722070719789E7 Median: 3083.0 90p: 3129.0 95p: 3134.0 99p: 3136.0 {code} Here, we have even a larger boost from V5, which is great, and I think this was exactly what was aimed for. I haven't tested multiple large ops per batch, but I also don't see many use-cases for such scenario. One of the things we probably want to improve is to avoid copying from {{Frame}} to outgoing buffers. We can use something in the spirit of {{SegmentBuilder}} that driver uses, and simply write frames directly to the outgoing buffer. It looks like this part may simplify memory management, since we'll have one thing less to worry about. Similar improvement can be done on the incoming path, but here it might be trickier to do so since we're reusing the code from {{net}} package. It looks like reusing code from {{net}} has actually made some of the things a bit harder to follow. Speaking of {{Segment}} in driver, we probably want to use naming that is consistent with server. One other comment I wanted to leave is on {{FlushItem}} and releasing things. Since FlushItem generally has two parts: incoming message (one we're responding to), and response one, it might make sense to release them together. I haven't attempted doing this in my patch, since I thought it might be better if we combine this change with copy-avoiding suggestion mentioned above. But in general, knowing that two logical parts are released together might make it all easier to follow. That said, a simple measurement has shown that this copying does not dominate the picture. In fact, simple profiling hasn't shown any obvious hotspots. We can (and probably should) open a separate ticket that could aim at performance improvements around native protocol. Some of the things I haven't tested which might be important to check that I unfortunately didn't have enough time to: * rapid and frequent disconnects and reconnects * sending and receiving messages other than QueryMessage (such as Options, etc) * backpressure on overload * behaviour in presence of delays, disconnects, and corruption (we also have a netty Proxy for that) > 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 >
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17162188#comment-17162188 ] Olivier Michallat commented on CASSANDRA-15299: --- {quote}There's one more issue with a driver, which is easy to reproduce {quote} >From the line numbers in the stack trace it looks like an old commit, and the >same issue Sam reported >[here|https://github.com/datastax/java-driver/pull/1444#pullrequestreview-421159435]. > Try upgrading to the latest (aeb4066b9), the issue should be fixed. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17162062#comment-17162062 ] Alex Petrov commented on CASSANDRA-15299: - Thank you for the changes. I'm mostly done with a second pass of the review. First, wanted to bring up a couple of important things: * Current memory management is rather hard to follow, and maybe we should make an effort to document and/or codify it somewhere. I've spent about a day following all paths and figuring out ins-and-outs of it. One thing that I've found is that in {{FrameSet#finish}}, we're releasing not only the frame that was encoded into the sending buffer, but also the one we flush. This is done in three places with {{writeAndFlush}}. I believe this was leading to crc mismatch bug that I was catching earlier (see stacktrace [1] below). What might have been happening is we were releasing the buffer too early, which would allow it to get recycled. * There were several places in {{SimpleClient}} where we were not accouting for bytes. One is that we were never releasing {{Frame}} of the response we were getting. I've implemented copying, which is not optimal, but should work for testing purposes. The other one is that we weren't giving the bytes back to the limits. They would be acquired via netty processing, so we need to release them eventually. I've added a simple hook to release those. An alterntive to this would be to not use limits at all, with a downside of loosing a bit of observability. Here's a branch with aforementioned changes. However, I haven't run CI on it, I'll check if it breaks anything later today: https://github.com/ifesdjeen/cassandra/pull/new/15299-alex {code} [1] org.apache.cassandra.net.Crc$InvalidCrc: Read -854589741, Computed 1432984585 at org.apache.cassandra.transport.CQLMessageHandler.processCorruptFrame(CQLMessageHandler.java:328) at org.apache.cassandra.net.AbstractMessageHandler.process(AbstractMessageHandler.java:217) at org.apache.cassandra.net.FrameDecoder.deliver(FrameDecoder.java:321) at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:285) at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:269) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at {code} Some small nits: * In {{CQLMessageHandler#processOneContainedMessage}}, when we can't acquire capacity and, subsequently, we're not passing the frame further down the line. Shouold we release the frame in this case, since usually we're releasing the source frame after flush. * {{ReusableBuffer}} is unused. * {{Server}} has a few unused imports and {{eventExecutorGroup}} which is unused. * I'm not sure if we currently handle releasing corrupted frames. * In {{FrameSet}}, we're currently relying on the fact that we'll be able to go through {{finish}} and release everything successfully. However, this might not always be the case. I couldn't trigger such error, but it might still be possible. Shouold we maybe make {{FrameSet}} auto-closeable and make sure we always release buffers in {{finally}}? I've also made a similar change to {{processItem}} which would add item to {{flushed}} to make sure it's released. That makes {{flushed}} variable name not quite right though. And some improvements that were done to simple client: * it now supports time outs * supports sending multiple messages in a single frame when using v5 * simpler pipelines * reuses code for frame processing There's one more issue with a driver, which is easy to reproduce (with attached jar and test), but here's a stack trace: {code} DEBUG [cluster1-nio-worker-1] 2020-07-21 11:54:20,474 Connection.java:1396 - Connection[/127.0.0.1:64050-2, inFlight=2, closed=false] connection error java.lang.AssertionError: null at com.datastax.driver.core.BytesToSegmentDecoder.decode(BytesToSegmentDecoder.java:56) at io.netty.handler.codec.LengthFieldBasedFrameDecoder.decode(LengthFieldBasedFrameDecoder.java:332) at io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:501) at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:440) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) at io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:93) at
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17159194#comment-17159194 ] Alex Petrov commented on CASSANDRA-15299: - I think that renaming is justified in this case. I've been looking at this patch for some time, and I still have to make a mental effort to translate "Frame" into "Envelope" and so on. I don't think merge size of the patch is a concern here, neither is backport size. We have to be forward-looking, and most often people develop patches against trunk anyways. I'd rather make a mental switch every time I'm looking at the old code, since this will allow for a simpler, more consistent naming in the code that's most actively worked on. > 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: Alex Petrov >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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17155676#comment-17155676 ] Sam Tunnicliffe commented on CASSANDRA-15299: - [~omichallat] I absolutely take on board all of your arguments. Now the changes are getting to a somewhat more decent shape, lets take another look and see if it a large scale rename is absolutely necessary. Maybe it's just Stockholm Syndrome, but I'm finding that the clashes between the {{o.a.c.net}} and {{o.a.c.transport}} classes are less and less of a problem. Perhaps we can live with things as they are after all. Unfortunately, I've been a bit short of time to focus on this since the last update, but I've made some changes to the server configuration to improve testability and added a first meaningful(ish) test. Fleshing out the test coverage should now be a bit easier, which I'll try to do next week. As I mentioned, I've been running end to end tests with the [java2772|https://github.com/datastax/java-driver/tree/java2772] branch of the java driver and have bundled a build from there for now. What would be really useful would be some experimental level of support in the python driver, for cqlsh and python dtests. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17148948#comment-17148948 ] Olivier Michallat commented on CASSANDRA-15299: --- About the name change, I would like to advocate one last time for renaming the new outer type, not the legacy inner type. I know you'd prefer the other way, and in a vacuum I would agree. But I think that in this case maintaining continuity is more important than perfect naming. For example: * the mere size of the patch. This will affect hundreds of unrelated lines. * those changes will get in the way later: create more conflicts when you backport something to a legacy branch, obscure {{git blame}} output, etc. * old commits still use the old naming. If you need to look at something in git history, you'll have to make the mental switch constantly. Not the end of the world, but it's just one more little thing. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17148831#comment-17148831 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Thanks for the comments [~ifesdjeen] & [~omichallat]. {quote} There are several things that I wanted to bring to your attention: {quote} I've handled most of these in a refactor of Flusher. As you suggested, for framed items we now collate the frames and only allocate the payloads when we flush to the netty channel. So now, we allocate the payload based on the actual number of bytes required for the specific channel. {quote}{{ExceptionHandlers$PostV5ExceptionHandler#exceptionCaught}}: when flushing an exception, we don't call release on the payload. {quote} Included in the {{minor cleanups}} commit {quote}There are several places in {{SimpleClient}} where {{largePayload#release}} isn't called. {quote} I've refactored the flushing large messages in {{SimpleClient}} to match {{Flusher}}, so this is working properly now. {quote}Other things... {quote} {quote}{{Dispatcher#processRequest}}, we don't need to cast error to {{Message.Response}} if we change its type to {{ErrorMessage}}. {quote} In {{CqlMessageHandler#releaseAfterFlush}}, we can call {{sourceFrame#release()}} instead of {{sourceFrame.body.release()}} for consistency with other calls Both in {{minor cleanups}} {quote}{{Server#requestPayloadInFlightPerEndpoint}} can be a non-static {{Server}} member. {quote} If you don't mind I'd prefer to leave this as it is for now as it's pre-existing and changing would require reworking CASSANDRA-15519 (changing limits at runtime). {quote}Should we hide {{flusher.queued.add()}} behind a method to disallow accessing queue directly? {quote} I've done this, but I'm not 100% convinced of its utility. As the two {{Flusher}} subclasses need access to the queue, we have to provide package private methods {{poll}} and {{isEmpty}} as well as one to {{enqueue}}. So unless we move {{Flusher}} to its own subpackage, the queue is effectively visible to everything else in {{o.a.c.Transport}} {quote}We can change the code a bit to make {{FlushItemConverter}} instances explicit. Right now, we basically have two converters both called {{#toFlushItem}} in {{CQLMessageHandler}} and {{LegacyDispatchHandler}}. We could have them as inner classes. It's somewhat useful since if you change the signature of this method, or stop using it, it'll be hard to find that it is actually an implementation of converter. {quote} > I've left this as it is just for the moment. I'm working on some tests which > supply a lambda to act as the converter, so I'll come back to this when those > have solidified a bit more. {quote}Looks like {{MessageConsumer}} could be generic, since we cast it to either request or response. {quote} I've parameterised {{MessageConsumer}} & {{CQLMessageHandler}} according to the subclass of Message they expect and extended this a bit by moving the logic out of {{Message$ProtocolEncoder}} to an abstract\{{ Message$Decoder}} with concrete subclasses for {{Request}} and {{Response}}. >bq.Looks like {{CQLMessageHandler#processCorruptFrame}}, initially had an >intention of handling recovery, but now just throws a CRC exception >regardless. This does match description, but usage of {{isRecoverable}} seems >to be redundant here, unless we change semantics of recovery. It is somewhat redundant here, except that it logs a slightly different message to indicate whether the CRC mismatch was found in the frame header or body. I'll leave it as it is for now as it's technically possible to recover from a corrupt body, but would be problematic for clients just now. I still have some comments to address, as well as those from [~omichallat] ... {quote}Frame$Decoder and other classes that are related to legacy path can be extracted to a separate class, since Frame itself is still useful, but classes that facilitate legacy encoding/decoding/etc can be extracted. {quote} {quote}Frame#encodeHeaderInto seems to be duplicating the logic we have in Frame$Encoder#encodeHeader, should we unify the two? Maybe we can have encoding/decoding methods shared for both legacy and new paths, for example, as static methods? {quote} {quote}As you have mentioned, it would be great to rename Frame to something different, like Envelope, since right now we have FrameDecoder#Frame and Frame$Decoder and variable names that correspond with class names, which makes it all hard to follow. {quote} > 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 >
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140729#comment-17140729 ] Olivier Michallat commented on CASSANDRA-15299: --- Another optimization: {{o.a.c.transport.Frame}} holds the body as a {{ByteBuf}}. This means we allocate an individual buffer for each message when we assemble its frame, only to copy it into a larger buffer later, when {{frame.encodeInto()}} is invoked. If {{Message.Codec}} had an {{encodeInto()}} method as well, it would be possible to hold onto the message until we're ready to write it to the payload, and avoid that intermediary copy. It's not that simple though: we can't easily change {{Frame}} because it would impact the legacy protocol code. Maybe the code could be reworked to assemble frames later. That's how it happens to work in the driver (our flusher deals with messages, not frames), so we were able to avoid the double allocation. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140675#comment-17140675 ] Alex Petrov commented on CASSANDRA-15299: - [~samt] thank you for the patch! I think it's a very good addition to v5 protocol. This is not a final review, just a "preview" or "first pass", since I wanted to break it down a little. I agree that the core of the patch is rather self-contained, which has helped reviewing it a lot. There are several things that I wanted to bring to your attention: * {{Flusher.java#encodeIntoFramedResponsePayload}}: in [one place|https://github.com/apache/cassandra/pull/646/files#diff-f2777c908750448a1472c430eab01d94R130] there's a TODO saying we should make sure to limit a buffer to {{MAX_SIZE - 1}}. But [later|https://github.com/apache/cassandra/pull/646/files#diff-f2777c908750448a1472c430eab01d94R142] we allocate without setting the limit. * {{Flusher.java#flushLargeMessage}}: in one place, when [flushing|https://github.com/apache/cassandra/pull/646/files#diff-f2777c908750448a1472c430eab01d94R161-R162] a large message, we don't call {{release}} on the payload. * {{ExceptionHandlers$PostV5ExceptionHandler#exceptionCaught}}: when [flushing|https://github.com/apache/cassandra/pull/646/files#diff-b962f06a031121f71b18d5ce59cdc51eR111] an exception, we don't call {{release}} on the payload. * There are several places in {{SimpleClient}} where {{largePayload#release}} isn't called. * We might want to rethink / optimise memory usage a bit. One thing is that we're always allocating buffers of {{FrameEncoder.Payload.MAX_SIZE - 1}}, which is 131081 bytes. Unfortunately, this size is rather large (even if we return unused bytes quickly). We don't have a memory leak caused by unreleased buffers partly because of this size: in buffer pool, we allocate chunks over 128KiB directly instead of going through pooled chunks. I realize that we're returning all "unused" space when finalizing the buffer (if we ignore or change >128KiB limit), but it looks like we can just avoid these allocations alltogether. What we could do to mitigate this probelm and avoid pre-allocating memory chunks is to collect not {{Channel/Payload}} pairs, but {{Channel/Collection}}, and allocate a chunk of required size already when we're flushing. * If I understand it correctly, {{Flusher#flushWrittenChannels}}, [here|https://github.com/apache/cassandra/pull/646/files#diff-f2777c908750448a1472c430eab01d94R221], a call to {{payloads.remove()}} might cause {{ConcurrentModificationException}} from hashmap. Other things are mostly discussions/suggestions/nits: * As you have mentioned, it would be great to rename {{Frame}} to something different, like {{Envelope}}, since right now we have {{FrameDecoder#Frame}} and {{Frame$Decoder}} and variable names that correspond with class names, which makes it all hard to follow. * Looks like {{MessageConsumer}} could be generic, since we cast it to either request or response. * {{Server#requestPayloadInFlightPerEndpoint}} can be a non-static Server member. * {{Frame$Decoder}} and other classes that are related to legacy path can be extracted to a separate class, since Frame itself is still useful, but classes that facilitate legacy encoding/decoding/etc can be extracted. * {{Frame#encodeHeaderInto}} seems to be duplicating the logic we have in {{Frame$Encoder#encodeHeader}}, should we unify the two? Maybe we can have encoding/decoding methods shared for both legacy and new paths, for example, as static methods? * {{Flusher#flushLargeMessage}} three paths (first/middle frames/last frame) cases can be combined to either parametrized method call or a single loop * {{finish/writeAndFlush/release}} triplet seems to be used together a lot. Should we have a helper method for the three? * Same goes for {{allocate}} + {{limit}} calls that often go together. * {{Dispatcher#processRequest}}, we don't need to cast {{error}} to {{Message.Response}} if we change its type to {{ErrorMessage}}. * Should we hide {{flusher.queued.add()}} behind a method to disallow accessing queue directly? * We can change the code a bit to make {{FlushItemConverter}} instances explicit. Right now, we basically have two converters both called {{#toFlushItem}} in {{CQLMessageHandler}} and {{LegacyDispatchHandler}}. We could have them as inner classes. It's somewhat useful since if you change the signature of this method, or stop using it, it'll be hard to find that it is actually an implementation of converter. * In {{CqlMessageHandler#releaseAfterFlush}}, we can call {{sourceFrame#release()}} instead of {{sourceFrame.body.release()}} for consistency with other calls * Looks like {{CQLMessageHandler#processCorruptFrame}}, initially had an intention of handling recovery, but now just throws a CRC exception regardless. This does match description, but
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140550#comment-17140550 ] Aleksey Yeschenko commented on CASSANDRA-15299: --- Agreed that it doesn't really matter (so long as this change ships with 4.0.0, on that I feel rather strong). > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140454#comment-17140454 ] Benedict Elliott Smith commented on CASSANDRA-15299: To be clear, I'm not in favour of bumping this out of scope - I think it's a super valuable, long overdue bug fix. I just think it's preferable not to advertise v5 at all until this lands because it's binary incompatible with the current v5 implementation. But, again, not a strong preference given the minimal impact. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140430#comment-17140430 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Yes, I agree that it doesn't really matter unless we decide to bump the whole framing change out of v5 and release it as is (minus 13304). The other changes that v5 introduce are relatively minor, which is why I'm in favour of keeping this in scope for v5 if at all possible. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140427#comment-17140427 ] Benedict Elliott Smith commented on CASSANDRA-15299: Hmm. If we're going to be binary incompatible with whatever we advertised in 3.10, it's not clear if that fact matters to us really? So we should probably never have advertised it, and we don't intend to honour it - nobody can be using it, nor expect continuity to 4.0, given that it's binary incompatible. Or am I missing something? But anyway, I don't really mind what option we take out of those, so I'll stop confusing matters if you feel that beta is preferable. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140407#comment-17140407 ] Sam Tunnicliffe commented on CASSANDRA-15299: - We've been advertising it (as beta) since 3.10, so I think sticking with that is the only acceptable option there. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140399#comment-17140399 ] Benedict Elliott Smith commented on CASSANDRA-15299: I see. In that case I would suggest it might be better to simply not advertise v5 at all if we cut a beta before this lands, but assigning beta protocol status sounds like an acceptable second best option. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17140398#comment-17140398 ] Sam Tunnicliffe commented on CASSANDRA-15299: - bq. I think it makes sense to dub v5 protocol alpha, or unavailable Programmatically we don't have the ability to mark protocol versions alpha. Assigning the beta status is more than just informative though, clients cannot automatically negotiate a connection with a beta version unless specifically requesting one. I think this mitigates, to an extent, the harm of the significant change during the (C*) beta phase. My preference would be to see how the next couple of weeks shake out with regard to this review & follow up and the other outstanding tickets. But personally, I'd be ok with cutting beta-1 without this & delivering it in a later beta. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17139987#comment-17139987 ] Benedict Elliott Smith commented on CASSANDRA-15299: I think it makes sense to dub v5 protocol alpha, or unavailable, or stub it out, since this will change the protocol more than we would permit in a beta. But otherwise I think everybody expressed a belief it was OK to move to beta without this, so long as it is delivered during beta. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17139984#comment-17139984 ] Josh McKenzie commented on CASSANDRA-15299: --- Where did we land from the ML? I think it was that we're going to dub v5 protocol beta until this gets in and not block rel of beta1 on this - did I get that right? > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17125675#comment-17125675 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Thanks, but there isn't really. [~omichallat]'s driver work has been really helpful in validating the approach and the server side changes are getting pretty close to being ready for review. Hopefully that can get started next week. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17125310#comment-17125310 ] Josh McKenzie commented on CASSANDRA-15299: --- {quote}There's also still quite a bit of internal stuff to be done, {quote} Anything anyone can do to help on this? This is one of our last blockers before cutting beta. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17113865#comment-17113865 ] Benedict Elliott Smith commented on CASSANDRA-15299: bq. That is, the specific lengths are chosen to ensure that the CRC 24/32 provides adequate protection. FWIW, depending on your definition of "adequate" CRC32 may well not fit the bill for 128KiB, so with CRC32 for the payload 128KiB should very much be an upper limit. A protocol change should move us to CRC64 or CRC128 before we consider increasing the maximum frame size. I personally doubt there will be much value in increasing the frame size soon anyway, since the goal is only to amortise the cost of splitting well and increase compression efficiency. We would want to revisit the compression algorithm we use before we increase the size, to find one able to exploit longer histories (while not requiring too many resources per connection). Which is to say it's definitely for a future protocol, and also to clarify that CRC32 was a poor man's compromise at the time of authoring the internode messaging, because there is no accelerated and readily accessible CRC64 or CRC128 implementation in our supported JDKs. I expect they will be available in the future though, or we can probably include our own using the modern instructions. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17113741#comment-17113741 ] Olivier Michallat commented on CASSANDRA-15299: --- The server is hitting [this line|https://github.com/beobal/cassandra/blob/1a8a79b1f5ff6f863263a28b1160431b8ba6eef5/src/java/org/apache/cassandra/net/FrameEncoderLZ4.java#L75] when it sends back a compressed large response. I haven't dug further, but the the offending length is exactly equal to 128KiB, it's probably just an off-by-one error when it's being split it into chunks. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17113615#comment-17113615 ] Olivier Michallat commented on CASSANDRA-15299: --- Adding an issue link to the Java driver pull request. Apart from the negotiation issue (that I "fixed" with a hacky server-side workaround), it seems to work pretty nicely, I see small messages getting batched and large messages getting split. Compression crashes for some reason, I'll look into it. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17112997#comment-17112997 ] Sam Tunnicliffe commented on CASSANDRA-15299: - bq. Last remark on the protocol: I think the response to the initial STARTUP message should never be wrapped either. I went back and forth on this a bit myself, and to be honest I don't feel super strongly about it. It was primarily implementation concerns that lead me to end up with the framing starting on the initial response. What you say makes sense though and that's clearly more important, so I'll look at reworking along those lines. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17112995#comment-17112995 ] Sam Tunnicliffe commented on CASSANDRA-15299: - When we spoke in on slack, and I'm afraid the answers I gave were only partially correct. https://the-asf.slack.com/archives/CK23JSY2K/p1589982060404400 The good news is that your approach is right and it will correctly decode the header, but the diagram is also correct. The disconnect is that the diagram is describing the bytes as they are laid out on the wire at the data layer, whereas in C* and the Java Driver we're working at a point somewhat higher up the stack. That's perhaps not as clear as it could be and I'll definitely work on improving the docs in that regard. To deconstruct your example, you have the bytes in the buffer with a binary representation of {{0100 0010 0001 1001 0010}} The first 3 bytes {{0100 0010}} are the header length + flag. {{4 |= (1 << 17) == 131076 == 0010 0100}} Writing them to the buffer in little endian ordering (in {{FrameEncodedCrc::put3b}}): {{0100 0010}} and reading them off the wire, where ethernet transmission specifies bytes are sent least significant bit first: {{0100 0010}} so the 18th bit read off the wire is the {{1}} representing the {{selfContained}} flag, but to extract it requires the steps you outlined including the endianness switch. The second 3 bytes in your buffer {{0001 1001 0010}} is the crc24 in little endian order. Here though, the bit order is less important as up the stack we're dealing in whole bytes. So on the wire we'd see {{1000 1001 0100}}, and read that as {{0001 1001 0010}}. Then in {{FrameDecoderCrc::readHeader6b}} we change the endianness to give {{0010 1001 0001 == 15923457 == crc24(131076)}} > 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 >
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17112780#comment-17112780 ] Olivier Michallat commented on CASSANDRA-15299: --- I'm getting close to a working driver prototype. Last remark on the protocol: I think the response to the initial STARTUP message should never be wrapped either. That's because it might be a legacy ERROR frame, if the driver is negotiating with a legacy server that doesn't support v5. I can't tell before I have decoded that response, so I have no way of guessing the correct format ahead of time. So modern framing would only start with the next request sent by the driver (AUTH_RESPONSE or whatnot). > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17112772#comment-17112772 ] Olivier Michallat commented on CASSANDRA-15299: --- We discussed the header format on Slack. Quick summary: * the header bytes are in little endian, * the fields are in the reverse order (the spec will be amended). So following the first example in my previous comment: * receive the full header {{0x04000201f9f2}} * extract the first 3 bytes: {{04 00 02}} * switch endianness: {{02 00 04}}, or in binary: {{0010 0100}} * extract the fields: 6-bit padding, selfContained flag (true), 17-bit payload length (4) > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17111564#comment-17111564 ] Olivier Michallat commented on CASSANDRA-15299: --- OK, not much to add on the field sizes, I hope the future will prove you right. {quote}The only real difference between having this in the header and nominally being part of the body is that the integrity of the header is protected {quote} The integrity of the payload is protected as well. Is it that the guarantee is stronger for the header because it is shorter? Speaking of the header, I think there is an issue in the way it's currently encoded. It's probably a bit early code for reviews, but I want to make sure I understand this correctly. If I use {{FrameEncoderCrc}} for a self-contained, uncompressed outer frame with a 4-bytes payload: {code:java} ByteBuffer buffer = ByteBuffer.allocate(6); FrameEncoderCrc.writeHeader(buffer, true, 4); System.out.println(Bytes.toHexString(buffer)); {code} The buffer ends up with {{0x04000201f9f2}}. In binary that's {{011110010010}}. If I align the first 24 bytes in the spec's diagram, the length is in a weird order, and the flag is not in the right position: {code:java} 0 1 2 3 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ |0 0 0 0 0 1 0 0 0 0 0 0 0 0 0 0 0|0|0 0 0 0 1 0| CRC24... +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ {code} That's because we do a byte-by-byte reversal, but our fields are not aligned on byte boundaries. I think the simplest fix would be to build the header data in network order, and only reverse it when we compute the CRC: {code:java} int header3b = dataLength; header3b <<= 1; if (isSelfContained) header3b |= 1; header3b <<= 6; // padding int crc = crc24(Integer.reverseBytes(header3b)>>8, 3); // Same as put3b, but big-endian frame.put(0, (byte)(header3b >>> 16)); frame.put(1, (byte)(header3b >>> 8) ); frame.put(2, (byte) header3b); put3b(frame, 3, crc); {code} This produces {{0x000240979ee2}}, or in binary: {code:java} 0 1 2 3 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ |0 0 0 0 0 0 0 0 0 0 0 0 0 0 1 0 0|1|0 0 0 0 0 0| CRC24... +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ {code} > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17110101#comment-17110101 ] Sam Tunnicliffe commented on CASSANDRA-15299: - bq. length fields are just big enough to accommodate the 128 KiB limit. Wouldn't it be more cautious to keep a bit of margin? bq. The layout of the header fields were chosen (in CASSANDRA-15066) for efficiency and correctness. Specifically regarding correctness, the lengths of both the header and the frame body are tightly coupled to the CRCs. That is, the specific lengths are chosen to ensure that the CRC 24/32 provides adequate protection. bq. maybe the header should reserve more space for flags bq. The framing format is intentionally extremely lightweight and we should be really cautious about introducing features _at this level of the protocol_ which require flags or any other configuration. I agree wholeheartedly with your point about previous issues that been made more difficult because of a lack of protocol extensibility. My counter argument is that those all relate to the CQL protocol itself, which remains unchanged by this. Changes to the "inner frame"/CQL messages themselves are decoupled from the "outer frame" format. bq. why store the uncompressed length in the header? In the legacy frame format, it's encoded in the compressed payload bq. As you mention in the LZ4 case, the uncompressed length isn't part of the compressed payload, it's prepended (and necessary to allocate a correctly sized buffer for decompression). The only real difference between having this in the header and nominally being part of the body is that the integrity of the header is protected, which saves us from potentially OOMing if the size is corrupted. > 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)
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17107811#comment-17107811 ] Olivier Michallat commented on CASSANDRA-15299: --- As I'm starting to work on the driver changes, I have a few more remarks about the binary format: * length fields are just big enough to accommodate the 128 KiB limit. Wouldn't it be more cautious to keep a bit of margin? I have no issue with the current value, or even the fact that it is hard-coded; but it feels a bit restrictive to remove any possibility of ever increasing it without a protocol change. * along the same lines, maybe the header should reserve more space for flags. We only have one so far, but as experience has shown, new needs can arise over time. Legacy messages use either 1 or 4 bytes. * why store the uncompressed length in the header? In the legacy frame format, it's encoded in the compressed payload (Snappy does that natively, and for LZ4 we prepend it). We have code that takes a buffer and returns a compressed buffer, it would be nice to reuse it without having to change anything. I don't see any advantage having it in the header, if the payload is corrupted we have nothing to decompress anyway, so the uncompressed length is of no use. Putting all those changes together, we could have a unique header format that works both with and without compression: {code:java} 0 1 2 3 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ | Payload Length | +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ | flags | CRC24 of Header | +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ {code} > 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.
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17106257#comment-17106257 ] Aleksey Yeschenko commented on CASSANDRA-15299: --- bq. Firstly, we think that not all users will be interested in enabling checksumming. Making it mandatory may keep them away from v5, potentially decreasing its adoption rate. The change is less about checksumming per-se, and more about improving protocol performance by making framing sane. In particular when compression is enabled. In addition, I strongly believe that no new protocol should be released without application-level checksumming built-in, and v5 is a new protocol. bq. For these drivers, it's a race against time that starts now: they must implement protocol checksumming before Cassandra 4.0 GA, or they will lose access to all the v5 features implemented so far, especially keyspace-per-query – which is a nice usability improvement that many users are waiting for. All of the v5-beta features implemented so far are nice-to-haves and non-essential, IMO. The new framing, to me, *is* the core of V5. And if those drivers don't immediately support the new protocol upon C* release, that is fine, too. C* and drivers are on separate release cadences, for one, and V4 protocol is still supported, for two. bq. Making checksumming opt-in would allow many drivers to be ready for Cassandra 4.0 GA, and others to catch up quickly. We feel this would greatly contribute to promoting Cassandra 4.0's adoption. Again, it would be nice if all the drivers supported protocol v5 by the time C* 4.0 was out, but I don't see it as an issue if they don't. Protocol v5 is not the selling feature of C* 4.0, and other changes should provide plenty of motivation for people to upgrade. The drivers will still be ready for 4.0 GA - so long as they can speak v4 - and implement v5 at some point later. > 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 >
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17106254#comment-17106254 ] Benedict Elliott Smith commented on CASSANDRA-15299: The new framing also replaces the protocol's broken approach to compression, so unless we maintain essentially two distinct v5 protocols we'd be talking about not supporting compression for these clients either. I don't think this is an acceptable option? I also think it is a bug that we have not enforced checksums until now. Given their low cost, I'm strongly against making them optional without strong evidence of benefit and, even then, only if permitted by the cluster administrator. If the only remaining option is for clients to lag the server in features, then I guess that's what I vote for. I'm surprised this is considered so onerous to implement by the clients, though. > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17105827#comment-17105827 ] David Capwell commented on CASSANDRA-15299: --- bq. Firstly, we think that not all users will be interested in enabling checksumming Curious why you make that statement, I would hope users don't want silent corruption (for the same reason all tables should be compressed, we should improve the format to not couple checksumming with compression). Since this is also between client and server, other than performance, how would the user even know about this without looking closer into the details? > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17105688#comment-17105688 ] Alexandre Dutra commented on CASSANDRA-15299: - Hi, as we have progressed on the driver-side implementation of this new feature, we feel that this should be an opt-in protocol feature. Firstly, we think that not all users will be interested in enabling checksumming. Making it mandatory may keep them away from v5, potentially decreasing its adoption rate. Secondly, we believe that without proper driver support, protocol v5 will not gain momentum. We are strongly committed to providing full support for checksumming at least in the Java driver, but other drivers might not be able to catch up, both those maintained by DataStax and those that are not, such as gocql: * Java 3.x, Java 4.x, Python and C# are currently on par with protocol v5 specs (the latter as of [today|https://groups.google.com/a/lists.datastax.com/d/msg/csharp-driver-user/V8hOVUb-84A/RSwauBgjAgAJ]). For these drivers, it's a race against time that starts now: they _must_ implement protocol checksumming before Cassandra 4.0 GA, or they will lose access to all the v5 features implemented so far, especially keyspace-per-query – which is a nice usability improvement that many users are waiting for. * NodeJS, C++ and PHP could also be ready for Cassandra 4.0 (most protocol v5 features are already there, only not exposed), but having to implement checksumming makes it unlikely that these drivers will be able to catch up. * Other DataStax drivers (Ruby) will take twice as much time to implement v5 if they need to include checksumming; it's unlikely that they would be ready this year. * gocql: not expected to catch up with protocol checksumming any time soon. Making checksumming opt-in would allow many drivers to be ready for Cassandra 4.0 GA, and others to catch up quickly. We feel this would greatly contribute to promoting Cassandra 4.0's adoption. > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17104465#comment-17104465 ] Sam Tunnicliffe commented on CASSANDRA-15299: - > find out that the _uncompressed_ size is above the limit, split it into 128 >KiB chunks, then compress each chunk separately and wrap it into its own >(uncontained) outer frame? Yes, exactly. > uncontained frames are never recoverable when compression is enabled Large messages are never recoverable when a corrupt payload is detected. Self-contained frames *could* be recoverable as long as only the payload is corrupt, but like we discussed earlier this is more complicated here than in the internode case due to the payload potentially containing multiple stream ids, so we may as well close the connection whenever a corrupt frame is encountered. > 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-beta > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17101000#comment-17101000 ] Olivier Michallat commented on CASSANDRA-15299: --- {quote}we keep accumulating messages into the payload until the max *uncompressed* size is reached {quote} So I guess for large messages we would use a similar approach: find out that the _uncompressed_ size is above the limit, split it into 128 KiB chunks, then compress each chunk separately and wrap it into its own (uncontained) outer frame? I think that's the only approach that will work because we need to decompress the first chunk separately on the receiving end, in order to read the expected message body size that tells us how many more frames follow. So we can't split after compression. If that's correct, then uncontained frames are never recoverable when compression is enabled: we need to decompress all the chunks to keep track of the total size. > 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-beta > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17100165#comment-17100165 ] Sam Tunnicliffe commented on CASSANDRA-15299: - bq. For context, how common are flipped bits in your experience? Dropping the whole connection every time might seem a bit heavy-handed, but if it's a rare occurrence maybe that's an acceptable tradeoff. Not that common, but I know they do happen very occasionally. Probably rarely enough to make that tradeoff acceptable. {quote}It might be worth mentioning that in the document as a hint for implementors {quote} Will do. > 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-beta > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17100037#comment-17100037 ] Olivier Michallat commented on CASSANDRA-15299: --- I hear your arguments about naming, I'll let others chime in. Thankfully these are low-level concepts that don't transpire through the driver's public API, so at least binary compatibility is not a factor. {quote}I suppose it might be possible to track the stream ids for a given message {quote} That sounds complicated, because the party that checks the CRC is not the one who assembled the message (client vs server or vice-versa). For context, how common are flipped bits in your experience? Dropping the whole connection every time might seem a bit heavy-handed, but if it's a rare occurrence maybe that's an acceptable tradeoff. {quote}The first of outer frame of a large message contains the inner frame header, which specifies the length in bytes of the CQL message body. {quote} Got it. That also answers my follow-up question about why a corrupted first frame is not recoverable. {quote}we keep accumulating messages into the payload until the max *uncompressed* size is reached [...] So there is some loss of density when using compression {quote} I see. It might be worth mentioning that in the document as a hint for implementors. > 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-beta > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17099833#comment-17099833 ] Sam Tunnicliffe commented on CASSANDRA-15299: - Hi [~omichallat] , thanks for the feedback this is really useful. {quote}You're hinting at renaming at a later point. I think that will be most welcome, the names as they are now are pretty confusing. And IMHO you should rename the outer container: even though "frame" is better suited for it, it's more important to preserve familiarity for people coming from the legacy code. {quote} I was absolutely planning on renaming, I just imagined that there might be several opinions on this, so was holding off to give the discussion a chance. My personal preference would be to rename the inner container; I totally appreciate your point about maintaining continuity for people coming from the legacy code, but there a few points that sway me in the opposite direction: 1) Internal consistency and compatibility. The proposal here is to reuse both the concepts and, where possible, the implementation from the internode su bsystem. 2) At some point legacy is legacy and we ought to pick the right names for things and concepts rather than be forever bound by previous decisions. This can definitely be painful, but is probably necessary at times. c.f. the use of {{Row}} in 2.2 and earlier and how that's changed wrt to {{Partition}} and CQL rows with 3.0. 3) I wonder if it's actually more important to aim for clarity in naming in order to better accomodate people without pre-existing familiarity with the codebase. Those of us who have been working on the code for some time already have a mental model and for sure, re-mapping the terms therein is annoying, somewhat detrimental to productivity and in some cases, potentially dangerous (though I don't think that's such a concern here). For those coming to the code for the first time though, abiguous and/or outdated naming surely has an even greater impact in terms of aggravating the learning curve. {quote}From a client point of view, a dropped frame will result in request timeouts. We have no way of providing a better error, since the stream ids of the failed requests are in the corrupt payload. I'm wondering if it might not be better to drop the connection all the time: at least the client gets immediate feedback (we could try to propagate a cause), instead of a bunch of requests timing out for no apparent reason. {quote} Yes, I agree. This is the major difference between the internode and client/server messaging schemes. Failing hard and fast whenever a corrupt frame is encountered seems the right thing to do at the moment. I suppose it might be possible to track the stream ids for a given message on the client side and have the server return an error, rather than closing the connection. But I'm not sure just how feasible that is on the client side. {quote}When two large messages are sent in a row, how will you tell the last outer frame of the first message from the first outer frame of the second? Don't you need some sort of is_last_frame flag in the header? {quote} The first of outer frame of a large message contains the inner frame header, which specifies the length in bytes of the CQL message body. The processor accumulates outer frames until that message has been completely received. The single CQL message per large message constraint is required for this to work. {quote}When compression is enabled, how are you going to test when the payload reaches the max size?...all the messages in the payload are compressed together, so you can't check the size as you go like you do for the uncompressed version {quote} That's correct, we keep accumulating messages into the payload until the max *uncompressed* size is reached, or we run out of messages. So there is some loss of density when using compression, but this is an implementation detail and could be improved later without requiring further changes to the protocol. > 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-beta > > > 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 >
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17099447#comment-17099447 ] Olivier Michallat commented on CASSANDRA-15299: --- I work on the Java driver. Here are a few comments/questions on your branch (let me know if there is a better medium for this). 1) Name clash: {quote}The introduction of this new format, brings an unfortunate naming clash between frames from the previous format [...] and the new outer frames. What was previously referred to as a Frame could perhaps be better described as an Envelope or Container [...] Prior to making any conclusion here though, for the purposes of this document, legacy frames are referred to as CQL frames and the new enclosures as outer frames {quote} You're hinting at renaming at a later point. I think that will be most welcome, the names as they are now are pretty confusing. And IMHO you should rename the outer container: even though "frame" is better suited for it, it's more important to preserve familiarity for people coming from the legacy code. 2) CRC check failures: this will result in either dropping the corrupt frame, or the whole connection (comments in {{InboundMessageHandler}}). >From a client point of view, a dropped frame will result in request timeouts. >We have no way of providing a better error, since the stream ids of the failed >requests are in the corrupt payload. I'm wondering if it might not be better >to drop the connection all the time: at least the client gets immediate >feedback (we could try to propagate a cause), instead of a bunch of requests >timing out for no apparent reason. 3) When two large messages are sent in a row, how will you tell the last outer frame of the first message from the first outer frame of the second? Don't you need some sort of {{is_last_frame}} flag in the header? 4) When compression is enabled, how are you going to test when the payload reaches the max size? This would have to be added to the logic in {{V5Flusher}}; but all the messages in the payload are compressed together, so you can't check the size as you go like you do for the uncompressed version. I was thinking of maybe compressing all available results and splitting in multiple uncontained frames if the compressed payload is too big, but that seems at odds with this invariant in {{InboundMessageHandler}}: {quote}All the bytes in an uncontained frame always belong to a single message.{quote} > 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-beta > > > 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
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17097525#comment-17097525 ] Sam Tunnicliffe commented on CASSANDRA-15299: - I've pushed a branch with preliminary patches here: https://github.com/beobal/cassandra/tree/15299-trunk including a [note| https://github.com/beobal/cassandra/blob/15299-trunk/cql_protocol_V5_framing.asc] on the proposed format changes. This is very much a WIP in terms of the implementation, and there is a sizeable list of outstanding {{TODOs}}, not least unit/functional/performance tests, but I wanted to get something that roughly works up to unblock driver development. I've been testing with {{SimpleClient}} and {{debug-cql}}, plus a horribly hacked version of driver 3.6 which just exercises a few types of interaction to provide a simple smoke test. [~jorgebg], [~eduard.tudenhoefner], [~adutra], [~blerer] [~csplinter], [~aholmber] : you guys all asked me about this at some point in the recent past, so now there's something to look at. I apologise for taking so long to get to this point, but other stuff got in the way I'm afraid. Hopefully, I should have a bit more time to focus on this now and see if we can get it into shape for beta. > 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-beta > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17047432#comment-17047432 ] Jorge Bay commented on CASSANDRA-15299: --- Let me know if I can help in anyway with this ticket (early review / tests). > 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: Aleksey Yeschenko >Priority: Normal > Labels: protocolv5 > Fix For: 4.0-beta > > > 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16936759#comment-16936759 ] Aleksey Yeschenko commented on CASSANDRA-15299: --- bq. Can you share what you had in mind? Of course. We would be reusing the framing protocols introduced by the messaging rewrite (see the links to code in the description above), including for when neither checksumming or compression are enabled ({{FrameDecoderUnprotected}} in messaging). bq. Are you thinking some kind of window for flushing messages? We already have that implicitly. The only difference will be that instead of emitting one frame for one message, multiple messages (if we have several to encode) will often map to a single frame - with per-frame CRC32 and compression, when either is enabled. > 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: Aleksey Yeschenko >Priority: Normal > Labels: client-impacting, protocolv5 > Fix For: 4.0-beta > > > 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 > parcicular, 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16936567#comment-16936567 ] Jorge Bay commented on CASSANDRA-15299: --- bq. we should switch to a framing protocol with multiple messages in a single frame Can you share what you had in mind? Would this mechanism be also available when checksumming and/or compression is disabled? Are you thinking some kind of window for flushing messages? > 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: Aleksey Yeschenko >Priority: Normal > Labels: client-impacting, protocolv5 > Fix For: 4.0-beta > > > 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 > parcicular, 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: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org