[jira] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2015-03-24 Thread Dave Brosius (JIRA)

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

Dave Brosius edited comment on CASSANDRA-6809 at 3/25/15 12:45 AM:
---

This commit did not apply cleanly. An issue (at least) between 
DatabaseDescriptor.java and Config.java. Please take a look.

oh and nit: Parameterized is spelled wrong :)


was (Author: dbrosius):
This commit did not apply cleanly. An issue (at least) between 
DatabaseDescriptor.java and Config.java. Please take a look.

> 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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2015-03-13 Thread Benedict (JIRA)

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

Benedict edited comment on CASSANDRA-6809 at 3/13/15 7:31 PM:
--

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) probably gives us pretty good behaviour without any extra 
tuning knobs


was (Author: benedict):
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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2015-01-20 Thread Branimir Lambov (JIRA)

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

Branimir Lambov edited comment on CASSANDRA-6809 at 1/20/15 6:47 PM:
-

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 (tested with 30ms periodic sync which to me makes much more sense the huge 
default). 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.


was (Author: blambov):
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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2015-01-13 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg edited comment on CASSANDRA-6809 at 1/13/15 8:20 PM:


Moving conversation to JIRA

Ariel
{quote}
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.
{quote}
Benedict
{quote}
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).
{quote}

Ariel
{quote}
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.
{quote}

Benedict
{quote}
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.
{quote}



was (Author: aweisberg):
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 uncompress

[jira] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2015-01-13 Thread Ariel Weisberg (JIRA)

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

Ariel Weisberg edited comment on CASSANDRA-6809 at 1/13/15 7:26 PM:


I finished my review. Comments are in the pull request. It looks good and could 
ship as is modulo some minor changes and if the the right tests are there (or 
get created). 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.



was (Author: aweisberg):
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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2014-09-19 Thread Jason Brown (JIRA)

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

Jason Brown edited comment on CASSANDRA-6809 at 9/19/14 9:02 PM:
-

[~blambov] The github link referenced above just brings 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.


was (Author: jasobrown):
[~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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2014-09-05 Thread Branimir Lambov (JIRA)

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

Branimir Lambov edited comment on CASSANDRA-6809 at 9/5/14 11:25 AM:
-

Raw test data attached in logtest.txt.


was (Author: blambov):
Raw test data.

> 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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2014-08-20 Thread Jason Brown (JIRA)

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

Jason Brown edited comment on CASSANDRA-6809 at 8/20/14 9:35 AM:
-

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 for the encrypted commit logs. 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 :)


was (Author: jasobrown):
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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2014-08-20 Thread Jason Brown (JIRA)

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

Jason Brown edited comment on CASSANDRA-6809 at 8/20/14 8:42 AM:
-

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. I'm not immediately sure how to measure that, 
but it's near 2am and coffee time doesn't begin for several more hours.


was (Author: jasobrown):
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] [Comment Edited] (CASSANDRA-6809) Compressed Commit Log

2014-08-20 Thread Jason Brown (JIRA)

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

Jason Brown edited comment on CASSANDRA-6809 at 8/20/14 8:37 AM:
-

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, in the manner initially described above, 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.


was (Author: jasobrown):
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)