[jira] [Commented] (CASSANDRA-7696) Fail to reconnect to other nodes after intermittent network failure (SSL + auth enabled)
[ https://issues.apache.org/jira/browse/CASSANDRA-7696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14086659#comment-14086659 ] Brandon Williams commented on CASSANDRA-7696: - I suspect this may somehow be related to CASSANDRA-6590 Fail to reconnect to other nodes after intermittent network failure (SSL + auth enabled) Key: CASSANDRA-7696 URL: https://issues.apache.org/jira/browse/CASSANDRA-7696 Project: Cassandra Issue Type: Bug Components: Core Environment: Cassandra 1.2.16, CentOS 6.5, Cluster spread to 3 datacenters with 1 node each. SSL enabled between Cassandra nodes and also to clients, authentization enabled. Reporter: Jiri Horky We experienced a strange problem after intermittent network failure when the affected node did not reconnect to the rest of the cluster but did allow to autenticate users (which was not possible during the actual network outage, see below). The cluster consists of 1 node in each of 3 datacenters, it uses C* 1.2.16 with SSL enabled both to clients and between C* nodes. The authentication is enabled as well. The problem started around 2014-08-01 when Cassandra first noticed a network problem: INFO [GossipTasks:1] 2014-08-01 07:47:52,618 Gossiper.java (line 823) InetAddress /A.B.D.20 is now DOWN INFO [GossipTasks:1] 2014-08-01 07:47:55,619 Gossiper.java (line 823) InetAddress mia10/A.B.C.20 is now DOWN The network came up for a while: INFO [GossipStage:1] 2014-08-01 07:51:29,380 Gossiper.java (line 809) InetAddress /A.B.C.20 is now UP INFO [HintedHandoff:1] 2014-08-01 07:51:29,381 HintedHandOffManager.java (line 296) Started hinted handoff for host: 9252f37c-1c9a-418b-a49f-6065511946e4 with IP: /A.B.C.20 INFO [GossipStage:1] 2014-08-01 07:51:29,381 Gossiper.java (line 809) InetAddress /A.B.D.20 is now UP INFO [HintedHandoff:2] 2014-08-01 07:51:29,385 HintedHandOffManager.java (line 296) Started hinted handoff for host: 97b1943a-3689-4e4a-a39d-d5a11c0cc309 with IP: /A.B.D.20 But it failed to send hints: INFO [HintedHandoff:1] 2014-08-01 07:51:39,389 HintedHandOffManager.java (line 427) Timed out replaying hints to /A.B.C.20; aborting (0 delivered) INFO [HintedHandoff:2] 2014-08-01 07:51:39,390 HintedHandOffManager.java (line 427) Timed out replaying hints to /A.B.D.20; aborting (0 delivered) Also, the log started to be flooded with failed autentication tries. My understanding is that authentication data are read with QUORUM which failed as the other two nodes were down: ERROR [Native-Transport-Requests:446116] 2014-08-01 07:51:39,985 QueryMessage.java (line 97) Unexpected error during query com.google.common.util.concurrent.UncheckedExecutionException: java.lang.RuntimeException: org.apache.cassandra.exceptions.ReadTimeoutException: Operation timed out - received only 0 responses. at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2258) at com.google.common.cache.LocalCache.get(LocalCache.java:3990) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3994) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4878) at org.apache.cassandra.service.ClientState.authorize(ClientState.java:292) at org.apache.cassandra.service.ClientState.ensureHasPermission(ClientState.java:172) at org.apache.cassandra.service.ClientState.hasAccess(ClientState.java:165) at org.apache.cassandra.service.ClientState.hasColumnFamilyAccess(ClientState.java:149) at org.apache.cassandra.cql3.statements.SelectStatement.checkAccess(SelectStatement.java:116) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:102) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:113) at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:87) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) at org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43) at org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at
[jira] [Commented] (CASSANDRA-7696) Fail to reconnect to other nodes after intermittent network failure (SSL + auth enabled)
[ https://issues.apache.org/jira/browse/CASSANDRA-7696?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14086703#comment-14086703 ] Jeremy Jongsma commented on CASSANDRA-7696: --- I saw something similar over the weekend, also running 1.2.16 with internode encryption and authentication in two datacenters. Other observations: 1) In DC1, all nodes reported that three nodes in DC2 were UNREACHABLE in nodetool describecluster 2) In DC2, all nodes reported that all other nodes were up and reachable, including the problem nodes in DC1. 3) DataStax OpsCenter (running in DC2) reported no connectivity issues, all nodes online. 4) I verified network connectivity was fine between all nodes. 5) Restarting Cassandra on DC1 nodes immediately restored connectivity. Fail to reconnect to other nodes after intermittent network failure (SSL + auth enabled) Key: CASSANDRA-7696 URL: https://issues.apache.org/jira/browse/CASSANDRA-7696 Project: Cassandra Issue Type: Bug Components: Core Environment: Cassandra 1.2.16, CentOS 6.5, Cluster spread to 3 datacenters with 1 node each. SSL enabled between Cassandra nodes and also to clients, authentization enabled. Reporter: Jiri Horky We experienced a strange problem after intermittent network failure when the affected node did not reconnect to the rest of the cluster but did allow to autenticate users (which was not possible during the actual network outage, see below). The cluster consists of 1 node in each of 3 datacenters, it uses C* 1.2.16 with SSL enabled both to clients and between C* nodes. The authentication is enabled as well. The problem started around 2014-08-01 when Cassandra first noticed a network problem: INFO [GossipTasks:1] 2014-08-01 07:47:52,618 Gossiper.java (line 823) InetAddress /A.B.D.20 is now DOWN INFO [GossipTasks:1] 2014-08-01 07:47:55,619 Gossiper.java (line 823) InetAddress mia10/A.B.C.20 is now DOWN The network came up for a while: INFO [GossipStage:1] 2014-08-01 07:51:29,380 Gossiper.java (line 809) InetAddress /A.B.C.20 is now UP INFO [HintedHandoff:1] 2014-08-01 07:51:29,381 HintedHandOffManager.java (line 296) Started hinted handoff for host: 9252f37c-1c9a-418b-a49f-6065511946e4 with IP: /A.B.C.20 INFO [GossipStage:1] 2014-08-01 07:51:29,381 Gossiper.java (line 809) InetAddress /A.B.D.20 is now UP INFO [HintedHandoff:2] 2014-08-01 07:51:29,385 HintedHandOffManager.java (line 296) Started hinted handoff for host: 97b1943a-3689-4e4a-a39d-d5a11c0cc309 with IP: /A.B.D.20 But it failed to send hints: INFO [HintedHandoff:1] 2014-08-01 07:51:39,389 HintedHandOffManager.java (line 427) Timed out replaying hints to /A.B.C.20; aborting (0 delivered) INFO [HintedHandoff:2] 2014-08-01 07:51:39,390 HintedHandOffManager.java (line 427) Timed out replaying hints to /A.B.D.20; aborting (0 delivered) Also, the log started to be flooded with failed autentication tries. My understanding is that authentication data are read with QUORUM which failed as the other two nodes were down: ERROR [Native-Transport-Requests:446116] 2014-08-01 07:51:39,985 QueryMessage.java (line 97) Unexpected error during query com.google.common.util.concurrent.UncheckedExecutionException: java.lang.RuntimeException: org.apache.cassandra.exceptions.ReadTimeoutException: Operation timed out - received only 0 responses. at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2258) at com.google.common.cache.LocalCache.get(LocalCache.java:3990) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3994) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4878) at org.apache.cassandra.service.ClientState.authorize(ClientState.java:292) at org.apache.cassandra.service.ClientState.ensureHasPermission(ClientState.java:172) at org.apache.cassandra.service.ClientState.hasAccess(ClientState.java:165) at org.apache.cassandra.service.ClientState.hasColumnFamilyAccess(ClientState.java:149) at org.apache.cassandra.cql3.statements.SelectStatement.checkAccess(SelectStatement.java:116) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:102) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:113) at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:87) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:287) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at