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

Hao Bryan Cheng commented on CASSANDRA-10477:
---------------------------------------------

Hello, we just observed this on a cluster running 2.1.11, Oracle Java 1.8.0_66.

A single machine experienced this issue, causing our entire cluster to grind to 
a halt on any quorum operations.

Our logs feature an extremely large number of:

{code}
ERROR [EXPIRING-MAP-REAPER:1] 2015-11-11 05:10:22,894 CassandraDaemon.java:227 
- Exception in threa
d Thread[EXPIRING-MAP-REAPER:1,5,main]
java.lang.AssertionError: /172.31.3.33
        at 
org.apache.cassandra.service.StorageProxy.submitHint(StorageProxy.java:949) 
~[apache-cas
sandra-2.1.11.jar:2.1.11]
        at 
org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:383) 
~[apache-ca
ssandra-2.1.11.jar:2.1.11]
        at 
org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:363) 
~[apache-ca
ssandra-2.1.11.jar:2.1.11]
        at org.apache.cassandra.utils.ExpiringMap$1.run(ExpiringMap.java:98) 
~[apache-cassandra-2.1
.11.jar:2.1.11]
        at 
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunna
ble.run(DebuggableScheduledThreadPoolExecutor.java:118) 
~[apache-cassandra-2.1.11.jar:2.1.11]
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
[na:1.8.0_66]
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
[na:1.8.0_66]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(Schedule
dThreadPoolExecutor.java:180) [na:1.8.0_66]
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThread
PoolExecutor.java:294) [na:1.8.0_66]
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) 
[na:1.8.
0_66]
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
[na:1.8.
0_66]
        at java.lang.Thread.run(Thread.java:745) [na:1.8.0_66]
{code}

Additionally, this is interspersed with every nearly every other neighbor node 
being marked down:

{code}
INFO  [GossipStage:1] 2015-11-11 04:14:25,369 Gossiper.java:1020 - Node 
/172.31.55.172 has restarted, now UP
INFO  [GossipStage:1] 2015-11-11 04:14:25,369 TokenMetadata.java:414 - Updating 
topology for /172.31.55.172
INFO  [GossipStage:1] 2015-11-11 04:14:25,369 TokenMetadata.java:414 - Updating 
topology for /172.31.55.172
INFO  [GossipStage:1] 2015-11-11 04:14:25,370 StorageService.java:1698 - Node 
/172.31.55.172 state jump to normal
INFO  [GossipStage:1] 2015-11-11 04:14:25,372 TokenMetadata.java:414 - Updating 
topology for /172.31.55.172
INFO  [GossipStage:1] 2015-11-11 04:14:25,372 TokenMetadata.java:414 - Updating 
topology for /172.31.55.172
INFO  [SharedPool-Worker-3] 2015-11-11 04:14:25,531 Gossiper.java:987 - 
InetAddress /172.31.55.172 is now UP
INFO  [SharedPool-Worker-5] 2015-11-11 04:14:25,536 Gossiper.java:987 - 
InetAddress /172.31.55.172 is now UP
INFO  [SharedPool-Worker-3] 2015-11-11 04:14:25,536 Gossiper.java:987 - 
InetAddress /172.31.55.172 is now UP
INFO  [SharedPool-Worker-1] 2015-11-11 04:14:25,536 Gossiper.java:987 - 
InetAddress /172.31.55.172 is now UP
INFO  [SharedPool-Worker-4] 2015-11-11 04:14:25,536 Gossiper.java:987 - 
InetAddress /172.31.55.172 is now UP
INFO  [HANDSHAKE-/172.31.55.172] 2015-11-11 04:14:25,537 
OutboundTcpConnection.java:485 - Handshaking version with /172.31.55.172
[snipped]
WARN  [GossipTasks:1] 2015-11-11 04:18:26,379 Gossiper.java:747 - Gossip stage 
has 15 pending tasks
; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-11-11 04:18:27,480 Gossiper.java:747 - Gossip stage 
has 17 pending tasks
; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-11-11 04:18:28,580 Gossiper.java:747 - Gossip stage 
has 19 pending tasks
; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-11-11 04:18:29,681 Gossiper.java:747 - Gossip stage 
has 21 pending tasks
; skipping status check (no nodes will be marked down)
WARN  [GossipTasks:1] 2015-11-11 04:18:30,781 Gossiper.java:747 - Gossip stage 
has 25 pending tasks
; skipping status check (no nodes will be marked down)
...
{code}

No other nodes were restarted in this time frame.

Please let us know if there is any additional information we can provide.


> java.lang.AssertionError in StorageProxy.submitHint
> ---------------------------------------------------
>
>                 Key: CASSANDRA-10477
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10477
>             Project: Cassandra
>          Issue Type: Bug
>         Environment: CentOS 6, Oracle JVM 1.8.45
>            Reporter: Severin Leonhardt
>            Assignee: Ariel Weisberg
>             Fix For: 2.1.x
>
>
> A few days after updating from 2.0.15 to 2.1.9 we have the following log 
> entry on 2 of 5 machines:
> {noformat}
> ERROR [EXPIRING-MAP-REAPER:1] 2015-10-07 17:01:08,041 
> CassandraDaemon.java:223 - Exception in thread 
> Thread[EXPIRING-MAP-REAPER:1,5,main]
> java.lang.AssertionError: /192.168.11.88
>         at 
> org.apache.cassandra.service.StorageProxy.submitHint(StorageProxy.java:949) 
> ~[apache-cassandra-2.1.9.jar:2.1.9]
>         at 
> org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:383) 
> ~[apache-cassandra-2.1.9.jar:2.1.9]
>         at 
> org.apache.cassandra.net.MessagingService$5.apply(MessagingService.java:363) 
> ~[apache-cassandra-2.1.9.jar:2.1.9]
>         at org.apache.cassandra.utils.ExpiringMap$1.run(ExpiringMap.java:98) 
> ~[apache-cassandra-2.1.9.jar:2.1.9]
>         at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  ~[apache-cassandra-2.1.9.jar:2.1.9]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_45]
>         at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
> [na:1.8.0_45]
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  [na:1.8.0_45]
>         at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  [na:1.8.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  [na:1.8.0_45]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_45]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> {noformat}
> 192.168.11.88 is the broadcast address of the local machine.
> When this is logged the read request latency of the whole cluster becomes 
> very bad, from 6 ms/op to more than 100 ms/op according to OpsCenter. Clients 
> get a lot of timeouts. We need to restart the affected Cassandra node to get 
> back normal read latencies. It seems write latency is not affected.
> Disabling hinted handoff using {{nodetool disablehandoff}} only prevents the 
> assert from being logged. At some point the read latency becomes bad again. 
> Restarting the node where hinted handoff was disabled results in the read 
> latency being better again.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to