[jira] [Commented] (CASSANDRA-15138) A cluster (RF=3) not recovering after two nodes are stopped

2020-06-06 Thread Yuji Ito (Jira)


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

Yuji Ito commented on CASSANDRA-15138:
--

Here are logs about this issue.
h3. Node1 (the living node):
{code:java}
TRACE [GossipStage:1] 2020-06-05 06:08:42,866 Gossiper.java:1101 - marking as 
down /10.42.2.90
INFO  [GossipStage:1] 2020-06-05 06:08:42,866 Gossiper.java:1105 - InetAddress 
/10.42.2.90 is now DOWN
TRACE [GossipStage:1] 2020-06-05 06:08:42,866 MessagingService.java:658 - 
Resetting pool for /10.42.2.90
...
DEBUG [GossipStage:1] 2020-06-05 06:08:42,868 FailureDetector.java:313 - 
Forcing conviction of /10.42.2.90
...
DEBUG [GossipStage:1] 2020-06-05 06:08:42,866 OutboundTcpConnection.java:205 - 
Enqueuing socket close for /10.42.2.90
DEBUG [GossipStage:1] 2020-06-05 06:08:42,866 OutboundTcpConnection.java:205 - 
Enqueuing socket close for /10.42.2.90
DEBUG [MessagingService-Outgoing-/10.42.2.90-Small] 2020-06-05 06:08:42,866 
OutboundTcpConnection.java:411 - Socket to /
10.42.2.90 closed
DEBUG [GossipStage:1] 2020-06-05 06:08:42,866 OutboundTcpConnection.java:205 - 
Enqueuing socket close for /10.42.2.90
TRACE [GossipStage:1] 2020-06-05 06:08:42,867 Server.java:630 - Status changed 
event : /10.42.2.90, DOWN
...
DEBUG [MessagingService-Outgoing-/10.42.2.90-Gossip] 2020-06-05 06:08:42,868 
OutboundTcpConnection.java:411 - Socket to /10.42.2.90 closed
...
DEBUG [MessagingService-Outgoing-/10.42.2.90-Small] 2020-06-05 06:08:42,869 
OutboundTcpConnection.java:425 - Attempting to connect to /10.42.2.90
DEBUG [MessagingService-Outgoing-/10.42.2.90-Small] 2020-06-05 06:08:42,873 
OutboundTcpConnection.java:533 - Done connecting to /10.42.2.90

DEBUG [MessagingService-Outgoing-/10.42.2.90-Gossip] 2020-06-05 06:08:48,732 
OutboundTcpConnection.java:411 - Socket to /10.42.2.90 closed
...
TRACE [GossipStage:1] 2020-06-05 06:27:51,065 OutboundTcpConnection.java:628 - 
Expiration of MessagingService-Outgoing-/10.42.2.90-Small took 234?s
...
DEBUG [MessagingService-Outgoing-/10.42.2.90-Small] 2020-06-05 06:28:09,750 
OutboundTcpConnection.java:411 - Socket to /10.42.2.90 closed
DEBUG [MessagingService-Outgoing-/10.42.2.90-Small] 2020-06-05 06:28:09,750 
OutboundTcpConnection.java:350 - Error writing to /10.42.2.90
java.io.IOException: Connection timed out
at sun.nio.ch.FileDispatcherImpl.write0(Native Method) ~[na:1.8.0_252]
at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) 
~[na:1.8.0_252]
at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) 
~[na:1.8.0_252]
at sun.nio.ch.IOUtil.write(IOUtil.java:51) ~[na:1.8.0_252]
at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:468) 
~[na:1.8.0_252]
at 
org.apache.cassandra.io.util.BufferedDataOutputStreamPlus.doFlush(BufferedDataOutputStreamPlus.java:323)
 ~[main/:na]
at 
org.apache.cassandra.io.util.BufferedDataOutputStreamPlus.flush(BufferedDataOutputStreamPlus.java:331)
 ~[main/:na]
