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

GC issues with LinkedBlockingQueue, very large heap sizes

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Won't Fix
    • Fix Version/s: 0.6
    • Component/s: None
    • Labels:
      None

      Description

      We were doing a large import over thrift today (250M rows) with 50 columns each. We noticed when this process started, that the heap usage slowly began to rise. Eventually hitting 10GB. After investigating heap dumps we noticed this is linked to GC problems in the JVM for LinkedBlockingQueue. Sending a hint through jconsole to perform GC didn't help either.

      See this article:

      http://tech.puredanger.com/2009/02/11/linkedblockingqueue-garbagecollection/

      JDK 1.6.18 doesn't have this fix yet either. We could possibly add the change from:
      http://svn.apache.org/repos/asf/harmony/standard/classlib/trunk/modules/concurrent/src/main/java/java/util/concurrent/LinkedBlockingQueue.java

      1. HeapMemoryLBQvsABQ.png
        73 kB
        Chris Goffinet
      2. Screen shot 2010-03-14 at 10.19.09 PM.png
        117 kB
        Chris Goffinet
      3. Screen shot 2010-03-14 at 10.20.31 PM.png
        139 kB
        Chris Goffinet
      4. WriteOpsLBQvsABQ.png
        57 kB
        Chris Goffinet

        Activity

        Hide
        lenn0x Chris Goffinet added a comment -

        After waiting a bit, doing a Full GC actually worked as mentioned in above articles. Heap dropped to appropriate levels.

        Show
        lenn0x Chris Goffinet added a comment - After waiting a bit, doing a Full GC actually worked as mentioned in above articles. Heap dropped to appropriate levels.
        Hide
        lenn0x Chris Goffinet added a comment -

        I took the class from harmony, and added it to Cassandra, making the modification changes. I don't see much change. Looking closer it seems to be that the majority of data comes from OutboundTCPConnection

        Show
        lenn0x Chris Goffinet added a comment - I took the class from harmony, and added it to Cassandra, making the modification changes. I don't see much change. Looking closer it seems to be that the majority of data comes from OutboundTCPConnection
        Show
        lenn0x Chris Goffinet added a comment - http://bugs.sun.com/view_bug.do;jsessionid=60c39aa55d3666c0c84dd70eb826?bug_id=6805775
        Hide
        jbellis Jonathan Ellis added a comment -

        if OutboundTCPConnection still has a reference to it, it should still be in the queue and show up in jmx.

        Show
        jbellis Jonathan Ellis added a comment - if OutboundTCPConnection still has a reference to it, it should still be in the queue and show up in jmx.
        Hide
        jbellis Jonathan Ellis added a comment -

        ... oops, missed the part about full GC getting it eventually.

        Show
        jbellis Jonathan Ellis added a comment - ... oops, missed the part about full GC getting it eventually.
        Hide
        lenn0x Chris Goffinet added a comment -

        JDK7 b85 does have this fix. As discussed with jbellis, I will try ArrayBlockingQueue. 0.5 with NIO used ConcurrentLinkedQueue

        Show
        lenn0x Chris Goffinet added a comment - JDK7 b85 does have this fix. As discussed with jbellis, I will try ArrayBlockingQueue. 0.5 with NIO used ConcurrentLinkedQueue
        Hide
        lenn0x Chris Goffinet added a comment -

        ArrayBlockingQueue helps keep memory under control. I set the capacity to 4096 (stagingQueueSize). Test is still running but within first 15 minutes its stable. I will measure different sizes, as this is going to limit writes (blocking queue). I think we will have to make this a tunable so users can trade off memory for higher write throughput.

        Show
        lenn0x Chris Goffinet added a comment - ArrayBlockingQueue helps keep memory under control. I set the capacity to 4096 (stagingQueueSize). Test is still running but within first 15 minutes its stable. I will measure different sizes, as this is going to limit writes (blocking queue). I think we will have to make this a tunable so users can trade off memory for higher write throughput.
        Hide
        jbellis Jonathan Ellis added a comment -

        tying it to stage queue size should work fine, maybe with a factor of 2x or 0.5 if your tests show that's useful.

        Show
        jbellis Jonathan Ellis added a comment - tying it to stage queue size should work fine, maybe with a factor of 2x or 0.5 if your tests show that's useful.
        Hide
        lenn0x Chris Goffinet added a comment -

        I setup a 10 node cluster using Rackaware, west/east coast. Average Latency between costs around 60ms~.

        stress.py parameters:

        Insert
        Columns = 50
        Number of Keys = 3M

        Show
        lenn0x Chris Goffinet added a comment - I setup a 10 node cluster using Rackaware, west/east coast. Average Latency between costs around 60ms~. stress.py parameters: Insert Columns = 50 Number of Keys = 3M
        Hide
        lenn0x Chris Goffinet added a comment -

        ABQ capacity was set to 8192

        Show
        lenn0x Chris Goffinet added a comment - ABQ capacity was set to 8192
        Hide
        jbellis Jonathan Ellis added a comment -

        [Chris adds that the pending operations for the messagingservice was 0, indicating that the lower throughput was from reduced concurrency in ABQ vs LBQ rather than blocking from the capped queue size.]

        Let's wait for the JDK's fix, because going back to 0.5 ConcurrentLinkedQueue + manual locking is likely to have the same concurrency characteristics as ABQ here.

        You might be able to mitigate it by adding

        -XX:CMSInitiatingOccupancyFraction=50

        to have it start full GC at 50% of heap... at the cost of doing more GCs of course.

        Show
        jbellis Jonathan Ellis added a comment - [Chris adds that the pending operations for the messagingservice was 0, indicating that the lower throughput was from reduced concurrency in ABQ vs LBQ rather than blocking from the capped queue size.] Let's wait for the JDK's fix, because going back to 0.5 ConcurrentLinkedQueue + manual locking is likely to have the same concurrency characteristics as ABQ here. You might be able to mitigate it by adding -XX:CMSInitiatingOccupancyFraction=50 to have it start full GC at 50% of heap... at the cost of doing more GCs of course.

          People

          • Assignee:
            Unassigned
            Reporter:
            lenn0x Chris Goffinet
          • Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development