[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=15407579#comment-15407579 ] Branimir Lambov commented on CASSANDRA-11828: - Squashed, rebased and added {{CHANGES.txt}} entry. Pre-3.0 code (disable passing control to post-flush): |[2.1|https://github.com/blambov/cassandra/tree/11828-no-post-flush-2.1]|[utest|http://cassci.datastax.com/job/blambov-11828-no-post-flush-2.1-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-no-post-flush-2.1-dtest/]| |[2.2|https://github.com/blambov/cassandra/tree/11828-no-post-flush-2.2]|[utest|http://cassci.datastax.com/job/blambov-11828-no-post-flush-2.2-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-no-post-flush-2.2-dtest/]| 3.0 and later (track unflushed intervals in commitlog and sstables): |[3.0|https://github.com/blambov/cassandra/tree/11828-rebased-3.0]|[utest|http://cassci.datastax.com/job/blambov-11828-rebased-3.0-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-rebased-3.0-dtest/]| |[3.9|https://github.com/blambov/cassandra/tree/11828-rebased-3.9]|[utest|http://cassci.datastax.com/job/blambov-11828-rebased-3.9-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-rebased-3.9-dtest/]| > 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)
[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=15406047#comment-15406047 ] Sylvain Lebresne commented on CASSANDRA-11828: -- Thanks, lgtm, I'm +1 on both patches. but could you attach patch and run tests for 2.2 (with the 2.1 merged, which seems to have conflicts) and 3.9? bq. 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 Right. My suggestion to make the base class immutable, with a specifically called out mutable (and thread-safe) variant for the case where we need it, as I usually like making immutability the default and making it clear when it's not, but I'm bikeshedding, especially as long as it's the only use of the class. I'm cool with your solution, we can always change things later if {IntegerInterval}} sees reuse and immutability is better for those reuse. > 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)
[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=15405580#comment-15405580 ] Branimir Lambov commented on CASSANDRA-11828: - Uploaded a patch for 2.1 that stops passing on control to the post-flush after an error here: |[2.1|https://github.com/blambov/cassandra/tree/11828-no-post-flush-2.1]|[utest|http://cassci.datastax.com/job/blambov-11828-no-post-flush-2.1-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-no-post-flush-2.1-dtest/]| > 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)
[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=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)
[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=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)
[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=15303986#comment-15303986 ] Branimir Lambov commented on CASSANDRA-11828: - With this change the delayed permission to compact that was introduced in CASSANDRA-9669 is no longer necessary. The branches below add a commit that removes the relevant code: |[2.2|https://github.com/blambov/cassandra/tree/11828-revert-compaction-wait-2.2]|[utest|http://cassci.datastax.com/job/blambov-11828-revert-compaction-wait-2.2-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-revert-compaction-wait-2.2-dtest/]| |[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/]| |[trunk|https://github.com/blambov/cassandra/tree/11828-revert-compaction-wait]|[utest|http://cassci.datastax.com/job/blambov-11828-revert-compaction-wait-testall/]|[dtest|http://cassci.datastax.com/job/blambov-11828-revert-compaction-wait-dtest/]| (doesn't include latest 9669 fix so index failures are expected) > 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)
[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=15293016#comment-15293016 ] Branimir Lambov commented on CASSANDRA-11828: - 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)
[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=15288783#comment-15288783 ] Branimir Lambov commented on CASSANDRA-11828: - This could potentially be a problem for disk failure policy stop as well, e.g. if a flush is requested via JMX. > 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)