Cassandra
  1. Cassandra
  2. CASSANDRA-3335

ThreadPoolExecutor creates threads as non-daemon and will block on shutdown by default

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Minor Minor
    • Resolution: Fixed
    • Fix Version/s: 1.0.7
    • Component/s: Core
    • Labels:
      None

      Description

      This is most obviously visible in OptionalTasks which should not block shutdown, but often does.

      1. 3335-v4.txt
        13 kB
        Jonathan Ellis
      2. 3335-v3.txt
        1 kB
        Jonathan Ellis
      3. 3335v3_jstack.txt
        264 kB
        Brandon Williams
      4. 3335-v2.txt
        15 kB
        Jonathan Ellis
      5. 3335.txt
        3 kB
        Jonathan Ellis

        Activity

        Hide
        Jonathan Ellis added a comment -

        Patch that switches DTPE to daemon threads, and Memtable's jamm TPE to DTPE. The only other direct use of TPE is the thrift handler for sync mode in ACD, which doesn't seem to have a problem killing things off when we ask it to so I left it alone.

        Show
        Jonathan Ellis added a comment - Patch that switches DTPE to daemon threads, and Memtable's jamm TPE to DTPE. The only other direct use of TPE is the thrift handler for sync mode in ACD, which doesn't seem to have a problem killing things off when we ask it to so I left it alone.
        Hide
        Brandon Williams added a comment -

        +1 on this patch for being technically correct, but it actually does not help. What I see is OptionalTasks jump to 100% on control-c, and then eventually it spins down and a few minutes later java finally exits. During the last bit, I can see jamm still working. I'm not sure why java is taking so long to exit, but it is responsive to nodetool the entire time, though tpstats shows nothing active.

        Show
        Brandon Williams added a comment - +1 on this patch for being technically correct, but it actually does not help. What I see is OptionalTasks jump to 100% on control-c, and then eventually it spins down and a few minutes later java finally exits. During the last bit, I can see jamm still working. I'm not sure why java is taking so long to exit, but it is responsive to nodetool the entire time, though tpstats shows nothing active.
        Hide
        Jonathan Ellis added a comment -

        v2 adds a log line before running any scheduled task. Maybe that will help pinpoint the culprit.

        Show
        Jonathan Ellis added a comment - v2 adds a log line before running any scheduled task. Maybe that will help pinpoint the culprit.
        Hide
        Jonathan Ellis added a comment -

        (all the task logging is done w/ the StorageService logger so it can be enabled separately from the reset of the package involved)

        Show
        Jonathan Ellis added a comment - (all the task logging is done w/ the StorageService logger so it can be enabled separately from the reset of the package involved)
        Hide
        Brandon Williams added a comment -

        Tracked down to the shutdown hook:

           java.lang.Thread.State: TIMED_WAITING (sleeping)
            at java.lang.Thread.sleep(Native Method)
            at org.apache.cassandra.utils.ExpiringMap.shutdown(ExpiringMap.java:103)
            at org.apache.cassandra.net.MessagingService.shutdown(MessagingService.java:495)
            at org.apache.cassandra.service.StorageService$2.runMayThrow(StorageService.java:426)
            at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
            at java.lang.Thread.run(Thread.java:662)
        

        Appears to be the ExpiringMap added in CASSANDRA-2034.

        Show
        Brandon Williams added a comment - Tracked down to the shutdown hook: java.lang.Thread.State: TIMED_WAITING (sleeping) at java.lang.Thread.sleep(Native Method) at org.apache.cassandra.utils.ExpiringMap.shutdown(ExpiringMap.java:103) at org.apache.cassandra.net.MessagingService.shutdown(MessagingService.java:495) at org.apache.cassandra.service.StorageService$2.runMayThrow(StorageService.java:426) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) at java.lang.Thread.run(Thread.java:662) Appears to be the ExpiringMap added in CASSANDRA-2034 .
        Hide
        Jonathan Ellis added a comment -

        Patch to stop adding new messages to the callback map when MessagingService is shutdown.

        Show
        Jonathan Ellis added a comment - Patch to stop adding new messages to the callback map when MessagingService is shutdown.
        Hide
        Brandon Williams added a comment -

        Doesn't seem to help. Thread dump attached.

        Show
        Brandon Williams added a comment - Doesn't seem to help. Thread dump attached.
        Hide
        Jonathan Ellis added a comment -

        Okay, take four here. The problem with v3 is that we were only blocking sendOneWay during shutdown, not addCallback, which is the source of the ExpiringMap entries we were waiting for.

        As I commented,

            /**
             * There isn't a good way to shut down the MessagingService. One problem (but not the only one)
             * is that StorageProxy has no way to communicate back to clients, "I'm nominally alive, but I can't
             * send that request to the nodes with your data."  Neither TimedOut nor Unavailable is appropriate
             * to return in that situation.
             *
             * So instead of shutting down MS and letting StorageProxy/clients cope somehow, we shut down
             * the Thrift service and then wait for all the outstanding requests to finish or timeout.
             */
        

        That part was straightforward. I also had to make the Thrift shutdown actually work – we were calling setSoTimeout to attempt to make accept() nonblocking, but "0" means "wait indefinitely" not "don't wait at all". Then we needed to handle the timeout in the accept loop.

        Finally, I did a bunch of cleanup to ExpiringMap and added trace-level logging in case we need to go at this again.

        Show
        Jonathan Ellis added a comment - Okay, take four here. The problem with v3 is that we were only blocking sendOneWay during shutdown, not addCallback, which is the source of the ExpiringMap entries we were waiting for. As I commented, /** * There isn't a good way to shut down the MessagingService. One problem (but not the only one) * is that StorageProxy has no way to communicate back to clients, "I'm nominally alive, but I can't * send that request to the nodes with your data." Neither TimedOut nor Unavailable is appropriate * to return in that situation. * * So instead of shutting down MS and letting StorageProxy/clients cope somehow, we shut down * the Thrift service and then wait for all the outstanding requests to finish or timeout. */ That part was straightforward. I also had to make the Thrift shutdown actually work – we were calling setSoTimeout to attempt to make accept() nonblocking, but "0" means "wait indefinitely" not "don't wait at all". Then we needed to handle the timeout in the accept loop. Finally, I did a bunch of cleanup to ExpiringMap and added trace-level logging in case we need to go at this again .
        Hide
        Brandon Williams added a comment -

        +1

        Show
        Brandon Williams added a comment - +1
        Hide
        Jonathan Ellis added a comment -

        committed

        Show
        Jonathan Ellis added a comment - committed

          People

          • Assignee:
            Jonathan Ellis
            Reporter:
            Brandon Williams
            Reviewer:
            Brandon Williams
          • Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development