[
https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13894191#comment-13894191
]
Ravi Prasad edited comment on CASSANDRA-6622 at 2/7/14 5:23 AM:
bq. Can you try the patch from CASSANDRA-6658?
Didn't help. What i'm seeing is, the other nodes in the ring take around 2-3
seconds for PHI on the replacing node to drop below convict threshold. But,
they also receive the stream plan from the replacing node with in 2 seconds of
starting of replacing node.
was (Author: ravilr):
bq. Can you try the patch from CASSANDRA-6658?
Didn't help. What i'm seeing is, the other nodes in the ring take around 2-3
seconds for PHI on the replacing node to drop below convict threshold. But,
they also receive the stream plan from the replacing node with in 2 seconds of
starting of replacing node. I think this would affect normal bootstrap also,
but there we sleep for RING_DELAY already.
Streaming session failures during node replace using replace_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)