[
https://issues.apache.org/jira/browse/CASSANDRA-10971?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15150187#comment-15150187
]
Benjamin Lerer commented on CASSANDRA-10971:
--------------------------------------------
My understanding of the problem is that in the case where the commit log cannot
flush to the disk fast enough, due to the compression overhead, the
{{CommitLogSegmentManager}} will keep on creating new {{CompressedSegments}}.
As each of those segments will use a new buffer (the ones of the pool being all
in use), Cassandra can run out of memory.
Will it not be simpler to add backpressure by limiting the number of active
segments?
What I mean is, if the {{CommitLogSegmentManager}} stops allocating new
segments once a certain number of active segments has been reached, it will
make the {{CommitLog.add}} method blocking until some segments have been
reclaimed.
It seems to me that, even in the case of {{MemoryMappedSegment}}, we should be
able to apply back pressure, if the disk cannot handle the load. Am I wrong on
that?
As I am not a CommitLog expert I might have missed something.
> Compressed commit log has no backpressure and can OOM
> -----------------------------------------------------
>
> Key: CASSANDRA-10971
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10971
> Project: Cassandra
> Issue Type: Improvement
> Components: Local Write-Read Paths
> Reporter: Ariel Weisberg
> Assignee: Ariel Weisberg
> Fix For: 3.x
>
>
> I validated this via a unit test that slowed the ability of the log to drain
> to the filesystem. The compressed commit log will keep allocating buffers
> pending compression until it OOMs.
> I have a fix that am not very happy with because the whole signal a thread to
> allocate a segment that depends on a resource that may not be available
> results in some obtuse usage of {{CompleatableFuture}} to rendezvous
> available buffers with {{CommitLogSegmentManager}} thread waiting to finish
> constructing a new segment. The {{CLSM}} thread is in turn signaled by the
> thread(s) that actually wants to write to the next segment, but aren't able
> to do it themselves.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)