at 
org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:342)
 [main/:na]
at 
org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:263)
 [main/:na]{code}
h3. Node2 (the second shutdown node)
{code:java}
DEBUG [StorageServiceShutdownHook] 2020-06-05 06:08:42,840 
StorageService.java:1480 - DRAINING: starting drain process
...
INFO  [StorageServiceShutdownHook] 2020-06-05 06:08:42,850 Gossiper.java:1647 - 
Announcing shutdown
...
TRACE [StorageServiceShutdownHook] 2020-06-05 06:08:42,858 
MessagingService.java:942 - 
ip-10-42-2-90.ap-northeast-1.compute.internal/10.42.2.90 sending 
GOSSIP_SHUTDOWN to 3506286@/10.42.2.16
...
TRACE [StorageServiceShutdownHook] 2020-06-05 06:08:42,858 
MessagingService.java:942 - 
ip-10-42-2-90.ap-northeast-1.compute.internal/10.42.2.90 sending 
GOSSIP_SHUTDOWN to 3506286@/10.42.2.16
...
TRACE [MessagingService-Incoming-/10.42.2.16] 2020-06-05 06:08:42,868 
IncomingTcpConnection.java:120 - Closing socket 
Socket[addr=/10.42.2.16,port=52646,localport=7000] - isclosed: false
TRACE [ACCEPT-ip-10-42-2-90.ap-northeast-1.compute.internal/10.42.2.90] 
2020-06-05 06:08:42,872 MessagingService.java:1315 - Connection version 11 from 
/10.42.2.16
TRACE [MessagingService-Incoming-/10.42.2.16] 2020-06-05 06:08:42,873 
MessagingService.java:1087 - Setting version 11 for /10.42.2.16
TRACE [MessagingService-Incoming-/10.42.2.16] 2020-06-05 06:08:42,873 
IncomingTcpConnection.java:155 - Set version for /10.42.2.16 to 11 (will use 11)
...
TRACE [StorageServiceShutdownHook] 2020-06-05 06:08:44,859 
MessagingService.java:1351 - Closing accept() thread
TRACE [StorageServiceShutdownHook] 2020-06-05 06:08:44,859 
IncomingTcpConnection.java:120 - Closing socket 
Socket[addr=/10.42.2.16,port=53386,localport=7000] - isclosed: false
TRACE [StorageServiceShutdownHook] 2020-06-05 06:08:44,859 
IncomingTcpConnection.java:120 

[jira] [Commented] (CASSANDRA-15138) A cluster (RF=3) not recovering after two nodes are stopped

2020-06-06 Thread Yuji Ito (Jira)


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

Yuji Ito commented on CASSANDRA-15138:
--

Hi,

I'm investigating this issue on 3.11.6.

When this issue happens, Node1 (the living node) can see that Node2 is alive, 
but Node2 (the second shutdown node) cannot see Node1. It seems that gossip 
isn't working well.

~{{[node1]$ nodetool status}}~
~{{Datacenter: datacenter1}}~
~{{===}}~
~{{Status=Up/Down}}~
~{{|/ State=Normal/Leaving/Joining/Moving}}~
~{{-- Address Load Tokens Owns (effective) Host ID Rack}}~
~{{UN 10.42.2.90 241.88 MiB 256 100.0% d4d9d194-1c10-4ccf-bcc8-62b596dc5fb0 
rack1}}~
~{{DN 10.42.2.190 241.98 MiB 256 100.0% 737f5158-1e06-405a-8f09-c9d2b308053f 
rack1}}~
~{{UN 10.42.2.16 326.41 KiB 256 100.0% 35d643fd-2a46-4aaa-b6fa-922a39e23d94 
rack1}}~

 

