cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sn...@apache.org
Subject [1/3] cassandra git commit: GCInspector logs very different times
Date Wed, 29 Apr 2015 21:49:24 GMT
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 9bcdd0fb7 -> 8f2a26c4f
  refs/heads/trunk c57984216 -> cdfa9ed7e


GCInspector logs very different times

Patch by Ariel Weisberg; Reviewed by Robert Stupp for CASSANDRA-9124


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8f2a26c4
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8f2a26c4
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8f2a26c4

Branch: refs/heads/cassandra-2.1
Commit: 8f2a26c4f109d88a8fc1a41138de1c3d0bf39a05
Parents: 9bcdd0f
Author: Ariel Weisberg <ariel.weisberg@datastax.com>
Authored: Wed Apr 29 23:38:53 2015 +0200
Committer: Robert Stupp <snazy@snazy.de>
Committed: Wed Apr 29 23:38:53 2015 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/service/GCInspector.java   | 140 +++++++++++++++++--
 2 files changed, 126 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8f2a26c4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ac68df9..39bcc7c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.6
+ * GCInspector logs very different times (CASSANDRA-9124)
  * Fix deleting from an empty list (CASSANDRA-9198)
  * Update tuple and collection types that use a user-defined type when that UDT
    is modified (CASSANDRA-9148, CASSANDRA-9192)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8f2a26c4/src/java/org/apache/cassandra/service/GCInspector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/GCInspector.java b/src/java/org/apache/cassandra/service/GCInspector.java
index c4bffac..df0527f 100644
--- a/src/java/org/apache/cassandra/service/GCInspector.java
+++ b/src/java/org/apache/cassandra/service/GCInspector.java
@@ -17,11 +17,12 @@
  */
 package org.apache.cassandra.service;
 
+import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryUsage;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.management.MBeanServer;
@@ -34,6 +35,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.sun.management.GarbageCollectionNotificationInfo;
+import com.sun.management.GcInfo;
 import org.apache.cassandra.io.sstable.SSTableDeletingTask;
 import org.apache.cassandra.utils.StatusLogger;
 
@@ -70,21 +72,57 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
         }
     }
 
+    static final class GCState
+    {
+        final GarbageCollectorMXBean gcBean;
+        final boolean assumeGCIsPartiallyConcurrent;
+        final boolean assumeGCIsOldGen;
+        private String[] keys;
+        long lastGcTotalDuration = 0;
+
+
+        GCState(GarbageCollectorMXBean gcBean, boolean assumeGCIsPartiallyConcurrent, boolean
assumeGCIsOldGen)
+        {
+            this.gcBean = gcBean;
+            this.assumeGCIsPartiallyConcurrent = assumeGCIsPartiallyConcurrent;
+            this.assumeGCIsOldGen = assumeGCIsOldGen;
+        }
+
+        String[] keys(GarbageCollectionNotificationInfo info)
+        {
+            if (keys != null)
+                return keys;
+
+            keys = info.getGcInfo().getMemoryUsageBeforeGc().keySet().toArray(new String[0]);
+            Arrays.sort(keys);
+
+            return keys;
+        }
+    }
+
     final AtomicReference<State> state = new AtomicReference<>(new State());
 
+    final Map<String, GCState> gcStates = new HashMap<>();
+
     public GCInspector()
     {
         MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
 
         try
         {
+            ObjectName gcName = new ObjectName(ManagementFactory.GARBAGE_COLLECTOR_MXBEAN_DOMAIN_TYPE
+ ",*");
+            for (ObjectName name : mbs.queryNames(gcName, null))
+            {
+                GarbageCollectorMXBean gc = ManagementFactory.newPlatformMXBeanProxy(mbs,
name.getCanonicalName(), GarbageCollectorMXBean.class);
+                gcStates.put(gc.getName(), new GCState(gc, assumeGCIsPartiallyConcurrent(gc),
assumeGCIsOldGen(gc)));
+            }
+
             mbs.registerMBean(this, new ObjectName(MBEAN_NAME));
         }
         catch (Exception e)
         {
             throw new RuntimeException(e);
         }
-
     }
 
     public static void register() throws Exception
@@ -98,7 +136,64 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
         }
     }
 
