[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15677239#comment-15677239 ] Aleksey Yeschenko commented on CASSANDRA-12281: --- Committed to 2.2 as [9cd7d540de2ea525982d139a4c8a11233c4e98c9|https://github.com/apache/cassandra/commit/9cd7d540de2ea525982d139a4c8a11233c4e98c9] and merged upwards in accordance with Joel's instructions. Thanks. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: 12281-2.2.patch, 12281-3.0.patch, 12281-3.X.patch, > 12281-trunk.patch, restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Go
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15669481#comment-15669481 ] Joel Knighton commented on CASSANDRA-12281: --- Thanks - your changes and CI look good. I also ran CI on your CASSANDRA-12281-trunk branch. Note to committer: there are very slight differences in the 2.2/3.0/3.x branches (not in substantial comment, but in comments and other minor fixes). The 3.x branch should merge cleanly into trunk, I believe. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: 12281-2.2.patch, 12281-3.0.patch, 12281-trunk.patch, > restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15646865#comment-15646865 ] Stefan Podkowinski commented on CASSANDRA-12281: There we go.. ||2.2||3.0||3.X|| |[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-12281-2.2]|[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-12281-3.0]|[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-12281-3.X]| |[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-2.2-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-3.0-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-3.X-dtest/]| |[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-2.2-testall/]|[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-3.0-testall/]|[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-3.X-testall/]| > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: 12281-2.2.patch, 12281-3.0.patch, 12281-trunk.patch, > restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15644323#comment-15644323 ] Joel Knighton commented on CASSANDRA-12281: --- Ah, good catch on the aggregate log message spanning the trace and debug cases. That makes a lot of sense - thanks for the explanation. I'll keep this at the top of my queue for when CI is available. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: 12281-2.2.patch, 12281-3.0.patch, 12281-trunk.patch, > restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15643772#comment-15643772 ] Stefan Podkowinski commented on CASSANDRA-12281: bq. Byteman is available for tests on the 2.2 branch since CASSANDRA-12377 Nice, I wasn't aware about that. I've now included the test for the 2.2 patch as well. bq. In the PendingRangeCalculatorService, I'm not sure we need to move the "Finished calculation for ..." log message to trace. Most Gossip/TokenMetadata state changes are logged at debug, especially when they reflect some detail about the aggregate state of an operation. I don't think the message itself is very useful, as it will log the execution time for both cases, when an actual calculation took place and on early exit by {{calculatePendingRanges}} due to a lack of relevant state changes. As the last case is only covered with a trace message in {{calculatePendingRanges}}, I don't think we should use debug for the message in {{PendingRangeTask}} either, as it will only will tell you how long the task took to execute, compared to the debug message in {{calculatePendingRanges}}, which will tell you how long an actual calculation took. Otherwise I've addressed the mentioned points of yours and will restart tests once cassci is available again. I'll follow up with the results. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: 12281-2.2.patch, 12281-3.0.patch, 12281-trunk.patch, > restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15637528#comment-15637528 ] Joel Knighton commented on CASSANDRA-12281: --- Thanks for the patch and your patience as I get to this for review! I've been quite busy lately. The approach overall seems sound. While calculating pending ranges can be a little slow, I don't think we risk falling too far behind, because the huge delays here appear to be a result of cascading delays to other tasks. The PendingRangeCalculatorService's restriction on one queued task that will reflect cluster state at time of execution helps with this. A few small questions/nits: - Is there a reason that the test is excluded from the 2.2 branch? Byteman is available for tests on the 2.2 branch since [CASSANDRA-12377], and I don't see anything else that stops the test from being useful there. - Generally, the tests are organized as a top-level class for some entity or fundamental operation in the codebase and then specific test methods for unit tests/regression tests. I think it would make sense to establish a {{PendingRangeCalculatorServiceTest}} and introduce the specific test for [CASSANDRA-12281] inside that class. - In the {{PendingRangeCalculatorService}}, I'm not sure we need to move the "Finished calculation for ..." log message to trace. Most Gossip/TokenMetadata state changes are logged at debug, especially when they reflect some detail about the aggregate state of an operation. - A few minor spelling fixes in the test "aquire" -> "acquire", "fist" -> "first". (Note that I normally wouldn't bother with these, but since the test could likely use a few other changes, I think it is worthwhile to fix these.) - In the test's setUp, the call to {{Keyspace.setInitialized}} is redundant. The call to {{SchemaLoader.prepareServer}} will already perform this. - CI looks good overall. The 3.0-dtest run has a few materialized view dtest failures that are likely unrelated, but it would be good if you could retrigger CI for at least this branch. - There's no CI/branch posted for the 3.X series. While this has barely diverged from trunk at this point, it'd be nice if you could run CI for this branch. Thanks again. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: 12281-2.2.patch, 12281-3.0.patch, 12281-trunk.patch, > restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15628949#comment-15628949 ] Stefan Podkowinski commented on CASSANDRA-12281: Ok, I'm going to pull out the code for running the calculation just once per KS setting and open a new ticket for that, as this doesn't seem to be the real problem at hand here. The attached patches will now just make sure that we don't block on incoming gossip messages during pending range calculation. ||2.2||3.0||trunk|| |[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-12281-2.2]|[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-12281-3.0]|[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-12281-trunk]| |[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-2.2-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-3.0-dtest/]|[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-trunk-dtest/]| |[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-2.2-testall/]|[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-3.0-testall/]|[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-12281-trunk-testall/]| > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15622189#comment-15622189 ] Andy Peckys commented on CASSANDRA-12281: - It's the same for me, only 1 keyspace > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:1165) > at > org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1128) > at > org.apache.cassandra.gms.GossipDigestAckVerbHandler.doVerb(Gossip
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15613900#comment-15613900 ] Dikang Gu commented on CASSANDRA-12281: --- [~spo...@gmail.com], in my case, we have large number of nodes in the cluster, but there is only one keyspace. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:1165) > at > org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1128) > a
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15601668#comment-15601668 ] Stefan Podkowinski commented on CASSANDRA-12281: Would you be ok with my approach in the [WIP branch|https://github.com/apache/cassandra/compare/cassandra-3.0...spodkowinski:WIP-12281], [~jkni]? I've implemented two significant changes here: # TokenMetaData.calculatePendingRanges() will no longer acquire a read lock for range calculation. This should be fine as long as this method is the only writer and synchronizes around the ranges map. Without the lock, it's now possible that range calculation results will be behind the most recent gossip state, but I think that's still preferred to avoiding updating the gossip state at all by blocked handlers. Eventually the calculation should catch up. # The ranges calculation will now only be done once per identical replication configuration, which should improve performance for clusters with a large number of keyspaces. It would be nice if [~apeckys] or [~dikanggu] could confirm if their affected clusters have a larger number of keyspaces or not, so we get a better idea if this could be a bigger factor at play. Let me know your comments and I'm going to create patches for 2.2 upwards and fire the tests in case we're good. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Stefan Podkowinski > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheck
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15591740#comment-15591740 ] Joel Knighton commented on CASSANDRA-12281: --- [~spo...@gmail.com] - that would be great. I certainly didn't intend to take the assignee role to block progress on it. Other work interrupted for longer than anticipated. Thanks and let me know if I can help! > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Joel Knighton > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.jav
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15591596#comment-15591596 ] Stefan Podkowinski commented on CASSANDRA-12281: I've created a unit test to reproduce this issue, see linked WIP branch. [~jkni], do you mind taking the reviewer role while I continue trying to come up with a patch? > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Joel Knighton >Priority: Minor > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:1
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15589212#comment-15589212 ] Andy Peckys commented on CASSANDRA-12281: - I just experienced that, had the PendingRangeCalculator block gossip and it backed up gossip on all nodes in the cluster. Out of 300+ nodes i had to stop and start gossip on over a 3rd of nodes to get them back into sync. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Joel Knighton >Priority: Minor > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15588617#comment-15588617 ] Stefan Podkowinski commented on CASSANDRA-12281: I just took a closer look at the source and stack dump and it also looks to me as if the pending ranges calculation is blocking all gossip tasks from executing. But this should not only affect a starting node, as the pending ranges calculation should take place on every node in the cluster after relevant changes in the cluster topology. It should also always take about the same amount of time before completion. You should be able to verify this by increasing the log level of {{org.apache.cassandra.service.PendingRangeCalculatorService}} and look for a "finished calculation for" log message with the amount of time it took to finish the range calculation. The number of pending GossipStage tasks should also increase on each node after the new node starts bootstrapping. As can be seen in the stack dump, the startup process is stuck in {{CassandraDaemon.waitForGossipToSettle()}} where the number of pending tasks are checked in assumption that we'll be able to continue starting up once all of them have been completed. You should be able to find [related INFO messages|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/service/CassandraDaemon.java#L588] in the log. Is this really being the case? Can a bootstrapping node trigger a very slow pending ranges calculation on all nodes that would effectively shut down all gossip in the cluster? > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Joel Knighton >Priority: Minor > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Me
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15587266#comment-15587266 ] Andy Peckys commented on CASSANDRA-12281: - I have 300+ node cluster in AWS that is also seeing this issue > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Joel Knighton >Priority: Minor > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:1165) > at > org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1128) > at > org.apache.cassa
[jira] [Commented] (CASSANDRA-12281) Gossip blocks on startup when another node is bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-12281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15531831#comment-15531831 ] Dikang Gu commented on CASSANDRA-12281: --- How big is your cluster? I see the same problem in one of our big cluster as well. > Gossip blocks on startup when another node is bootstrapping > --- > > Key: CASSANDRA-12281 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12281 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Eric Evans >Assignee: Joel Knighton >Priority: Minor > Attachments: restbase1015-a_jstack.txt > > > In our cluster, normal node startup times (after a drain on shutdown) are > less than 1 minute. However, when another node in the cluster is > bootstrapping, the same node startup takes nearly 30 minutes to complete, the > apparent result of gossip blocking on pending range calculations. > {noformat} > $ nodetool-a tpstats > Pool NameActive Pending Completed Blocked All > time blocked > MutationStage 0 0 1840 0 > 0 > ReadStage 0 0 2350 0 > 0 > RequestResponseStage 0 0 53 0 > 0 > ReadRepairStage 0 0 1 0 > 0 > CounterMutationStage 0 0 0 0 > 0 > HintedHandoff 0 0 44 0 > 0 > MiscStage 0 0 0 0 > 0 > CompactionExecutor3 3395 0 > 0 > MemtableReclaimMemory 0 0 30 0 > 0 > PendingRangeCalculator1 2 29 0 > 0 > GossipStage 1 5602164 0 > 0 > MigrationStage0 0 0 0 > 0 > MemtablePostFlush 0 0111 0 > 0 > ValidationExecutor0 0 0 0 > 0 > Sampler 0 0 0 0 > 0 > MemtableFlushWriter 0 0 30 0 > 0 > InternalResponseStage 0 0 0 0 > 0 > AntiEntropyStage 0 0 0 0 > 0 > CacheCleanupExecutor 0 0 0 0 > 0 > Message type Dropped > READ 0 > RANGE_SLICE 0 > _TRACE 0 > MUTATION 0 > COUNTER_MUTATION 0 > REQUEST_RESPONSE 0 > PAGED_RANGE 0 > READ_REPAIR 0 > {noformat} > A full thread dump is attached, but the relevant bit seems to be here: > {noformat} > [ ... ] > "GossipStage:1" #1801 daemon prio=5 os_prio=0 tid=0x7fe4cd54b000 > nid=0xea9 waiting on condition [0x7fddcf883000] >java.lang.Thread.State: WAITING (parking) > at sun.misc.Unsafe.park(Native Method) > - parking to wait for <0x0004c1e922c0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$FairSync) > at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199) > at > java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock(ReentrantReadWriteLock.java:943) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:174) > at > org.apache.cassandra.locator.TokenMetadata.updateNormalTokens(TokenMetadata.java:160) > at > org.apache.cassandra.service.StorageService.handleStateNormal(StorageService.java:2023) > at > org.apache.cassandra.service.StorageService.onChange(StorageService.java:1682) > at > org.apache.cassandra.gms.Gossiper.doOnChangeNotifications(Gossiper.java:1182) > at org.apache.cassandra.gms.Gossiper.applyNewStates(Gossiper.java:1165) > at > org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:1128) > at >