cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Ravi Prasad (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace of same address
Date Sat, 08 Feb 2014 20:09:21 GMT

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

Ravi Prasad commented on CASSANDRA-6622:
----------------------------------------

bq. You mean rise above it, so the node is still being convicted? Can you add new logs? Maybe
now it actually is the restart event, so trying that patch with 6658 might work.

Tried with 6658 patch and 0001-don-t-signal-restart-of-dead-states.txt applied on cassandra-2.0.5
tag. Still see the same thing, where FD convicts the streaming session. I'm attaching the
logs(6622_logs.tgz).  This should be easily reproducible when replacing a dead node in a cluster
with same ip address. the issue is, the peer nodes could take 1-3 seconds to see the previously
down node (now replacing) to be up(to reset the PHI score of the down node). Since, the streaming
request arrives before this reset happens, they could be convicted leading to stream close.
So, i think  a couple of seconds sleep time for gossip to settle, before the bootstrap/streaming
starts is what is needed?

1.) node x.x.x.72 was dead
2.) node x.x.x.80's FD keeps notifying its listener to convict as PHI for .72 > threshold,
every minute.
3.) node x.x.x.72 is restarted with replace_address=x.x.x.72 at 18:56:27,806
4.) node x.x.x.72 : Gossip thread started at 18:56:33,308 after shadow gossip round
5.) node x.xx.72:  Starts stream request at 18:56:35,443
 INFO [main] 2014-02-08 18:56:35,405 StorageService.java (line 947) JOINING: Starting to bootstrap...
 INFO [main] 2014-02-08 18:56:35,443 StreamResultFuture.java (line 82) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653]
Executing streaming plan for Bootstrap
6.) node x.x.x.80 : still hasn't seen the gossip from .72 with new generation at 18:56:35,031
TRACE [GossipTasks:1] 2014-02-08 18:56:35,031 FailureDetector.java (line 229) PHI for /x.x.x.72
: 36700.042810594234
TRACE [GossipTasks:1] 2014-02-08 18:56:35,032 FailureDetector.java (line 233) notifying listeners
that /x.x.x.72 is down
7.) node x.x.x.80 : got the stream request at 18:56:35,450
 INFO [STREAM-INIT-/x.x.x.72:47408] 2014-02-08 18:56:35,450 StreamResultFuture.java (line
116) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653] Received streaming plan for Bootstrap
8.) node x.x.x.80: at 18:56:36,090, still hasn't reset the interval times for .72
TRACE [GossipTasks:1] 2014-02-08 18:56:36,090 FailureDetector.java (line 229) PHI for /x.x.x.72
: 36700.87918907657
TRACE [GossipTasks:1] 2014-02-08 18:56:36,090 FailureDetector.java (line 233) notifying listeners
that /x.x.x.72 is down
9.) node x.x.x.80:  closes the stream session due to convict() notification:
 INFO [GossipTasks:1] 2014-02-08 18:56:36,090 StreamResultFuture.java (line 181) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653]
Session with /x.x.x.72 is complete
 WARN [GossipTasks:1] 2014-02-08 18:56:36,091 StreamResultFuture.java (line 210) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653]
Stream failed
10.) node x.x.x.80:  at 18:56:36,097,  Gossiper thread on x.x.x.80 clears the interval times
for .72, thereby resetting the PHI.
DEBUG [GossipStage:1] 2014-02-08 18:56:36,097 Gossiper.java (line 790) Clearing interval times
for /x.x.x.72 due to generation change
TRACE [GossipStage:1] 2014-02-08 18:56:36,097 FailureDetector.java (line 203) reporting /x.x.x.72
11.) node x.x.x.80:  PHI score for .72 at 18:56:37,094
TRACE [GossipTasks:1] 2014-02-08 18:56:37,094 FailureDetector.java (line 229) PHI for /x.x.x.72
: 0.06483452387313912




> Streaming session failures during node replace of same address
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-6622
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: RHEL6, cassandra-2.0.4
>            Reporter: Ravi Prasad
>            Assignee: Brandon Williams
>         Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz
>
>
> When using replace_address, Gossiper ApplicationState is set to hibernate, which is a
down state. We are seeing that the peer nodes are seeing streaming plan request even before
the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes
convicts the replacing node by closing the stream handler.  
> I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL
before bootstrapping, would avoid this scenario.
> Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node
as down, 2 secs after  the streaming init request):
> {noformat}
>  INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line
116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap
> ....
>  INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream
#5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete
>  WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream
#5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed
>  INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x
is now DOWN
> ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream
#5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred
> java.lang.RuntimeException: Outgoing stream handler has been closed
>         at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175)
>         at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436)
>         at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358)
>         at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293)
>         at java.lang.Thread.run(Thread.java:722)
>  INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181)
[Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete
>  WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210)
[Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Mime
View raw message