cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Benjamin Lerer (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-10971) Compressed commit log has no backpressure and can OOM
Date Mon, 22 Feb 2016 20:33:18 GMT

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

Benjamin Lerer commented on CASSANDRA-10971:
--------------------------------------------

If I am not mistaken, a {{CompressedSegment}} or {{FileDirectSegment}} will release its buffer
once it has been fully written to the disk whereas segments will stay active until they are
recycled. By consequence, it might be better to use as limit the number of non fully written
segments rather than the number of active ones. 
It seems that it could be done by counting the number of available segments which have a non-null
buffer. 

As a nit, I think it might be safer to use an {{instanceof FileDirectSegment}} for {{enforceSegmentLimit}}
in trunk. In case somebody decide to add a new sub-class to {{FileDirectSegment}} 

> 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: Bug
>          Components: Local Write-Read Paths
>            Reporter: Ariel Weisberg
>            Assignee: Ariel Weisberg
>             Fix For: 3.0.x, 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)

Mime
View raw message