[
https://issues.apache.org/jira/browse/CASSANDRA-487?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12765631#action_12765631
]
Jonathan Ellis commented on CASSANDRA-487:
------------------------------------------
One other race here (not in my patch):
if (contains(connection))
{
return;
}
needs to be inside the lock_.
> Message Serializer slows down/stops responding
> ----------------------------------------------
>
> Key: CASSANDRA-487
> URL: https://issues.apache.org/jira/browse/CASSANDRA-487
> Project: Cassandra
> Issue Type: Bug
> Affects Versions: 0.4
> Reporter: Sammy Yu
> Assignee: Sammy Yu
> Fix For: 0.5
>
> Attachments:
> 0001-Added-locks-around-remove-operation-so-that-Concurre.patch,
> 487-lock-all-connection-ops.patch, system-487.log.gz
>
>
> We ran into an issue with where the MESSAGE-SERIALIZER-POOL piles up with
> tasks.
> $ /usr/sbin/nodeprobe -host localhost tpstats
> FILEUTILS-DELETE-POOL, pending tasks=0
> MESSAGING-SERVICE-POOL, pending tasks=0
> MESSAGE-SERIALIZER-POOL, pending tasks=10785714
> RESPONSE-STAGE, pending tasks=0
> BOOT-STRAPPER, pending tasks=0
> ROW-READ-STAGE, pending tasks=0
> MESSAGE-DESERIALIZER-POOL, pending tasks=0
> GMFD, pending tasks=0
> LB-TARGET, pending tasks=0
> CONSISTENCY-MANAGER, pending tasks=0
> ROW-MUTATION-STAGE, pending tasks=0
> MESSAGE-STREAMING-POOL, pending tasks=0
> LOAD-BALANCER-STAGE, pending tasks=0
> MEMTABLE-FLUSHER-POOL, pending tasks=0
> In the log, this seems to have happened when we stopped 2 of the other nodes
> in our cluster. This node will time out on any thrift requests. Looking
> through the logs we found the following two exceptions:
> java.util.ConcurrentModificationException
> at
> java.util.AbstractList$Itr.checkForComodification(AbstractList.java:372)
> at java.util.AbstractList$Itr.next(AbstractList.java:349)
> at java.util.Collections.sort(Collections.java:120)
> at
> org.apache.cassandra.net.TcpConnectionManager.getLeastLoaded(TcpConnectionManager.java:108)
> at
> org.apache.cassandra.net.TcpConnectionManager.getConnection(TcpConnectionManager.java:71)
> at
> org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:306)
> at
> org.apache.cassandra.net.MessageSerializationTask.run(MessageSerializationTask.java:66)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:619)
> java.util.NoSuchElementException
> at java.util.AbstractList$Itr.next(AbstractList.java:350)
> at java.util.Collections.sort(Collections.java:120)
> at
> org.apache.cassandra.net.TcpConnectionManager.getLeastLoaded(TcpConnectionManager.java:108)
> at
> org.apache.cassandra.net.TcpConnectionManager.getConnection(TcpConnectionManager.java:71)
> at
> org.apache.cassandra.net.MessagingService.getConnection(MessagingService.java:306)
> at
> org.apache.cassandra.net.MessageSerializationTask.run(MessageSerializationTask.java:66)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:619)
> This appears to have happened on all 4 MESSAGE-SERIALIZER-POOL threads
> I will attach the complete log.
--
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.