Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-487

Message Serializer slows down/stops responding

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Fixed
    • 0.4
    • None
    • None
    • Normal

    Description

      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.

      Attachments

        1. system-487.log.gz
          12 kB
          Sammy Yu
        2. 487-lock-all-connection-ops.patch
          5 kB
          Jonathan Ellis
        3. 0001-Added-locks-around-remove-operation-so-that-Concurre.patch
          1 kB
          Sammy Yu

        Activity

          People

            sammy.yu Sammy Yu
            sammy.yu Sammy Yu
            Sammy Yu
            Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: