[ 
https://issues.apache.org/jira/browse/CASSANDRA-11010?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

T Jake Luciani updated CASSANDRA-11010:
---------------------------------------
    Assignee: Paulo Motta

> Bootstrap hangs on 3.2
> ----------------------
>
>                 Key: CASSANDRA-11010
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11010
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: T Jake Luciani
>            Assignee: Paulo Motta
>            Priority: Blocker
>             Fix For: 3.x
>
>
> Doing some testing I've noticed with 3.2 nodes can hang when bootstrapping. 
> below is the snippet from the log of a hung node.
> {quote}
> INFO  [main] 2016-01-14 02:53:25,537 StorageService.java:1281 - JOINING: 
> Starting to bootstrap...
> INFO  [main] 2016-01-14 02:53:25,564 StreamResultFuture.java:88 - [Stream 
> #fb512640-ba69-11e5-8f71-f38b96ae2156] Executing streaming plan for Bootstrap
> DEBUG [main] 2016-01-14 02:53:25,564 StreamCoordinator.java:144 - Connecting 
> next session fb512640-ba69-11e5-8f71-f38b96ae2156 with 172.31.4.27.
> INFO  [StreamConnectionEstablisher:1] 2016-01-14 02:53:25,565 
> StreamSession.java:238 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Starting streaming to /172.31.4.27
> DEBUG [StreamConnectionEstablisher:1] 2016-01-14 02:53:25,566 
> ConnectionHandler.java:82 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Sending stream init for incoming stream
> DEBUG [StreamConnectionEstablisher:1] 2016-01-14 02:53:25,568 
> ConnectionHandler.java:87 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Sending stream init for outgoing stream
> INFO  [StreamConnectionEstablisher:1] 2016-01-14 02:53:25,574 
> StreamCoordinator.java:266 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156, 
> ID#0] Beginning stream session with /172.31.4.27
> DEBUG [STREAM-OUT-/172.31.4.27] 2016-01-14 02:53:25,574 
> ConnectionHandler.java:334 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Sending Prepare (2 requests,  0 files}
> DEBUG [STREAM-IN-/172.31.4.27] 2016-01-14 02:53:25,597 
> ConnectionHandler.java:262 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Received Prepare (0 requests,  0 files}
> DEBUG [STREAM-OUT-/172.31.4.27] 2016-01-14 02:53:25,597 
> ConnectionHandler.java:334 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Sending Complete
> DEBUG [STREAM-IN-/172.31.4.27] 2016-01-14 02:53:25,597 
> ConnectionHandler.java:262 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Received Complete
> DEBUG [STREAM-IN-/172.31.4.27] 2016-01-14 02:53:25,598 
> ConnectionHandler.java:110 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Closing stream connection handler on /172.31.4.27
> INFO  [STREAM-IN-/172.31.4.27] 2016-01-14 02:53:25,598 
> StreamResultFuture.java:185 - [Stream #fb512640-ba69-11e5-8f71-f38b96ae2156] 
> Session with /172.31.4.27 is complete
> {quote}
> Vs the log of a successful bootstrap.
> {quote}
> INFO  [main] 2016-01-14 03:54:13,068 StreamResultFuture.java:88 - [Stream 
> #7963f460-ba72-11e5-b8d5-834198444b31] Executing streaming plan for Bootstrap
> INFO  [StreamConnectionEstablisher:1] 2016-01-14 03:54:13,070 
> StreamSession.java:238 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31] 
> Starting streaming to /172.31.18.61
> INFO  [StreamConnectionEstablisher:1] 2016-01-14 03:54:13,086 
> StreamCoordinator.java:266 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31, 
> ID#0] Beginning stream session with /172.31.18.61
> INFO  [STREAM-IN-/172.31.18.61] 2016-01-14 03:54:13,198 
> StreamResultFuture.java:171 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31 
> ID#0] Prepare completed. Receiving 1 files(102 bytes), sending 0 files(0 
> bytes)
> INFO  [StreamConnectionEstablisher:2] 2016-01-14 03:54:13,206 
> StreamSession.java:238 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31] 
> Starting streaming to /172.31.21.58
> INFO  [StreamConnectionEstablisher:2] 2016-01-14 03:54:13,209 
> StreamCoordinator.java:266 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31, 
> ID#0] Beginning stream session with /172.31.21.58
> INFO  [STREAM-IN-/172.31.21.58] 2016-01-14 03:54:13,255 
> StreamResultFuture.java:185 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31] 
> Session with /172.31.21.58 is complete
> INFO  [StreamReceiveTask:1] 2016-01-14 03:54:13,590 
> StreamResultFuture.java:185 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31] 
> Session with /172.31.18.61 is complete
> INFO  [StreamReceiveTask:1] 2016-01-14 03:54:13,594 
> StreamResultFuture.java:217 - [Stream #7963f460-ba72-11e5-b8d5-834198444b31] 
> All sessions completed
> INFO  [StreamReceiveTask:1] 2016-01-14 03:54:13,597 StorageService.java:1234 
> - Bootstrap completed! for the tokens [3074457345618258602]
> INFO  [main] 2016-01-14 03:54:14,424 StorageService.java:1924 - Node 
> /172.31.29.26 state jump to NORMAL
> {quote}
> The jstack of the bad bootstrap shows it waiting for the stream to complete 
> even though there are no outstanding streams.
> {quote}
> "main" #1 prio=5 os_prio=0 tid=0x00007f0c8c00afc0 nid=0x747a waiting on 
> condition [0x00007f0c95e31000]
>    java.lang.Thread.State: WAITING (parking)
>       at sun.misc.Unsafe.park(Native Method)
>       - parking to wait for  <0x00000006d8addd18> (a 
> com.google.common.util.concurrent.AbstractFuture$Sync)
>       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.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>       at 
> com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285)
>       at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116)
>       at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1345)
>       at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:933)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:707)
>       - locked <0x00000006eee44028> (a 
> org.apache.cassandra.service.StorageService)
>       at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:578)
>       - locked <0x00000006eee44028> (a 
> org.apache.cassandra.service.StorageService)
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:349)
>       at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:551)
>       at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:680)
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to