[jira] [Updated] (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:all-tabpanel ] Brandon Williams updated CASSANDRA-17564: - Change Category: Operability Complexity: Normal Fix Version/s: 3.11.x 4.0.x Status: Open (was: Triage Needed) > 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 >
[jira] [Updated] (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:all-tabpanel ] Haoze Wu updated CASSANDRA-17564: - Description: 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);