[
https://issues.apache.org/jira/browse/CASSANDRA-21022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18039030#comment-18039030
]
Jaydeepkumar Chovatia commented on CASSANDRA-21022:
---------------------------------------------------
Fixing the actual deadlock might take time, but one workaround to prevent the
deadlock is increasing _ring_delay_ms_ from default 30s to 120s.
> Deadlock during bootstrap: MigrationStage and PendingRangeCalculator block on
> keyspace initialization via LoadingMap
> --------------------------------------------------------------------------------------------------------------------
>
> Key: CASSANDRA-21022
> URL: https://issues.apache.org/jira/browse/CASSANDRA-21022
> Project: Apache Cassandra
> Issue Type: Bug
> Reporter: Jaydeepkumar Chovatia
> Priority: Normal
>
> *Overview*
> During node bootstrap, a deadlock occurs between the MigrationStage thread
> loading keyspaces and the PendingRangeCalculator thread. This causes the node
> to hang indefinitely during startup, preventing a successful bootstrap.
> This happened in a massive cluster of 600 nodes while an additional 300 nodes
> were being added simultaneously.
>
> *Deadlock Pattern*
> The deadlock involves three key components interacting through the LoadingMap
> keyspace cache:
> *Thread 1: MigrationStage:1* (holds the LoadingMap promise for a keyspace)
> Schema.merge()
> → Schema.createKeyspace()
> → Keyspace.open()
> → LoadingMap.blockingLoadIfAbsent() [line 105] - Creates promise,
> executes load function
> → Keyspace.<init>() [line 448]
> → ColumnFamilyStore.<init>() [line 521]
> → CompactionStrategyManager.<init>() [line 177]
> → getDiskBoundaries()
> → DiskBoundaryManager.getLocalRanges() [line 130]
> → PendingRangeCalculatorService.blockUntilFinished() ←
> BLOCKS waiting for Thread 2
>
> *Thread 2: PendingRangeCalculator:1* (triggered by gossip state changes)
> StorageService.handleStateNormal() [line 3499]
> → PendingRangeCalculatorService.update()
> → PendingRangeCalculatorService task [line 81]
> → Keyspace.open(keyspaceName)
> → LoadingMap.blockingLoadIfAbsent() [line 122] ← BLOCKS waiting for
> promise held by Thread 1
>
> {*}Result{*}: Classic circular wait deadlock
>
> Additional Blocked Threads
>
> Multiple other threads also block waiting for the same keyspace to load:
> - *main* thread (during StorageService.bootstrap() →
> invalidateLocalRanges())
> - *GossipStage:1* (during handleStateNormal() → updateTokenMetadata() →
> invalidateLocalRanges())
> - *ScheduledTasks:1* (memtable flush attempting getDiskBoundaries())
> - {*}IndexSummaryManager:1{*}{*}{*}
> - *jmxtrans-agent-1* (JMX metrics collection)
>
> *Root Cause*
> The issue stems from a circular dependency during bootstrap:
> 1. Keyspace initialization requires disk boundaries
> (CompactionStrategyManager initialization)
> 2. Disk boundaries require pending ranges when
> StorageService.instance.isBootstrapMode() is true
> (DiskBoundaryManager.java:127-130)
> 3. Pending ranges require PendingRangeCalculator to finish
> (blockUntilFinished())
> 4. PendingRangeCalculator tries to open keyspaces
> (PendingRangeCalculatorService.java:81)
> 5. Opening keyspaces blocks on the LoadingMap promise held by step 1
>
> From DiskBoundaryManager.java:127-130:
> if (StorageService.instance.isBootstrapMode()
> && !StorageService.isReplacingSameAddress())
> {
> PendingRangeCalculatorService.instance.blockUntilFinished(); // ←
> Blocking call during init
> localRanges = tmd.getPendingRanges(cfs.keyspace.getName(),
> FBUtilities.getBroadcastAddressAndPort());
> }
>
> Thread Dump Evidence
>
> "MigrationStage:1" - Thread t@82
> java.lang.Thread.State: WAITING
> at
> org.apache.cassandra.service.PendingRangeCalculatorService.blockUntilFinished(PendingRangeCalculatorService.java:77)
> at
> org.apache.cassandra.db.DiskBoundaryManager.getLocalRanges(DiskBoundaryManager.java:130)
> at
> org.apache.cassandra.db.DiskBoundaryManager.getDiskBoundaries(DiskBoundaryManager.java:56)
> at
> org.apache.cassandra.db.compaction.CompactionStrategyManager.<init>(CompactionStrategyManager.java:177)
> at
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:521)
> at org.apache.cassandra.db.Keyspace.<init>(Keyspace.java:448)
> at
> org.apache.cassandra.utils.concurrent.LoadingMap.blockingLoadIfAbsent(LoadingMap.java:105)
> at
> org.apache.cassandra.schema.Schema.maybeAddKeyspaceInstance(Schema.java:241)
>
> "PendingRangeCalculator:1" - Thread t@84
> java.lang.Thread.State: WAITING
> at
> org.apache.cassandra.utils.concurrent.LoadingMap.blockingLoadIfAbsent(LoadingMap.java:122)
> at
> org.apache.cassandra.schema.Schema.maybeAddKeyspaceInstance(Schema.java:241)
> at org.apache.cassandra.db.Keyspace.open(Keyspace.java:175)
> at
> org.apache.cassandra.service.PendingRangeCalculatorService.lambda$new$1(PendingRangeCalculatorService.java:81)
>
> "main" - Thread t@1
> java.lang.Thread.State: WAITING
> at
> org.apache.cassandra.utils.concurrent.LoadingMap.blockingLoadIfAbsent(LoadingMap.java:122)
> at
> org.apache.cassandra.schema.Schema.maybeAddKeyspaceInstance(Schema.java:241)
> at org.apache.cassandra.db.Keyspace.open(Keyspace.java:175)
> at
> org.apache.cassandra.service.StorageService.invalidateLocalRanges(StorageService.java:2372)
> at
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:2321)
>
> {*}Steps to Reproduce{*}{*}{*}
> The issue appears to be a race condition that occurs during bootstrap when:
> 1. Node is joining the cluster (bootstrap mode)
> 2. Schema migrations are being processed (MigrationStage active)
> 3. Gossip state changes trigger handleStateNormal() events
> 4. Multiple keyspaces are being loaded concurrently
>
> Reproduction may be more likely in environments with:
> - Many keyspaces
> - Active schema changes during bootstrap
> - NetworkTopologyStrategy with many tokens (increases pending range
> calculation time)
>
> {*}Impact{*}{*}{*}
> - Node becomes completely stuck during bootstrap
> - Requires process restart, but restart hits the same deadlock
> - Prevents cluster expansion
> - Manual intervention required (potentially removing schema changes or
> manipulating gossip state)
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]