[
https://issues.apache.org/jira/browse/CASSANDRA-9669?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14704693#comment-14704693
]
Branimir Lambov commented on CASSANDRA-9669:
--------------------------------------------
The code looks good now, +1.
bq. the ranges are inclusive-start, exclusive-end, the inverse of what you
expected
Yes, they are the inverse of what I expected, but that's actually
start-exclusive, end-inclusive, the [shouldReplay
comment|https://github.com/apache/cassandra/commit/459b96333c84837fe757d706c2a6be91b8b27f2e#diff-2b76f3efa4aaa38339ab8f4869c9b7bfR88]
is correct. I'd also add "as a mutation's replay position is assigned after it
is added to the log" to it.
Deleting the files at [the start of the
test|https://github.com/apache/cassandra/commit/459b96333c84837fe757d706c2a6be91b8b27f2e#diff-b3802331f8dcba05356ad47ee54fe6dfR321]
may cause problems on Windows, but I don't know how to this safely in 2.0.
> If sstable flushes complete out of order, on restart we can fail to replay
> necessary commit log records
> -------------------------------------------------------------------------------------------------------
>
> Key: CASSANDRA-9669
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9669
> Project: Cassandra
> Issue Type: Bug
> Components: Core
> Reporter: Benedict
> Assignee: Benedict
> Priority: Critical
> Labels: correctness
> Fix For: 3.x, 2.1.x, 2.2.x, 3.0.x
>
>
> While {{postFlushExecutor}} ensures it never expires CL entries out-of-order,
> on restart we simply take the maximum replay position of any sstable on disk,
> and ignore anything prior.
> It is quite possible for there to be two flushes triggered for a given table,
> and for the second to finish first by virtue of containing a much smaller
> quantity of live data (or perhaps the disk is just under less pressure). If
> we crash before the first sstable has been written, then on restart the data
> it would have represented will disappear, since we will not replay the CL
> records.
> This looks to be a bug present since time immemorial, and also seems pretty
> serious.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)