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

Reply via email to