cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Brandon Williams (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-6622) Streaming session failures during node replace of same address
Date Tue, 11 Feb 2014 22:48:24 GMT

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

Brandon Williams updated CASSANDRA-6622:
----------------------------------------

    Attachment: 6622-v2.txt

> 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
>             Fix For: 1.2.16, 2.0.6
>
>         Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, 6622-v2.txt,
6622_logs.tgz, 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