~{{[node2]$ nodetool status}}~
~{{Datacenter: datacenter1}}~
~{{===}}~
~{{Status=Up/Down}}~
~{{|/ State=Normal/Leaving/Joining/Moving}}~
~{{-- Address Load Tokens Owns (effective) Host ID Rack}}~
~{{UN 10.42.2.90 241.88 MiB 256 100.0% d4d9d194-1c10-4ccf-bcc8-62b596dc5fb0 
rack1}}~
~{{DN 10.42.2.190 241.98 MiB 256 100.0% 737f5158-1e06-405a-8f09-c9d2b308053f 
rack1}}~
~{{DN 10.42.2.16 326.41 KiB 256 100.0% 35d643fd-2a46-4aaa-b6fa-922a39e23d94 
rack1}}~

 

I think a connection issue might happen between sending GOSSIP_SHUTDOWN and 
MessageService shutdown in the shutdown process, {{StorageService#drain()}}.
 # Node2 (the second shutdown node) sends GOSSIP_SHUTDOWN to other nodes
 # Node1 (the living node) receives GOSSIP_SHUTDOWN
 # Node1 closes all OutboundConnections to Node2
 # Node1 tries to send other messages like MUTATION because a client keeps 
requesting operations
 # *Node1 opens a new connection of OutboundConnection-small* to Node2 again if 
Node2 hasn't closed the IncomingTcpConnection yet
 ** It's a timing issue. When Node2 has already closed the connection, this 
issue doesn't happen.
 # Node2 closes all connection for the shutdown *even though Node1 still leaves 
opening OutboundConnection-small to Node2*
 # When Node2 restarts, Node1 and Node2 try their gossip by opening a new 
gossip connection
 ** GOSSIP_DIGEST_SYN, GOSSIP_DIGEST_ACK, GOSSIP_DIGEST_ACK2 are handled 
successfully because they are sent by the new gossip connection which is opened 
after Node2 has restarted
 # Node1 tries to send the REQUEST_RESPONSE message as an EchoMessage reply by 
using the OutboundConnection-small which has been left, but the connection is 
out of order
 ** Node2 has sent the EchoMessage in `markAlive()` for Node1's ACK2
 # After the OutboundConnection-small's stream times out, the connection is 
closed and a new connection is established, Node2 will succeed the gossip for 
Node1

 

I'm not sure that this issue happens on 4.0. It might happen because the 
shutdown process looks the same as 3.11.x.

> A cluster (RF=3) not recovering after two nodes are stopped
> ---
>
> Key: CASSANDRA-15138
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15138
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Membership
>Reporter: Hiroyuki Yamada
>Priority: Normal
>
> I faced a weird issue when recovering a cluster after two nodes are stopped.
>  It is easily reproduce-able and looks like a bug or an issue to fix.
>  The following is a step to reproduce it.
> === STEP TO REPRODUCE ===
>  * Create a 3-node cluster with RF=3
>     - node1(seed), node2, node3
>  * Start requests to the cluster with cassandra-stress (it continues
>  until the end)
>     - what we did: cassandra-stress mixed cl=QUORUM duration=10m
>  -errors ignore -node node1,node2,node3 -rate threads\>=16
>  threads\<=256
>  - (It doesn't have to be this many threads. Can be 1)
>  * Stop node3 normally (with systemctl stop or kill (without -9))
>     - the system is still available as expected because the quorum of nodes is
>  still available
>  * Stop node2 normally (with systemctl stop or kill (without -9))
>     - the system is NOT available as expected after it's stopped.
>     - the client gets `UnavailableException: Not enough replicas
>  available for query at consistency QUORUM`
>     - the client gets errors right away (so few ms)
>     - so far it's all expected
>  * Wait for 1 mins
>  * Bring up node2 back
>     - {color:#ff}The issue happens here.{color}
>     - the client gets ReadTimeoutException` or WriteTimeoutException
>  depending on if the request is read or write even after the node2 is
>  up
>     - the client gets errors after about 5000ms or 2000ms, which are
>  request timeout for write and read request
>     - what node1 reports with `nodetool status` and what node2 reports
>  are not consistent. (node2 thinks node1 is 

[jira] [Commented] (CASSANDRA-15138) A cluster (RF=3) not recovering after two nodes are stopped

2019-05-23 Thread Hiroyuki Yamada (JIRA)


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

Hiroyuki Yamada commented on CASSANDRA-15138:
-

[~jmeredithco] Yes, that is correct. Sorry for not stating it.

> A cluster (RF=3) not recovering after two nodes are stopped
> ---
>
> Key: CASSANDRA-15138
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15138
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Membership
>Reporter: Hiroyuki Yamada
>Priority: Normal
>
> I faced a weird issue when recovering a cluster after two nodes are stopped.
>  It is easily reproduce-able and looks like a bug or an issue to fix.
>  The following is a step to reproduce it.
> === STEP TO REPRODUCE ===
>  * Create a 3-node cluster with RF=3
>     - node1(seed), node2, node3
>  * Start requests to the cluster with cassandra-stress (it continues
>  until the end)
>     - what we did: cassandra-stress mixed cl=QUORUM duration=10m
>  -errors ignore -node node1,node2,node3 -rate threads\>=16
>  threads\<=256
>  - (It doesn't have to be this many threads. Can be 1)
>  * Stop node3 normally (with systemctl stop or kill (without -9))
>     - the system is still available as expected because the quorum of nodes is
>  still available
>  * Stop node2 normally (with systemctl stop or kill (without -9))
>     - the system is NOT available as expected after it's stopped.
>     - the client gets `UnavailableException: Not enough replicas
>  available for query at consistency QUORUM`
>     - the client gets errors right away (so few ms)
>     - so far it's all expected
>  * Wait for 1 mins
>  * Bring up node2 back
>     - {color:#ff}The issue happens here.{color}
>     - the client gets ReadTimeoutException` or WriteTimeoutException
>  depending on if the request is read or write even after the node2 is
>  up
>     - the client gets errors after about 5000ms or 2000ms, which are
>  request timeout for write and read request
>     - what node1 reports with `nodetool status` and what node2 reports
>  are not consistent. (node2 thinks node1 is down)
>     - It takes very long time to recover from its state
> === STEPS TO REPRODUCE ===
> Some additional important information to note:
>  * If we don't start cassandra-stress, it doesn't cause the issue.
>  * Restarting node1 and it recovers its state right after it's restarted
>  * Setting lower value in dynamic_snitch_reset_interval_in_ms (to 6
>  or something) fixes the issue
>  * If we `kill -9` the nodes, then it doesn't cause the issue.
>  * Hints seems not related. I tested with hints disabled, it didn't make any 
> difference.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15138) A cluster (RF=3) not recovering after two nodes are stopped

2019-05-23 Thread Jon Meredith (JIRA)


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

Jon Meredith commented on CASSANDRA-15138:
--

Just spotted your email to the user mailing list - looks like C* 3.11.4

> A cluster (RF=3) not recovering after two nodes are stopped
> ---
>
> Key: CASSANDRA-15138
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15138
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Membership
>Reporter: Hiroyuki Yamada
>Priority: Normal
>
> I faced a weird issue when recovering a cluster after two nodes are stopped.
>  It is easily reproduce-able and looks like a bug or an issue to fix.
>  The following is a step to reproduce it.
> === STEP TO REPRODUCE ===
>  * Create a 3-node cluster with RF=3
>     - node1(seed), node2, node3
>  * Start requests to the cluster with cassandra-stress (it continues
>  until the end)
>     - what we did: cassandra-stress mixed cl=QUORUM duration=10m
>  -errors ignore -node node1,node2,node3 -rate threads\>=16
>  threads\<=256
>  - (It doesn't have to be this many threads. Can be 1)
>  * Stop node3 normally (with systemctl stop or kill (without -9))
>     - the system is still available as expected because the quorum of nodes is
>  still available
>  * Stop node2 normally (with systemctl stop or kill (without -9))
>     - the system is NOT available as expected after it's stopped.
>     - the client gets `UnavailableException: Not enough replicas
>  available for query at consistency QUORUM`
>     - the client gets errors right away (so few ms)
>     - so far it's all expected
>  * Wait for 1 mins
>  * Bring up node2 back
>     - {color:#ff}The issue happens here.{color}
>     - the client gets ReadTimeoutException` or WriteTimeoutException
>  depending on if the request is read or write even after the node2 is
>  up
>     - the client gets errors after about 5000ms or 2000ms, which are
>  request timeout for write and read request
>     - what node1 reports with `nodetool status` and what node2 reports
>  are not consistent. (node2 thinks node1 is down)
>     - It takes very long time to recover from its state
> === STEPS TO REPRODUCE ===
> Some additional important information to note:
>  * If we don't start cassandra-stress, it doesn't cause the issue.
>  * Restarting node1 and it recovers its state right after it's restarted
>  * Setting lower value in dynamic_snitch_reset_interval_in_ms (to 6
>  or something) fixes the issue
>  * If we `kill -9` the nodes, then it doesn't cause the issue.
>  * Hints seems not related. I tested with hints disabled, it didn't make any 
> difference.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-15138) A cluster (RF=3) not recovering after two nodes are stopped

2019-05-23 Thread Jon Meredith (JIRA)


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

Jon Meredith commented on CASSANDRA-15138:
--

Thanks for the detailed steps in the report. Which versions of Cassandra have 
you reproduced the issue with?

> A cluster (RF=3) not recovering after two nodes are stopped
> ---
>
> Key: CASSANDRA-15138
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15138
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Membership
>Reporter: Hiroyuki Yamada
>Priority: Normal
>
> I faced a weird issue when recovering a cluster after two nodes are stopped.
>  It is easily reproduce-able and looks like a bug or an issue to fix.
>  The following is a step to reproduce it.
> === STEP TO REPRODUCE ===
>  * Create a 3-node cluster with RF=3
>     - node1(seed), node2, node3
>  * Start requests to the cluster with cassandra-stress (it continues
>  until the end)
>     - what we did: cassandra-stress mixed cl=QUORUM duration=10m
>  -errors ignore -node node1,node2,node3 -rate threads\>=16
>  threads\<=256
>  - (It doesn't have to be this many threads. Can be 1)
>  * Stop node3 normally (with systemctl stop or kill (without -9))
>     - the system is still available as expected because the quorum of nodes is
>  still available
>  * Stop node2 normally (with systemctl stop or kill (without -9))
>     - the system is NOT available as expected after it's stopped.
>     - the client gets `UnavailableException: Not enough replicas
>  available for query at consistency QUORUM`
>     - the client gets errors right away (so few ms)
>     - so far it's all expected
>  * Wait for 1 mins
>  * Bring up node2 back
>     - {color:#ff}The issue happens here.{color}
>     - the client gets ReadTimeoutException` or WriteTimeoutException
>  depending on if the request is read or write even after the node2 is
>  up
>     - the client gets errors after about 5000ms or 2000ms, which are
>  request timeout for write and read request
>     - what node1 reports with `nodetool status` and what node2 reports
>  are not consistent. (node2 thinks node1 is down)
>     - It takes very long time to recover from its state
> === STEPS TO REPRODUCE ===
> Some additional important information to note:
>  * If we don't start cassandra-stress, it doesn't cause the issue.
>  * Restarting node1 and it recovers its state right after it's restarted
>  * Setting lower value in dynamic_snitch_reset_interval_in_ms (to 6
>  or something) fixes the issue
>  * If we `kill -9` the nodes, then it doesn't cause the issue.
>  * Hints seems not related. I tested with hints disabled, it didn't make any 
> difference.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org