[ https://issues.apache.org/jira/browse/CASSANDRA-6809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=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)