[ https://issues.apache.org/jira/browse/CASSANDRA-11828?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15404040#comment-15404040 ]
Branimir Lambov commented on CASSANDRA-11828: --------------------------------------------- Thank you for the thorough review and the good suggestions. Applied them as a new commit in the 3.0 version: |[3.0|https://github.com/blambov/cassandra/tree/11828-revert-compaction-wait-3.0]|[utest|http://cassci.datastax.com/job/blambov-11828-revert-compaction-wait-3.0-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-revert-compaction-wait-3.0-dtest/]| except removing the synchronization from {{IntegerIterval.Set.add}} as the individual interval class is thread-safe and hence the consistency is better if the set class also is (also, this is a longer-running operation so synchronization is preferable to compare-and-set). Additionally untied {{IntervalSet}} from {{ReplayPosition}} and did a couple of smaller cleanups. Please take another look. On the topic of where it should be applied, I agree it is a better idea to leave it out of 2.x. I wouldn't completely revert CASSANDRA-11448 -- it is still preferable to die/stop the transports on error, but we shouldn't be passing control on to the post-flush. I will prepare a patch that does this next. > Commit log needs to track unflushed intervals rather than positions > ------------------------------------------------------------------- > > Key: CASSANDRA-11828 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11828 > Project: Cassandra > Issue Type: Bug > Components: Local Write-Read Paths > Reporter: Branimir Lambov > Assignee: Branimir Lambov > Fix For: 2.2.x, 3.0.x, 3.x > > > In CASSANDRA-11448 in an effort to give a more thorough handling of flush > errors I have introduced a possible correctness bug with disk failure policy > ignore if a flush fails with an error: > - we report the error but continue > - we correctly do not update the commit log with the flush position > - but we allow the post-flush executor to resume > - a successful later flush can thus move the log's clear position beyond the > data from the failed flush > - the log will then delete segment(s) that contain unflushed data. > After CASSANDRA-9669 it is relatively easy to fix this problem by making the > commit log track sets of intervals of unflushed data (as described in > CASSANDRA-8496). -- This message was sent by Atlassian JIRA (v6.3.4#6332)