You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Sylvain Lebresne (JIRA)" <ji...@apache.org> on 2016/07/05 14:55:11 UTC

[jira] [Commented] (CASSANDRA-11828) Commit log needs to track unflushed intervals rather than positions

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

Sylvain Lebresne commented on CASSANDRA-11828:
----------------------------------------------

s I like the general approach, I'd like to discuss where we should commit this as this is a non-trivial patch and touching critical part of the code.

As CASSANDRA-11448 created this bug, I assume 2.1 onwards is affected? But it's only easy to fix on 2.2+ due to CASSANDRA-9669, correct?

Assuming my understanding is correct, wouldn't it be simpler/less risky to just revert CASSANDRA-11448 for 2.1 and 2.2? Getting your flush writer to die when your run out of space isn't ideal, but it feels somewhat minor compared to possibly losing data.

Anyway, with that out of the way, some remarks on the patch:
* In {{CommitLogSegment}}, the "semi-lack" of abstraction of {{IntegerIntervals}} is a bit confusing, and I'm not sure it's justified performance-wise. For instance, it takes more time that necessary to understand that {{cfDirty}} also actually holds an interval, not just a position. I'd rather create a true {{IntInterval}} class (which can use a {{long}} internally if it wants, but I'm not sure that's justified), with maybe a mutable {{AtomicIntInterval}} sublcass having the {{expandToCover}} ({{coverInMap}} imo belongs to {{CommitLogSegment}}). Of course, {{IntInterval.Set}} can very well continue to use arrays internally since we never iterate on the set except for the tests (so in practice we'll rarely create actual {{IntInterval}} objects).
* I don't think {{IntegerIntervals.Set.add()}} needs to be synchronized (it's called only a synchronized method in practice). It's not a big deal, but making the class thread-unsafe will be imo more expected in case of future reuse.
* The new classes introduced ({{IntegerIntervals}} and {{ReplayIntervalSet}}) lack a minimum of javadoc, and could use a little bit more comments in general.
* Variable naming in {{CommitLogSegment}} is now confusing. Should rename {{cleanPos}} to {{cleanInterval}} etc. Some comment should also be updated (at least on top of {{cfDirty}}/{{cfClean}} declarations and in {{CommitLogReplayer}}) accordingly.
* At the end of {{CommitLogTest.testOutOfOrderLogDiscard}}, I'd add a comment on the last assert saying something like "In the absence of error, this should be 0 because forceRecycleAllSegments would have cleaned all segment. Because we know there was an error, we want to have something to replay" (took me a minute to figure out what that test was really testing).
* I'm not sure to understand why {{CommitLogReplayer.firstNotCovered()}} uses the the first range {{getValue()}} instead of the {{getKey()}} (i.e. the {{lowerBound()}}). Also, leaving the {{ranges}} in {{ReplayIntervalSet}} private and using properly named accessors would be clearer imo.
* We should fix the TODO in {{Tracker}} (not sure I understand it fully, but I would suspect it's fine to not notify an invalidated CF).

Nits:
* there is a few use of {{Integer}} in IntegerIntervals.Set where {{int}} would be fine.
* there is a few inlined usage of {{getCurrentColumnFamilyStore()}} in {{CQLTester}}, can you replace them by calling the new method instead?
* A few tests (in {{CommitLogTest.testUnwriteableFlushRecovery}} and {{IntegerIntervalsTest}}) don't uses braces after {{if}}/{{for}} even though the body is multi-line. I find it a bit esoteric and inconsistent with the code base, which hurts reading imo.



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