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