[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14556583#comment-14556583 ] Alan Boudreault commented on CASSANDRA-6809: I did multiple tests locally and on cstar_perf. Everything looks good so far and the performance are great. I also tested the disk usage through the different compression algorithms and the benefit were good as expected: For the record, here are the links of the main cstar_perf tests: http://cstar.datastax.com/tests/id/a5f48f90-007c-11e5-adde-42010af0688f http://cstar.datastax.com/tests/id/8f0c4dfe-007c-11e5-aa79-42010af0688f http://cstar.datastax.com/tests/id/de7512ea-007c-11e5-adde-42010af0688f The DeflateCompressor is a bit slower, but IIRC, it is expected and not very used. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance, qa-resolved Fix For: 2.2.0 beta 1 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14379649#comment-14379649 ] Branimir Lambov commented on CASSANDRA-6809: Fix-up and rename [here|https://github.com/apache/cassandra/commit/9bd5aaa050ab3826c5f8339cbcdc85984adf9047], and [diff|https://github.com/apache/cassandra/commit/9bd5aaa050ab3826c5f8339cbcdc85984adf9047.diff]. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14378865#comment-14378865 ] Ariel Weisberg commented on CASSANDRA-6809: --- I did +1 the final set of changes. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14378857#comment-14378857 ] Benedict commented on CASSANDRA-6809: - Could I just confirm that [~aweisberg] has +1'd the final changes? The +1 is a little ambiguous, as it is contextualised wrt byte buffer pooling, and doesn't make clear the last tweaks have been +1'd. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14378875#comment-14378875 ] Benedict commented on CASSANDRA-6809: - bq. I _would still_ be +1 is too ambiguous for me to risk breaking trunk on :) Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14375856#comment-14375856 ] Aleksey Yeschenko commented on CASSANDRA-6809: -- +1 to committing as is. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14375918#comment-14375918 ] Alan Boudreault commented on CASSANDRA-6809: [~benedict] Adding a note to retest my simple dtest with the latest branch this week. No objection to see this committed right now though. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14366930#comment-14366930 ] Benedict commented on CASSANDRA-6809: - I wouldn't hold up commit over that, no. [~aboudreault] are you planning to do some further testing on this before commit, or should I go ahead and commit this to trunk? Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14365867#comment-14365867 ] Ariel Weisberg commented on CASSANDRA-6809: --- I think what Benedict is describing where we only deallocate if the queue has more than one available (and 3) is good idea. If the knob is set too low it might give better behavior at steady state? I would still be +1 since I think it won't matter once we multi-thread compression (and I intend to do that for 3.1). Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14360969#comment-14360969 ] Ariel Weisberg commented on CASSANDRA-6809: --- I don't think it works as a hard limit. Filesystems can hiccup for a long time and if you buffer to private memory you avoid seeing the hiccups. A high watermark isn't great either because you commit memory that isn't needed most of the time. Maybe I am not following what you are suggesting. When we have ponies we will be writing to private memory, probably around 128 megabytes, to avoid being at the mercy of the filesystem. Once compression is asynchronous to the filesystem and parallel the # of buffers can be small because compression will tear through fast enough to make the buffers available again. So you would have memory waiting to drain to the filesystem (128 megabytes) and a small number of buffers to aggregate log records until they are sent for compression. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14360980#comment-14360980 ] Benedict commented on CASSANDRA-6809: - The idea of making it a hard limit instead of the concrete number is to fix it as something much larger than you would like it to be, but no larger than you really must expect it to go, so that it can scale gracefully with some blips and avoid those blips having severe negative repurcussions. My expectation is that whatever strategy we use here we will also use for non-compressed once we migrate to manually managed memory buffers, and it seems to me always having deallocations trail utilisation by 1 (so the first to become unused we do not deallocate, but if another becomes unused we release one of the two) Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14360367#comment-14360367 ] Benedict commented on CASSANDRA-6809: - bq. Good catch, the number of segments needed can depend on the sync period, thus this does need to be exposed. Done, as a non-published option in cassandra.yaml for now What's stopping us ensuring there are always enough (or treating this as a hard limit rather than an actual number)? Seems like ensuring we have the number of buffers in flight + 1 should be enough to reach a steady state rapidly without overcommitting memory... Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14360347#comment-14360347 ] Branimir Lambov commented on CASSANDRA-6809: Rebased and updated [here|https://github.com/apache/cassandra/compare/trunk...blambov:6809-compressed-logs-rebase], now using ByteBuffers for compressing. Some fixes to the {{DeflateCompressor}} implementation are included. {quote} At https://github.com/apache/cassandra/compare/trunk...blambov:6809-compressed-logs#diff-d07279710c482983e537aed26df80400R340 If archiving fails it appears to delete the segment now. Is that the right thing to do? {quote} It deletes if archival was successful ({{deleteFile = archiveSuccess}}). The old code was doing the same thing, a bit more confusingly ({{deleteFile = !archiveSuccess ? false : true}}). bq. CSLM's understanding of segment size is skewed because compressed segments are less than the expected segment size in reality. With real compression ratios it's going to be off by 30-50%. If when the size is known it's tracking could be corrected it would be nice. Unfortunately that's not trivial. Measurement must happen when the file is done writing, which is a point CLSM doesn't currently have access to; moreover that could be triggered from either sync() or close() on the segment and I don't want to include the risk that I don't get all updates working correctly into this patch. Changed the description of the parameter to reflect what it is currently measuring. This can and should be fixed soon after, though, and I'll open an issue as soon as this is committed. bq. For the buffer pooling. I would be tempted to not wait for the collector to get to the DBB. If the DBB is promoted due to compaction or some other allocation hog it may not be reclaimed for some time. In CompressedSegment.close maybe null the field then invoke the cleaner on the buffer. There is a utility method for doing that so you don't have to access the interface directly (generates a compiler warning). Done. bq. Also make MAX_BUFFERPOOL_SIZE configurable via a property. I have been prefixing internal C* properties with cassandra. I suspect that at several hundred megabytes a second we will have more than 3 32 megabyte buffers in flight. I have a personal fear of shipping constants that aren't quite right and putting them all in properties can save waiting for code changes. Good catch, the number of segments needed can depend on the sync period, thus this does need to be exposed. Done, as a non-published option in cassandra.yaml for now. bq. I tested on Linux. If I drop the page cache on the new code it doesn't generate reads. I tested the old code and it generated a few hundred megabytes of reads. Thank you. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14355743#comment-14355743 ] Ariel Weisberg commented on CASSANDRA-6809: --- I really like what you did with CommitLogStressTest. At https://github.com/apache/cassandra/compare/trunk...blambov:6809-compressed-logs#diff-d07279710c482983e537aed26df80400R340 If archiving fails it appears to delete the segment now. Is that the right thing to do? CSLM's understanding of segment size is skewed because compressed segments are less than the expected segment size in reality. With real compression ratios it's going to be off by 30-50%. If when the size is known it's tracking could be corrected it would be nice. For the buffer pooling. I would be tempted to not wait for the collector to get to the DBB. If the DBB is promoted due to compaction or some other allocation hog it may not be reclaimed for some time. In CompressedSegment.close maybe null the field then invoke the cleaner on the buffer. There is a utility method for doing that so you don't have to access the interface directly (generates a compiler warning). Also make MAX_BUFFERPOOL_SIZE configurable via a property. I have been prefixing internal C* properties with cassandra. I suspect that at several hundred megabytes a second we will have more than 3 32 megabyte buffers in flight. I have a personal fear of shipping constants that aren't quite right and putting them all in properties can save waiting for code changes. I tested on Linux. If I drop the page cache on the new code it doesn't generate reads. I tested the old code and it generated a few hundred megabytes of reads. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14340607#comment-14340607 ] Branimir Lambov commented on CASSANDRA-6809: New version uploaded to [the same github branch|https://github.com/apache/cassandra/compare/trunk...blambov:6809-compressed-logs]. Includes removal of recycling as agreed in CASSANDRA-8771, as the compressed path was using unnecessarily large amounts of memory for prepared segments to write in. bq. I am more comfortable with the single threaded task generation in Benedict's diff. I would rather see that or no multi-threading for now. Rolled back the multithreaded sync capability. bq. Cap the size of the buffer pool to a fixed number of buffers. If you end up allocating more than that number, free the memory rather then pooling. Done now. Previously the size of allocated memory was defined by commitlog_segment_size_in_mb, which unnecessarily connected two unrelated limits. Removing recycling fixes this problem. bq. I’d still like to see the JSON field in commit log descriptor be reusable for other config parameters. Done. Caused a change in CommitLogTest as the descriptor is now larger. bq. In CommitLogSegment constructor it is no longer truncating the file, why is it no longer necessary? As the size is not preset for {{CompressedSegment}}, it makes no sense to truncate/resize at open. The resize was thus moved to {{MemoryMappedSegment}}; it no longer needs to truncate, but allocates the space for the file. I am not sure it is really necessary (Windows at least resizes file on memmapping), but the Java docs do not guarantee mapping will succeed if the file is not big enough so I left it in place. I'd appreciate a test on Linux to confirm CASSANDRA-8729 is no longer a problem with uncompressed logs after this patch. {quote} * In cassandra.yaml, for commitlog_compression_threads can you document that the default is 1? * awaitTermination changed from block forever to block for 3600 seconds. I have never been a fan of ExecutorService's async termination. If it was supposed to finish it's last task it should terminate and if it doesn't then something is wrong. {quote} No longer relevant. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: docs-impacting, performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14313299#comment-14313299 ] Ariel Weisberg commented on CASSANDRA-6809: --- * In cassandra.yaml, for commitlog_compression_threads can you document that the default is 1? * Cap the size of the buffer pool to a fixed number of buffers. If you end up allocating more than that number, free the memory rather then pooling. * awaitTermination changed from block forever to block for 3600 seconds. I have never been a fan of ExecutorService's async termination. If it was supposed to finish it's last task it should terminate and if it doesn't then something is wrong. * I’d still like to see the JSON field in commit log descriptor be reusable for other config parameters. * In CommitLogSegment constructor it is no longer truncating the file, why is it no longer necessary? I am more comfortable with the single threaded task generation in Benedict's diff. I would rather see that or no multi-threading for now. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14303666#comment-14303666 ] Alan Boudreault commented on CASSANDRA-6809: Great. Thanks. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14303660#comment-14303660 ] Branimir Lambov commented on CASSANDRA-6809: Yes, this is the expected behaviour. The rationale is in the discussion above-- compression with a fixed size of the compressed segment is quite a bit more complex as it requires a level of microsegments that mutation threads write in, in addition to what exists now. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14303633#comment-14303633 ] Alan Boudreault commented on CASSANDRA-6809: [~barnie] Can you confirm if this is the desired behavior? Using commitlog_compression, the commitlog files created on disk are smaller than what we set in cassandra.yaml. Using default configuration, commitlogs are 23MB rather than 32MB. I thought that the compression would only affect the data written inside the commitlogs, and not the file size itself. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14298891#comment-14298891 ] Branimir Lambov commented on CASSANDRA-6809: Added a stress test exercising all log options to the long test suite and fixed several issues: - Initialization could fail with NPE or similar as some threads started within the CommitLog constructor could use fields which weren't initialized yet. - Made sure the length of sync sections is precise; replay could read beyond the end of the file or read stale mutations from reused compressed buffers when the length is not precise. - Made sure completed segment files are always closed. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14293420#comment-14293420 ] Branimir Lambov commented on CASSANDRA-6809: Rebased and applied review comments. New version uploaded [here|https://github.com/blambov/cassandra/compare/6809-compressed-logs]. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14289213#comment-14289213 ] Branimir Lambov commented on CASSANDRA-6809: Thank you, I did not realise you are interested in parallelism between segments only. Of course, what you suggest is the right solution if we are limited to that; I approached the problem with the assumption that we need shorter sections (of the same segment) that are to progress in parallel. I can see that this should work well enough with large sync periods, including the 10s default. I am happy to continue with either approach, or without multithreaded compression altogether. I am now going back to addressing the individual issues Ariel raised. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14289242#comment-14289242 ] Benedict commented on CASSANDRA-6809: - bq. Thank you, I did not realise you are interested in parallelism between segments only. Well, I considered that a natural extension, i.e. a follow up ticket. One I still consider reasonably straight forward to add: a mutator thread can partition the commit range once it's processed ~1Mb, and simply append the Callable to a shared queue. The sync thread can then drain this when it decides to initiate a sync. bq. I can see that this should work well enough with large sync periods, including the 10s default. I'm reasonably confident this will work as well or better for all sync periods. In particular it better guarantees honouring the sync periods, and is less likely to encourage random write behaviour. Of course, the main benefit is its simplicity. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14288522#comment-14288522 ] Benedict commented on CASSANDRA-6809: - It's a bit late at night to throw this together, so this is only an expression of intent, but here's quick hour's attempt at the approach I am advocating. It should be complete, but ignoring the extension ideas or exploring any other avenues, and with only around 97% care (i.e. I'm disclaiming minor mistakes). All I've done is split the sync() into a prepareSync() which returns a List of CallableRunnable - the callable is the compression and write-to-buffer work, and the Runnable is the FS barrier work. The former is submitted to an executor service, which may be an inline executor. The result is then executed inline in the sync thread. I think it is a pretty simple change, and it is very clear what its behaviour is. Let me know what you think. I think we should probably split this discussion into its own ticket, though, since the work you've done on compression is good to commit as is IMO. Branch [here|https://github.com/belliottsmith/cassandra/commits/6809-extend] Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14287853#comment-14287853 ] Branimir Lambov commented on CASSANDRA-6809: {quote} * single sync thread forms sections at regular time intervals and sends them to compression executor/phase (SPMC queue), * sync thread waits on futures and syncs each in order {quote} I gave your suggestion a day of development, but it still introduces more problems than it solves. I took it [this far|https://github.com/blambov/cassandra/compare/blambov:compressed-cl...compressed-cl-compressionexecutor]. It's already significantly more complicated than the option I proposed, and I got worse performance and still some uncertainties around recycling and shutdown. Perhaps I did not put this clearly, but I don't see a point introducing a trigger for compression other than a sync. Reducing write latency for a 10s sync period is of no value whatsoever; with a short period, especially in batch mode where it really matters, you wouldn't want to start a compression cycle before the batch is completed anyway (if you did, a better solution to the problem is to just compress each mutation individually). We have ample flexibility in the sync period (time) and segment size (space) to be able to use compression efficiently. Granted, this may require documenting different defaults for compression, but this is something I would much prefer to live with than extra code complexity needed to work around badly chosen parameters. Assuming sync-only triggering and short periods, your suggestion requires decoupling of sync starts from sync completions with a queue of sync requests in flight. That's what I implemented in the code above. Am I doing something wrong? Going back to the previous approach (updated to fix problem with sync possibly completing earlier than it should), bq. We're now no longer honouring the sync interval; we are syncing more frequently, which may reduce disk throughput. The exact time of syncing in relation to each other may also vary, likely into lock-step under saturation, so that there may be short periods of many competing syncs potentially yielding pathological disk behaviour, and introducing competition for the synchronized blocks inside the segments, in effect introducing a MPMC queue, eliminating those few micros of benefit. The sync frequency is as specified, the intervals will vary, but writes to disk are still serial so disks should behave normally. There will be competition on waitForSync if compression is constantly late but, as you say, in this case the magnitude of the overheads is too small to matter. A bigger problem is that I can imagine a pathological situation where only one thread is doing work if the others have nothing to do but also become late waiting for it, and start the next cycle at the same time. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14285476#comment-14285476 ] Benedict commented on CASSANDRA-6809: - bq. assuming that the sync period is sane (e.g. ~100ms) The sync period is, by default, 10s, and to my knowledge this is what many users run with - so in general we will only compress each individual segment. This is still sane, since the cluster has redundancy, although a sync period of between 100ms and 500ms might be more suitable for high traffic nodes. Still, it's probably not a big deal since we only care about compression when under saturation, which should mean many segments. I only mention it, since it is an easy extension. This extension also means the sync thread may have compressed data _waiting_ for it when it runs, reducing the latency until sync completion. bq. Let me try to rephrase what you are saying to make sure I understand it correctly: Almost: * single sync thread forms sections at regular time intervals and sends them to compression executor/phase (SPMC queue), * _sync thread waits on futures and syncs each in order_ Or, with the extension: * mutators periodically submit segment to compressor * once compressor completes an entire segment, requestExtraSync() is called (instead of in advanceAllocatingFrom()) bq. Why is this simpler, or of comparable complexity? We have two steps in explanation, instead of five. More importantly, there is no interleaving of events to reason about between the sync threads, and the lastSync is accurate (which is important since this could artificially pause writes). This also means future improvements here are easier and safer to deliver, because we don't have to reason about how they interplay with each other. In particular, marking lastSync roll over after each segment is synced is a natural improvement (to ensure write latencies don't spike under load) but is challenging to introduce with multiple sync threads. Since we don't expect this feature to be used widely (we expect multiple CL disks to be used instead, if you're bottlenecking) the simpler approach seems more sensible to me. bq. Wouldn't the two extra queues waste resources and increase latency? We have zero in the typical case, and one extra queue in the uncommon use case. If we introduce enough threads that compression is faster than disk, then there will be near zero synchronization costs; of course, if that is not the case, and we are bottlenecking on compression still, then we aren't really losing much (a few micros. every few hundred millis, at 250MB/s compression speed), so it doesn't seem likely to be significant. We're now no longer honouring the sync interval; we are syncing more frequently, which may reduce disk throughput. The exact time of syncing in relation to each other may also vary, likely into lock-step under saturation, so that there may be short periods of many competing syncs potentially yielding pathological disk behaviour, and introducing competition for the synchronized blocks inside the segments, in effect introducing a MPMC queue, eliminating those few micros of benefit. (FTR, the MPMC, SPMC, MPSC aspects are likely not important here. The only concern is thread signalling, but this is the wrong order of magnitude to matter when bottlenecking on disk or compression of large chunks) Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14285436#comment-14285436 ] Branimir Lambov commented on CASSANDRA-6809: The current approach boils down to multiple sync threads which * form sections at regular time intervals, * compress the section, * wait for any previous syncs to have retired, * write and flush the compressed data, * retire the sync. (In the uncompressed case a single thread which skips the second and third step.) Let me try to rephrase what you are saying to make sure I understand it correctly: * single sync thread forms sections at regular time intervals and sends them to compression executor/phase (SPMC queue), * compression task sends completed sections to flush executor/phase (MPSC queue, ordering and wait for the first in-flight one required), * flush task retires syncs in order. Is this what you mean? Why is this simpler, or of comparable complexity? Wouldn't the two extra queues waste resources and increase latency? Smaller-than-segment batches (sections) are already part of the design in both cases, assuming that the sync period is sane (e.g. ~100ms). In both approaches there's room to further separate write and flush at the expense of added complexity. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14284213#comment-14284213 ] Benedict commented on CASSANDRA-6809: - I haven't looked _closely_ at the patch, but it seems to me that a simpler approach would be to split CL.sync() into a compression phase and a sync phase, so that there remains only one sync writer. The first phase would submit to an executor service, and the second would flush them to disk (possibly started immediately, but waiting on the completion of each compression chunk). If 0 compression threads, the compression executor can be an inline executor. Submission to the compression stage could in a follow up change be submitted by a mutator rather than the sync thread, and this would permit smaller-than-segment batches to be submitted, so that there is both less delay between sync starting and flushing, and so that parallelism can be achieved for flush of a single segment. The main advantage, though, is it is easier to reason about than multiple sync threads. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14284186#comment-14284186 ] Branimir Lambov commented on CASSANDRA-6809: Multiple sync threads weren't really supported by the code, but it wasn't very hard to make it work. I updated [the branch|https://github.com/blambov/cassandra/tree/compressed-cl] to not rely on synchronization for all writing and added an option to use more than one thread for compression. With this LZ4 compressed logs can surpass uncompressed even on SSDs. The diff is [here|https://github.com/blambov/cassandra/commit/873cc2bc147e4e1e8209e79c60ea5cd295d2da42]; a large portion of it is code indented differently (Is there any way to make github recognize this?). Admittedly this solution doesn't use threads optimally (each thread still waits for its writes to materialize), but IMHO is straightforward and simple. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: ComitLogStress.java, logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14277638#comment-14277638 ] Branimir Lambov commented on CASSANDRA-6809: Thanks for the review, Ariel. The plan was to implement compression without introducing too much extra complexity thus I've stayed away from adding new queues and stages dedicated to compression. Even so, a pipeline that achieves the highest possible throughput is quite doable by simply using more than one sync thread, and the size of the compression window is easily controlled via the sync period. Since we don't have an incremental compression mechanism, compression necessarily has to happen at the end, i.e. when the whole of the set of mutations to compress has been written to the buffer (this is usually not at the end of the segment). CASSANDRA-7075 is valuable on its own right. Arguably RAID 0 is not good enough in either performance or reliability. None of the extra complexity we introduce there is made necessary by compression-related concerns, but one of the side effects of it is the availability of more than one sync thread for compression. It is a solution of sorts to the lack of CPU saturation from this patch, but it is not at all the only way to achieve it. This code was written before ByteBuffer compression was made available; I will definitely make use of that now, but I wonder if that should not be a separate patch so that we don't have to block on/conflict with Jake's patch. The playback tests are in the various RecoveryManagerTests in o.a.c.db; the tests are the same for the uncompressed (test/testold target) and compressed case (test-compressed target). For performance tests the ultimate measure is cassandra-stress; ComitLogStress is a simple microbenchmark of how much we can push that favors compression -- make sure to run it with periodic rather than batch sync. A latency test is probably needed for batch mode; this probably needs some changes to the service to make sure the sync period can go low enough for the write latency to show up. I have not looked at the other suggestions yet; I have to switch modes from the quite different node allocation work-- give me a couple of days. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14275726#comment-14275726 ] Ariel Weisberg commented on CASSANDRA-6809: --- I finished my review. Comments are in the pull request. It looks good and could ship as is. I have some thoughts about potential scope creep I would advocate for. Also some other directions for enhancement the commit log could go in as well as some reservations about performance in some cases. I only just noticed CommitLog stress so I need to check that out so I can understand the numbers and what is being tested. RE CASSANDRA-7075 multiple CL disks. I see this as a work around for not having RAID-0 of the volumes being used for the CL and that is it. And that may introduce it's own load balancing issues as well as a mess of code for scattering/gathering mutations that I am less comfortable with. Writing a CL pipeline that can do the maximum supported sequential IO to a single file is doable, and if I had a choice it is what I would rather write. From a user perspective it is a nice feature to not to be forced to provide a RAID volume and to me that should be the primary motivation. Also fascinating (to me) piece of trivia. When I tested in the past I could call force() on a mapped byte buffer far fewer times then I could call force() on a FileChannel. So if I had a battery backed disk controller and I appended a page (in a preallocated file) and called force() in a loop with a MappedByteBuffer it would do a few hundreds syncs a second, but with FileChannel.force it would do a few thousand. MBB was slow enough to be a concern for synchronous commits. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14275865#comment-14275865 ] Ariel Weisberg commented on CASSANDRA-6809: --- Moving conversation to JIRA Ariel {noformat} What is the test coverage, unit or otherwise for compressed/uncompressed playback? It would be good to see the new code produce the mutations out of both compressed and uncompressed segments. I think that this kind of performance work isn't done without a micro benchmark so we know where we stand and so the next person to work on the CL can track the impact of their changes. I would really like to see a JMH test added that evaluates the compressed and uncompressed versions for throughput for tiny mutations as well as normal (say 1k). When testing compression I have found that generating compressible data doesn't seem to work well. I never got the advertised compression rates out of Snappy. You might need to grab a real corpus like some HTML or the text of alice and wonderland to give the compressor something reasonable to chew on. I have some concern about waiting to do compression until the end. It's fine in the common case and configuration, but there are situations where you want to start the flush immediately and it would be nice if it compressed and wrote a few hundred kilobytes at a time. You haven't taken away the mapped option so latency sensitive users can opt for that and it can be enhanced later if there is demand. Compression is single threaded and any time spent in force() can't also be used to do compression. This can really cut into compression throughput and it is a function of the latency of force() which varies. With synchronous commits it's can have an impact. The ICompressor interface and some of the implementations are also a little limiting. You should be able to compress the data directly into the memory mapped file. Snappy java supports that although LZ4 doesn't appear to. It also makes more sense to compress and then checksum. There is less data to checksum after compression, and another advantage of getting the order right is that you can use a JNI call to a native checksum implementation which will be much faster. You can use a larger checksum like murmur3 over each compressed block. The checksum will be better then a 4-byte checksum, but use less space overall. Yet another advantage is that Murmur3 doesn't have a lookup table evicting random cache lines (on every core) as part of every single mutation that flows through C*. I only bring this stuff up because this is a format change and getting it all done at once might make the amount of multi-version support code that has to be written smaller as opposed to making these changes incrementally across multiple versions. {noformat} Benedict {noformat} I'm not really sure JMH is the right tool for disk bound workloads? To add to Ariel's excellent points, another thing to consider is recycling the ByteBuffer we use, and potentially also use a DirectByteBuffer. As it stands this will create a non-trivial amount of garbage that may live long enough to be promoted (what with being disk bound). {noformat} Ariel {noformat} JMH doesn't solve the hard problems of for disk bound workloads certainly. It doesn't hurt either and it has some support for parameterizing and reporting. I didn't know about CommitLogStress when I asked about JMH. We don't need to build something new if something is already in place. I thought about the ByteBuffer pooling and lack of DBB. ICompressor doesn't accept DBB for input or output. The allocation is large so it will go straight into the old generation. C* runs CMS pretty regularly, but maybe not that regularly. It might make sense to pool two of them and if anymore are needed beyond that to allocate and discard. Since the allocation is CL segment size it's feels like a lot of memory to pin for one purpose. {noformat} Benedict {noformat} CommitLogStress is not very good, but JMH is architected around microbenchmarks or very small operations, which this doesn't really fit. We now support DBB in LZ4, so there's no reason to not roll it out here. Pooling as many as we need makes sense to me, no reason to constrain ourselves to just 2? Perhaps periodically prune if instances aren't being used. We're currently pinning as much memory as we're ahead of disk by, although admittedly here we only need it so long as we're writing to the page cache. Which is perhaps another optimisation - the buffer should be recycled/discarded as soon as we've compressed its contents. {noformat} Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee:
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14270142#comment-14270142 ] Jason Brown commented on CASSANDRA-6809: Please reassign as I'm not going to realistically get a chance to review for the next few weeks. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14269301#comment-14269301 ] Aleksey Yeschenko commented on CASSANDRA-6809: -- [~jasobrown] Are you still interested in reviewing this, or should I reassign the reviewer? We kinda want this to happen in 3.0. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14140658#comment-14140658 ] Branimir Lambov commented on CASSANDRA-6809: Patch is available for review at https://github.com/blambov/cassandra/pull/2: {panel} Provides two implementations of commit log segments, one matching the previous memory-mapped writing method for uncompressed logs and one that uses in-memory buffers and compresses sections between sync markers before writing to the log. Replay is changed to decompress these sections and keep track of the uncompressed position to correctly identify the replay position. The compression class and parameters are specified in cassandra.yaml and stored in the commit log descriptor. Tested by the test-compression target, which now enables LZ4Compression of commit logs in addition to compression for SSTables. {panel} [~jasobrown]: Using a writer interface will probably be a little cleaner from a design point of view, but if we want to preserve all features of the current approach the two writing methods and the log segment class are so tightly coupled that it doesn't really matter. The measurements I did compared the number of commit log writes of a fixed size that one could perform in a given time period (the ComitLogStress test introduced in CASSANDRA-3578 and slightly updated here). Memory-mapped IO does seem to provide some benefit at least on Windows, which for me means we should not be removing it yet. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14140668#comment-14140668 ] Benedict commented on CASSANDRA-6809: - Thanks, I'll get to review this soon. As Jonathan mentioned just a moment ago in another ticket, just posting the branch is sufficient, since JIRA is our main history for discussion etc., and if we ever want to comment code directly that's possible without a pull request. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14141286#comment-14141286 ] Jason Brown commented on CASSANDRA-6809: I'd also like to have a chance to review, as well. I'll try to get to it over the next few days. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14141292#comment-14141292 ] Jason Brown commented on CASSANDRA-6809: [~blambov] The github link you add just bring me to a page that says There isn't anything to compare. Tried looking at your c* repo, as well, on github but no obvious branches to poke at. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14141293#comment-14141293 ] Benedict commented on CASSANDRA-6809: - [here|https://github.com/blambov/cassandra/tree/compressed-cl] Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14141321#comment-14141321 ] Jason Brown commented on CASSANDRA-6809: I think I read compressed-ci as compressed-cli and completely skipped past it :). Thanks, [~benedict] Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14122738#comment-14122738 ] Branimir Lambov commented on CASSANDRA-6809: I'm doing some testing before deciding what exactly to implement, and I'm seeing that the current implementation of memory-mapped IO for commit logs does not help slow HDDs, but has a very noticeable effect (10-20%) on SSDs. It also appears SSD writing is CPU-bound and not helped by compression. Based on this and the discussion above, my plan for implementing compression is the following: * the current CommitLogSegment will become an abstract base class with two subclasses that define how the buffer accepting the mutation is constructed, and how data is written from it. * one subclass will be the current implementation, with a memory-mapped buffer to achieve the least CPU overhead and generally provide the fastest path for SSDs when encryption is not required. * the other will construct in-memory buffers of the size of the CLS, and compress the sync sections before writing to a FileChannel (in the sync thread). It should be trivial to add encryption to this. * compression should improve performance on any medium if done before encryption, so I do not think it makes any sense to add support for encryption to the memory-mapped option at all. The ComitLogStress test I am using cannot measure the effect of recycling commit log files, and it seems that any change in that is orthogonal to adding support for compression. I will leave reevaluating the need for recycling for a separate ticket. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14122754#comment-14122754 ] Benedict commented on CASSANDRA-6809: - I'm +1 on your over all intended approach, but I'd appreciate some elaboration on what steps you've taken for testing: * What platforms? * What CPU config? * What measurement of CPU time allocation? (10-20% bump may be reasonable if its kernel time, but I'm a little surprised it has such an impact, as we'd be talking about bulk writing large quantities of linear data, which should have reasonably low CPU impact) If you could post your in progress branches and tests to github so I can follow what you're doing, that would be great Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14123492#comment-14123492 ] Jason Brown commented on CASSANDRA-6809: [~barnie] Your proposal is very similar to to what I've implemented wrt to adding encryption to the commitlog. Instead of subclassing CLS, I created a new interface, hastily named SegmentWriter, with two impls (one non-encrypted, the other encrypted), which is then referenced from CLS. More or less, the same as what you are thinking, I'm sure. Also, It's not clear to me what you are measuring: bq. I'm seeing that the current implementation of memory-mapped IO for commit logs does not help slow HDDs, but has a very noticeable effect (10-20%) on SSDs Is this the time for the sync() call to complete? So, then, you are evaluating the value of using mmap? If we're not planning on using mmap, are you thinking about buffering data ourselves (which is then flushed by the sync thread)? Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 Attachments: logtest.txt It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103569#comment-14103569 ] Jason Brown commented on CASSANDRA-6809: bq. we may as well allocate on the sync thread(s) ... Wait, so I'm confused a bit now. Are you still proposing we go with your initial proposal as stated in the first comment of this ticket, with micro-segments and sync threads? Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103580#comment-14103580 ] Benedict commented on CASSANDRA-6809: - Per-disk sync threads... Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103622#comment-14103622 ] Jason Brown commented on CASSANDRA-6809: bq. If we're dropping recycling, ... bottlenecking anything. Reread this paragraph several times, now it makes sense. I wasn't thinking about the write perf, necessarily, but about having the file contiguous on disk. However, since the commit log files are, more or less, one-time use (meaning, we're not doing tons of random nor sequential I/O reads on them), I guess worrying about a large contiguous block on disk isn't necessary. bq. Per-disk sync threads I'm still not sure sync threads are totally necessary. If you are worried about the time for the mmap'ed buffers to flush in the same thread that's handling all the CL entry processing + any possible compression or encryption, a simple solution might be to have a sync thread that merely invokes the mmap buffer flush. Thus, the main CL thread(s) can continue processing the new entries and writing to the mmap buffer, but the sync thread eats the cost of the msync. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103626#comment-14103626 ] Jason Brown commented on CASSANDRA-6809: If we did try an async flush of the buffers in a sync thread, I think it would be instructive to actually measure that it is demonstratively beneficial, rather than assuming that it is. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103661#comment-14103661 ] Benedict commented on CASSANDRA-6809: - bq. Thus, the main CL thread(s) can continue processing the new entries and writing to the mmap buffer, I think you're still thinking in 2.0 terms. In 2.1 the mutation threads write to the commit log buffer directly. We will need these buffers to be purely in memory (not mmapped) and then passed to the sync threads when ready for writing to be compressed and written to disk. bq. I wasn't thinking about the write perf, necessarily, but about having the file contiguous on disk I'm not convinced this will behave any differently performance-wise (i.e. if we were to preallocate the file's size), however we don't now know the size of the file we'll be writing, so pre-allocating doesn't really help much anymore. bq. a simple solution might be to have a sync thread that merely invokes the mmap buffer flush But this flush is synchronous? Specifically, we want multiple to be in flight at once. I also do not think there's any benefit to using mmap any more, since you bring it up. We should switch to regular output streams, so that we can simply wrap it in a compressed output stream. If you mean to start using async IO in Java, this isn't really any superior here - it still involves extra threads with a hidden thread pool, and extra (IMO) complexity beyond simply performing it on the relevant thread, but also would involve compressing the entire segment before writing to the file, which introduces extra latency (or having a complex to-and-fro shuttle compressed data to an async sink, waiting for result, etc), for no benefit. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103680#comment-14103680 ] Jason Brown commented on CASSANDRA-6809: bq. I think you're still thinking in 2.0 terms True, I haven't caught up with the CL changes for 2.1, but in my defense, I need to ship something ASAP on 2.0 :). Will do my homework on 2.1. bq. I'm not convinced this will behave any differently performance-wise Yeah, me neither. Perhaps my meandering prose didn't express that I think we can retire the pre-allocate with the segment recycle. bq. mmap We can still use mmap with OutputStream, in fact I've done it already on my local hacking. However, we should evaluate if we want the mmap functionality. I haven't thought enough about it to have a strong opinion. Either way, you still go through the page cache, which is the part I'd like to avoid. bq. If you mean to start using async IO in Java ... Nope, at least, not here :) Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103687#comment-14103687 ] Benedict commented on CASSANDRA-6809: - Sounds like we're approximately in agreement then :-) FTR, I don't really want to incorporate these changes pre-3.0 on the mainline (though doesn't sound like you're suggesting it)... Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14102212#comment-14102212 ] Jason Brown commented on CASSANDRA-6809: So, I'll be honest, I'm not sure why this complicated solution is better than using a simple file-level compression (conceptually, something like adding another OutputStream decorator). I don't believe we need a slavish fixation on always staying within the pre-allocated size bound of the file - using less than what's declared in the file is a minor inefficiency, and using more (making the file larger) should just write out the extra data anyways (perhaps not contiguously), perhaps with a minor/trivial penalty (one that I'm not sure is measurable / affects the real-world behavior of cassandra). The file will get resized to the declared size in the yaml upon recycling, anyways, so I don't see a real disk consumption problem. As I'm working on CASSANDRA-6018, the simpler solution is looking very sensible and is easy to reason about. Admittedly, I'm building that on top of 2.0 and haven't even begun thinking about merging into 2.1 (and the changes that have happened there). But, I think before we go down this of micro-segments and more threads/pools, can we reevaluate simpler techniques that do not render the codebase more complicated for what is arguably not that big of a win in the first place? Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14102247#comment-14102247 ] Benedict commented on CASSANDRA-6809: - bq. is better than using a simple file-level compression * Don't want to bound by a single CPU performing compression/sync * Reduced file system fragmentation through obeying the size limits (less and less of a problem as we move to SSDs, but for HDDs pretty important for CL performance) Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14102901#comment-14102901 ] Jonathan Ellis commented on CASSANDRA-6809: --- bq. Don't want to bound by a single CPU performing compression/sync But I'm okay with being bound by a single cpu *per disk*, and CASSANDRA-7075 will allow multiple disks. bq. Reduced file system fragmentation through obeying the size limits I'm okay with taking that minor hit, too. Simple is a feature. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103210#comment-14103210 ] Benedict commented on CASSANDRA-6809: - It's also potentially less capable of utilising machines with small numbers of high performance disks (i.e. compression is generally slower than many SSDs can now write), so it may not be future proof, and won't utilise all of our users' hardware maximally, but I doubt that will be a major issue for the moment and it should at least improve the current situation. If we're comfortable with that, and with dropping CLS recycling (which I am, I was never really convinced it was worth the effort) we can simplify things quite a lot. So I'm cool with that approach. We'll want to limit the concurrency in this case, however, to e.g. 25% of num CPUs (so we're only 50% of physical cores on HT CPUs), as occupying every CPU with compressing/encrypting large blocks of data will result in potentially large latency spikes. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103211#comment-14103211 ] Jonathan Ellis commented on CASSANDRA-6809: --- Sounds reasonable. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103387#comment-14103387 ] Jason Brown commented on CASSANDRA-6809: bq. It's also potentially less capable of utilising machines with... / won't utilise all of our users' hardware maximally Let's face it, if you're hitting the CL hard, you're also hitting the memtables, et al, pretty hard, as well, due a consistently large number of writes coming in. As long as the CL doesn't fall behind/take up too many resources/does it's job and doesn't complain, I think that's the sweet spot. I'd rather have the resources optimized for the client-facing path. bq. dropping CLS recycling ... never really convinced it was worth the effort I tend to agree with this. I'd like to think it's a nice win, but I have no numbers to back it up. I think the pre-allocation is good (and a cheap operation, at least by LOC), but the recycling I'm not sure of. [~rbranson], it's been a few years since you implemented this in CASSANDRA-3411, how do feel about it now, with several years of production experience? bq. We'll want to limit the concurrency in this case Does this statement apply to this ticket or CASSANDRA-7075? Seems more apropos to the later. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103412#comment-14103412 ] Benedict commented on CASSANDRA-6809: - bq. Does this statement apply to this ticket or CASSANDRA-7075? Seems more apropos to the later. CASSANDRA-7075 has no reason to gate the concurrency by anything other than available disks, so it would be a bit out of context there, however without CASSANDRA-7075 this obviously isn't an issue here. Maxing out one CPU with compression would be very easy, however, so I suggest this ticket now depends on CASSANDRA-7075. bq. I think the pre-allocation is good (and a cheap operation, at least by LOC) I disagree. If we're dropping recycling, we may as well allocate on the sync thread(s). No point having an extra thread and moving part when the vast majority of time is spent writing to disk. We're talking a few hundred micros extra cost, across an entire sync, which only ever stops a sync from progressing if the disk isn't keeping up, at which point we're done for anyway. With a 10s sync periodic, a cost of even 1ms is 0.01% of total disk bandwidth lost, and 1ms is a greatly exaggerated cost for creating a file. We also only pay this each time we rollover a segment, so by default at most once every 32Mb, 1ms every 32Mb is a 32Gb/s throughput rate, which is never going to come close to bottlenecking anything. bq. Let's face it, if you're hitting the CL hard, you're also hitting the memtables, et al, pretty hard, as well, due a consistently large number of writes coming in. As long as the CL doesn't fall behind/take up too many resources/does it's job and doesn't complain, I think that's the sweet spot. I'd rather have the resources optimized for the client-facing path. Not necessarily. You could be doing in place updates, or you could have very large memtables (esp. with offheap memtables now) which could easily outpace CL without overwrites, without even needing to flush memtables. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14103417#comment-14103417 ] Benedict commented on CASSANDRA-6809: - That said, if we have a separate thread per disk, we can have it allocate its next file _after_ its sync completes, thereby saving this cost most of the time too but avoiding any asynchronicity complexity Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: Branimir Lambov Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13978921#comment-13978921 ] Benedict commented on CASSANDRA-6809: - I would probably go one step further still: # CommitLogSegment should be split into two concepts: the on-disk file and new (much smaller) in-memory micro-segments/chunks # Micro-segments largely replace current CL segments as far as writers are concerned, but once full they're placed on a queue (without reordering) to be compressed; # The compression queue is processed by potentially multiple threads, and whilst it may complete out of order, the results are consumed in-order only # Once compressed, they're placed on a new queue for the sync thread; whether or not they are synced immediately or once a full on-disk segment's worth is buffered is probably unimportant; probably it would be good to make this configurable When an out-of-band, periodic or batch sync occurs, we force finish the current micro-segment, and consider ourselves synced() once that segment has worked its way through the pipeline and hit the disk There are at least a few other issues to consider: # We need to be able to create oversized micro-segments for when our mutations are larger than this # We probably want to bound the amount of memory we can have waiting outstanding for a sync. Since we are currently limited by the buffer cache, one simple solution might be to have separate sync and flush threads, so that we can quickly flush data into the OS buffer cache without waiting for a sync. Otherwise we can manage our own internal dynamically sized pool. # This design might permit us to to simplify segment file allocation (CommitLogSegmentManager)? All in all it's not a trivial change if we want to do it right, and it is probably not going to benefit _that_ many users, so I'm inclined to put this one on the back burner for now. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: T Jake Luciani Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6809) Compressed Commit Log
[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13978939#comment-13978939 ] T Jake Luciani commented on CASSANDRA-6809: --- bq. All in all it's not a trivial change if we want to do it right, and it is probably not going to benefit that many users, so I'm inclined to put this one on the back burner for now. I'm inclined to agree, I'm not sure how big the gain would be so I'd like to experiment first anyway. Compressed Commit Log - Key: CASSANDRA-6809 URL: https://issues.apache.org/jira/browse/CASSANDRA-6809 Project: Cassandra Issue Type: Improvement Reporter: Benedict Assignee: T Jake Luciani Priority: Minor Labels: performance Fix For: 3.0 It seems an unnecessary oversight that we don't compress the commit log. Doing so should improve throughput, but some care will need to be taken to ensure we use as much of a segment as possible. I propose decoupling the writing of the records from the segments. Basically write into a (queue of) DirectByteBuffer, and have the sync thread compress, say, ~64K chunks every X MB written to the CL (where X is ordinarily CLS size), and then pack as many of the compressed chunks into a CLS as possible. -- This message was sent by Atlassian JIRA (v6.2#6252)