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

Branimir Lambov edited comment on CASSANDRA-11828 at 5/20/16 8:53 AM:
----------------------------------------------------------------------

Patch uploaded here:
|[2.2|https://github.com/blambov/cassandra/tree/11828-cl-ss-intervals-2.2]|[utest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-2.2-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-2.2-dtest/]|
|[3.0|https://github.com/blambov/cassandra/tree/11828-cl-ss-intervals-3.0]|[utest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-3.0-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-3.0-dtest/]|
|[trunk|https://github.com/blambov/cassandra/tree/11828-cl-ss-intervals]|[utest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-dtest/]|

Changes the commit log segment dirty and clean tracking to intervals: one dirty 
interval per cf that covers the span of writes to that cf, and a set of clean 
intervals (which would normally be a single contiguous one). The segment is 
only discarded if the clean set completely covers the dirty interval; if a 
failed flush left a hole the segment will remain.

Sstables are also changed to track covered replay intervals so that compaction 
that includes a table flushed after a failed one doesn't obscure the unflushed 
region from the commit log replayer.


was (Author: blambov):
Patch uploaded here:
|[2.2|https://github.com/blambov/cassandra/tree/11828-cl-ss-intervals-2.2-rebased]|[utest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-2.2-rebased-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-2.2-rebased-dtest/]|
|[3.0|https://github.com/blambov/cassandra/tree/11828-cl-ss-intervals-3.0-rebased]|[utest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-3.0-rebased-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-3.0-rebased-dtest/]|
|[trunk|https://github.com/blambov/cassandra/tree/11828-cl-ss-intervals-rebased]|[utest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-rebased-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-cl-ss-intervals-rebased-dtest/]|

Changes the commit log segment dirty and clean tracking to intervals: one dirty 
interval per cf that covers the span of writes to that cf, and a set of clean 
intervals (which would normally be a single contiguous one). The segment is 
only discarded if the clean set completely covers the dirty interval; if a 
failed flush left a hole the segment will remain.

Sstables are also changed to track covered replay intervals so that compaction 
that includes a table flushed after a failed one doesn't obscure the unflushed 
region from the commit log replayer.

> 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)

Reply via email to