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

Reply via email to