-    public void handleNotification(Notification notification, Object handback)
+    /*
+     * Assume that a GC type is at least partially concurrent and so a side channel method
+     * should be used to calculate application stopped time due to the GC.
+     *
+     * If the GC isn't recognized then assume that is concurrent and we need to do our own
calculation
+     * via the the side channel.
+     */
+    private static boolean assumeGCIsPartiallyConcurrent(GarbageCollectorMXBean gc)
+    {
+        switch (gc.getName())
+        {
+                //First two are from the serial collector
+            case "Copy":
+            case "MarkSweepCompact":
+                //Parallel collector
+            case "PS MarkSweep":
+            case "PS Scavenge":
+            case "G1 Young Generation":
+                //CMS young generation collector
+            case "ParNew":
+                return false;
+            case "ConcurrentMarkSweep":
+            case "G1 Old Generation":
+                return true;
+            default:
+                //Assume possibly concurrent if unsure
+                return true;
+        }
+    }
+
+    /*
+     * Assume that a GC type is an old generation collection so SSTableDeletingTask.rescheduleFailedTasks()
+     * should be invoked.
+     *
+     * Defaults to not invoking SSTableDeletingTask.rescheduleFailedTasks() on unrecognized
GC names
+     */
+    private static boolean assumeGCIsOldGen(GarbageCollectorMXBean gc)
+    {
+        switch (gc.getName())
+        {
+            case "Copy":
+            case "PS Scavenge":
+            case "G1 Young Generation":
+            case "ParNew":
+                return false;
+            case "MarkSweepCompact":
+            case "PS MarkSweep":
+            case "ConcurrentMarkSweep":
+            case "G1 Old Generation":
+                return true;
+            default:
+                //Assume not old gen otherwise, don't call
+                //SSTableDeletingTask.rescheduleFailedTasks()
+                return false;
+        }
+    }
+
+    public void handleNotification(final Notification notification, final Object handback)
     {
         String type = notification.getType();
         if (type.equals(GarbageCollectionNotificationInfo.GARBAGE_COLLECTION_NOTIFICATION))
@@ -106,25 +201,40 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
             // retrieve the garbage collection notification information
             CompositeData cd = (CompositeData) notification.getUserData();
             GarbageCollectionNotificationInfo info = GarbageCollectionNotificationInfo.from(cd);
+            String gcName = info.getGcName();
+            GcInfo gcInfo = info.getGcInfo();
 
-            long duration = info.getGcInfo().getDuration();
+            long duration = gcInfo.getDuration();
+
+            /*
+             * The duration supplied in the notification info includes more than just
+             * application stopped time for concurrent GCs. Try and do a better job coming
up with a good stopped time
+             * value by asking for and tracking cumulative time spent blocked in GC.
+             */
+            GCState gcState = gcStates.get(gcName);
+            if (gcState.assumeGCIsPartiallyConcurrent)
+            {
+                long previousTotal = gcState.lastGcTotalDuration;
+                long total = gcState.gcBean.getCollectionTime();
+                gcState.lastGcTotalDuration = total;
+                duration = total - previousTotal; // may be zero for a really fast collection
+            }
 
             StringBuilder sb = new StringBuilder();
             sb.append(info.getGcName()).append(" GC in ").append(duration).append("ms.  ");
-
             long bytes = 0;
-            List<String> keys = new ArrayList<>(info.getGcInfo().getMemoryUsageBeforeGc().keySet());
-            Collections.sort(keys);
-            for (String key : keys)
+            Map<String, MemoryUsage> beforeMemoryUsage = gcInfo.getMemoryUsageBeforeGc();
+            Map<String, MemoryUsage> afterMemoryUsage = gcInfo.getMemoryUsageAfterGc();
+            for (String key : gcState.keys(info))
             {
-                MemoryUsage before = info.getGcInfo().getMemoryUsageBeforeGc().get(key);
-                MemoryUsage after = info.getGcInfo().getMemoryUsageAfterGc().get(key);
+                MemoryUsage before = beforeMemoryUsage.get(key);
+                MemoryUsage after = afterMemoryUsage.get(key);
                 if (after != null && after.getUsed() != before.getUsed())
                 {
                     sb.append(key).append(": ").append(before.getUsed());
                     sb.append(" -> ");
                     sb.append(after.getUsed());
-                    if (!key.equals(keys.get(keys.size() - 1)))
+                    if (!key.equals(gcState.keys[gcState.keys.length - 1]))
                         sb.append("; ");
                     bytes += before.getUsed() - after.getUsed();
                 }
@@ -146,8 +256,8 @@ public class GCInspector implements NotificationListener, GCInspectorMXBean
             if (duration > MIN_LOG_DURATION_TPSTATS)
                 StatusLogger.log();
 
-            // if we just finished a full collection and we're still using a lot of memory,
try to reduce the pressure
-            if (info.getGcName().equals("ConcurrentMarkSweep"))
+            // if we just finished an old gen collection and we're still using a lot of memory,
try to reduce the pressure
+            if (gcState.assumeGCIsOldGen)
                 SSTableDeletingTask.rescheduleFailedTasks();
         }
     }


Mime
View raw message