You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Haoze Wu (Jira)" <ji...@apache.org> on 2022/07/18 03:23:00 UTC

[jira] [Commented] (CASSANDRA-17564) Add synchronization to wait for outstanding tasks in the compaction executor and nonPeriodicTasks during CassandraDaemon setup

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

Haoze Wu commented on CASSANDRA-17564:
--------------------------------------

[~smiklosovic] 

Hi Stefan. I'm following up on whether our scripts can work and whether you have more thoughts on this. If this issue is something we need to fix or improve in Cassandra, I guess we can try to fix it ASAP. Thank you for your time!

> Add synchronization to wait for outstanding tasks in the compaction executor and nonPeriodicTasks during CassandraDaemon setup
> ------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-17564
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-17564
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Local/Compaction
>            Reporter: Haoze Wu
>            Priority: Normal
>             Fix For: 3.11.x, 4.0.x
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> We have been testing Cassandra 3.11.10 for a while. During a node start, we found that a synchronization guarantee implied by the code comments is not enforced. Specifically, in the `invalidate` method called in this call stack (in version 3.11.10):
> {code:java}
> org.apache.cassandra.service.CassandraDaemon#main:786
> org.apache.cassandra.service.CassandraDaemon#activate:633
> org.apache.cassandra.service.CassandraDaemon#setup:261
> org.apache.cassandra.schema.LegacySchemaMigrator#migrate:83
> org.apache.cassandra.schema.LegacySchemaMigrator#unloadLegacySchemaTables:137
> java.lang.Iterable#forEach:75
> org.apache.cassandra.schema.LegacySchemaMigrator#lambda$unloadLegacySchemaTables$1:137
> org.apache.cassandra.db.ColumnFamilyStore#invalidate:542 {code}
> In line 564~570 within `public void invalidate(boolean expectMBean)`:
> {code:java}
>         latencyCalculator.cancel(false);
>         compactionStrategyManager.shutdown();
>         SystemKeyspace.removeTruncationRecord(metadata.cfId);  // line 566
>         data.dropSSTables();                                   // line 568
>         LifecycleTransaction.waitForDeletions();               // line 569
>         indexManager.invalidateAllIndexesBlocking();
> {code}
> According to the code and the comments, we suppose `data.dropSSTables()` in line 568 will submit some tidier tasks to the `nonPeriodicTasks` thread pool. Call stack in version 3.11.10:
> {code:java}
> org.apache.cassandra.db.lifecycle.Tracker#dropSSTables:233
> org.apache.cassandra.db.lifecycle.Tracker#dropSSTables:238
> org.apache.cassandra.db.lifecycle.Tracker#dropSSTables:267
> org.apache.cassandra.utils.concurrent.Refs#release:241
> org.apache.cassandra.utils.concurrent.Ref#release:119
> org.apache.cassandra.utils.concurrent.Ref#release:225
> org.apache.cassandra.utils.concurrent.Ref#release:326
> org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier#tidy:2205 {code}
> Then, `LifecycleTransaction.waitForDeletions()` in line 569 is
> {code:java}
>     /**
>      * Deletions run on the nonPeriodicTasks executor, (both failedDeletions or global tidiers in SSTableReader)
>      * so by scheduling a new empty task and waiting for it we ensure any prior deletion has completed.
>      */
>     public static void waitForDeletions()
>     {
>         LogTransaction.waitForDeletions();
>     }
> {code}
> And then call `waitForDeletions` in `LogTransaction`:
> {code:java}
>     static void waitForDeletions()
>     {
>         FBUtilities.waitOnFuture(ScheduledExecutors.nonPeriodicTasks.schedule(Runnables.doNothing(), 0, TimeUnit.MILLISECONDS));
>     }
> {code}
> From the comments, we think it ensures that all existing tasks in `nonPeriodicTasks` are drained. However, we found some tidier tasks are still running in `nonPeriodicTasks` thread pool.
> We suspect that those tidier tasks should be guaranteed to finish during server setup, because of its exception handling. In version 3.11.10, these tidier tasks are submitted to `nonPeriodicTasks` in `SSTableReader$InstanceTidier#tidy:2205`, and have the exception handling `FileUtils.handleFSErrorAndPropagate(new FSWriteError(e, file))` (within the call stack `SSTableReader$InstanceTidier$1#run:2223` => `LogTransaction$SSTableTidier#run:386` => `LogTransaction#delete:261`).
> The `FileUtils.handleFSErrorAndPropagate` handles this `FSWriteError`. We found that it checks the `CassandraDaemon.setupCompleted` flag in call stack within (`FileUtils#handleFSErrorAndPropagate:507` => `JVMStabilityInspector#inspectThrowable:60` => `JVMStabilityInspector#inspectThrowable:106` => `JVMStabilityInspector#inspectDiskError:73` => `FileUtils#handleFSError:494` => `DefaultFSErrorHandler:handleFSError:58`)
> {code:java}
>         if (!StorageService.instance.isDaemonSetupCompleted())  // line 58
>             handleStartupFSError(e);                            // line 59
> {code}
> The `handleStartupFSError` in line 59 will trigger server crash immediately. It prevents the faulty state early in the startup phase. On the other hand, if the `CassandraDaemon.setupCompleted` flag is set, we found that the server tolerates the exception, even in the stop mode in `disk_failure_policy`.
> Since those tidier tasks still appear after `LifecycleTransaction.waitForDeletions()`, we did more experiments to further confirm that if those tasks got exceptions after the `CassandraDaemon.setupCompleted` flag is set, the server will suffer from some internal issues, e.g., fail to handle table read/write. Therefore, we suspect there could be some concurrency issues — some tidier tasks are not waited.
> To figure out the root cause of this concurrency issue, we re-examine line 564~570 within `public void invalidate(boolean expectMBean)` in the CassandraDaemon main thread:
> {code:java}
>         latencyCalculator.cancel(false);
>         compactionStrategyManager.shutdown();
>         SystemKeyspace.removeTruncationRecord(metadata.cfId);  // line 566
>         data.dropSSTables();                                   // line 568
>         LifecycleTransaction.waitForDeletions();               // line 569
>         indexManager.invalidateAllIndexesBlocking();{code}
> After some investigation, we found that the tidier tasks generated in line 568 are actually waited by `waitForDeletions` in line 569. However, some additional tidier tasks may be asynchronously generated from `removeTruncationRecord` line 566. `SystemKeyspace#removeTruncationRecord` essentially submits some `ColumnFamilyStore$Flush` tasks, which submit `CompactionManager$BackgroundCompactionCandidate` tasks, which submit tidier tasks. We will provide the related 3 call stacks at the end of this report. Here we want to first emphasize that these tidier tasks should finish before setting the `CassandraDaemon.setupCompleted` flag. Otherwise, the potential exception may silently result in a corrupted state while the server remains “available”.
> To confirm that these tidier tasks are not waited, we keep investigating the remaining control-flow in `org.apache.cassandra.service.CassandraDaemon#setup`, after calling `LegacySchemaMigrator#migrate`. Actually we found some code about aborting the potential compactions (and tidier tasks):
> It enumerates all `ColumnFamilyStore` and calls `ColumnFamilyStore#disableAutoCompaction` in line 303 to abort their potential compaction tasks. Essentially it calls `CompactionStrategyManager#disable` to assign “false” to the `enabled` flag, so that, in line 160 in `CompactionManager#submitBackground`:
> {code:java}
>     public List<Future<?>> submitBackground(final ColumnFamilyStore cfs)
>     {
>         if (cfs.isAutoCompactionDisabled())
>         {
>             logger.trace("Autocompaction is disabled");
>             return Collections.emptyList();
>         }
>         // ...
>     } {code}
> New compaction tasks will not be submitted due to the disabled autocompaction.
> However, the potential existing compaction tasks and tidier tasks are still running, and they may generate more tidier tasks.
> Therefore, we propose that we should wait for the existing compaction tasks (in compaction executor) and tidier tasks (in nonPeriodicTasks) after disabling the autocompaction.
>  
> P.S.
> Version 3.11.10, the call stack of `CassandraDaemon#main` submitting `ColumnFamilyStore$Flush` tasks:
> {code:java}
> org.apache.cassandra.service.CassandraDaemon#main:786
> org.apache.cassandra.service.CassandraDaemon#activate:633
> org.apache.cassandra.service.CassandraDaemon#setup:261
> org.apache.cassandra.schema.LegacySchemaMigrator#migrate:83
> org.apache.cassandra.schema.LegacySchemaMigrator#unloadLegacySchemaTables:137
> java.lang.Iterable#forEach:75
> org.apache.cassandra.schema.LegacySchemaMigrator#lambda$unloadLegacySchemaTables$1:137
> org.apache.cassandra.db.ColumnFamilyStore#invalidate:542
> org.apache.cassandra.db.ColumnFamilyStore#invalidate:566
> org.apache.cassandra.db.SystemKeyspace#removeTruncationRecord:668
> org.apache.cassandra.db.SystemKeyspace#forceBlockingFlush:829
> org.apache.cassandra.db.ColumnFamilyStore#forceFlush:954
> org.apache.cassandra.db.ColumnFamilyStore#switchMemtableIfCurrent:885
> org.apache.cassandra.db.ColumnFamilyStore#switchMemtable:904
> {code}
> Version 3.11.10, the call stack of `ColumnFamilyStore$Flush#run` submitting `CompactionManager$BackgroundCompactionCandidate`:
> {code:java}
> java.lang.Thread#run:748
> org.apache.cassandra.concurrent.NamedThreadFactory#lambda$threadLocalDeallocator$0:84
> java.util.concurrent.ThreadPoolExecutor$Worker#run:624
> java.util.concurrent.ThreadPoolExecutor#runWorker:1149
> org.apache.cassandra.db.ColumnFamilyStore$Flush#run:1128
> org.apache.cassandra.db.ColumnFamilyStore$Flush#flushMemtable:1155
> org.apache.cassandra.db.ColumnFamilyStore#replaceFlushed:1642
> org.apache.cassandra.db.compaction.CompactionStrategyManager#replaceFlushed:400
> org.apache.cassandra.db.compaction.CompactionManager#submitBackground:186
> {code}
> Version 3.11.10, the call stack of `CompactionManager$BackgroundCompactionCandidate` submitting tidier tasks:
> {code:java}
> java.lang.Thread#run:748
> org.apache.cassandra.concurrent.NamedThreadFactory#lambda$threadLocalDeallocator$0:84
> java.util.concurrent.ThreadPoolExecutor$Worker#run:624
> java.util.concurrent.ThreadPoolExecutor#runWorker:1149
> java.util.concurrent.FutureTask#run:266
> java.util.concurrent.Executors$RunnableAdapter#call:511
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate#run:272
> org.apache.cassandra.db.compaction.AbstractCompactionTask#execute:61
> org.apache.cassandra.db.compaction.CompactionTask#executeInternal:85
> org.apache.cassandra.utils.WrappedRunnable#run:28
> org.apache.cassandra.db.compaction.CompactionTask#runMayThrow:223
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter#finish:123
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional#finish:179
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional#prepareToCommit:168
> org.apache.cassandra.io.sstable.SSTableRewriter#doPrepare#385
> org.apache.cassandra.db.lifecycle.LifecycleTransaction#checkpoint:327
> org.apache.cassandra.db.lifecycle.LifecycleTransaction#checkpoint:356
> org.apache.cassandra.utils.concurrent.Refs#release:241
> org.apache.cassandra.utils.concurrent.Ref#release:119
> org.apache.cassandra.utils.concurrent.Ref$State#release:225
> org.apache.cassandra.utils.concurrent.Ref$GlobalState#release:326
> org.apache.cassandra.io.sstable.format.SSTableReader$InstanceTidier#tidy:2205 {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org