You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2011/02/08 16:51:23 UTC

svn commit: r1068453 - in /cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra: net/MessagingService.java service/GCInspector.java utils/StatusLogger.java

Author: jbellis
Date: Tue Feb  8 15:51:22 2011
New Revision: 1068453

URL: http://svn.apache.org/viewvc?rev=1068453&view=rev
Log:
move GCInspector.logStats to StatusLogger.log, to avoid creating the impression that any time we log stats, it's because of GC
patch by jbellis

Added:
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/StatusLogger.java
Modified:
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/GCInspector.java

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java?rev=1068453&r1=1068452&r2=1068453&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/net/MessagingService.java Tue Feb  8 15:51:22 2011
@@ -45,15 +45,11 @@ import org.apache.cassandra.io.util.Data
 import org.apache.cassandra.locator.ILatencySubscriber;
 import org.apache.cassandra.net.io.SerializerType;
 import org.apache.cassandra.net.sink.SinkManager;
-import org.apache.cassandra.service.GCInspector;
 import org.apache.cassandra.service.ReadCallback;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.FileStreamTask;
 import org.apache.cassandra.streaming.StreamHeader;
-import org.apache.cassandra.utils.ExpiringMap;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.SimpleCondition;
+import org.apache.cassandra.utils.*;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
 public final class MessagingService implements MessagingServiceMBean
@@ -528,7 +524,7 @@ public final class MessagingService impl
         }
 
         if (logTpstats)
-            GCInspector.instance.logStats();
+            StatusLogger.log();
     }
 
     private static class SocketThread extends Thread

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/GCInspector.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/GCInspector.java?rev=1068453&r1=1068452&r2=1068453&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/GCInspector.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/service/GCInspector.java Tue Feb  8 15:51:22 2011
@@ -26,19 +26,13 @@ import java.lang.reflect.InvocationTarge
 import java.lang.reflect.Method;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
-import javax.management.JMX;
 import javax.management.MBeanServer;
-import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
 
-import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.IExecutorMBean;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.CompactionManager;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.StatusLogger;
 
 public class GCInspector
 {
@@ -50,7 +44,6 @@ public class GCInspector
     public static final GCInspector instance = new GCInspector();
 
     private HashMap<String, Long> gctimes = new HashMap<String, Long>();
-    private final MBeanServer server = ManagementFactory.getPlatformMBeanServer();
 
     List<Object> beans = new ArrayList<Object>(); // these are instances of com.sun.management.GarbageCollectorMXBean
 
@@ -133,63 +126,12 @@ public class GCInspector
                 logger.info(st);
             else if (logger.isDebugEnabled())
                 logger.debug(st);
+
             if (gcw.getDuration() > MIN_DURATION_TPSTATS)
-            {
-                logStats();
-            }
+                StatusLogger.log();
         }
     }
 
-    public void logStats()
-    {
-        // everything from o.a.c.concurrent
-        logger.info(String.format("%-25s%10s%10s", "Pool Name", "Active", "Pending"));
-        Set<ObjectName> request, internal;
-        try
-        {
-            request = server.queryNames(new ObjectName("org.apache.cassandra.request:type=*"), null);
-            internal = server.queryNames(new ObjectName("org.apache.cassandra.internal:type=*"), null);
-        }
-        catch (MalformedObjectNameException e)
-        {
-            throw new RuntimeException(e);
-        }
-        for (ObjectName objectName : Iterables.concat(request, internal))
-        {
-            String poolName = objectName.getKeyProperty("type");
-            IExecutorMBean threadPoolProxy = JMX.newMBeanProxy(server, objectName, IExecutorMBean.class);
-            logger.info(String.format("%-25s%10s%10s",
-                                      poolName, threadPoolProxy.getActiveCount(), threadPoolProxy.getPendingTasks()));
-        }
-        // one offs
-        logger.info(String.format("%-25s%10s%10s",
-                                  "CompactionManager", "n/a", CompactionManager.instance.getPendingTasks()));
-        int pendingCommands = 0;
-        for (int n : MessagingService.instance().getCommandPendingTasks().values())
-        {
-            pendingCommands += n;
-        }
-        int pendingResponses = 0;
-        for (int n : MessagingService.instance().getResponsePendingTasks().values())
-        {
-            pendingResponses += n;
-        }
-        logger.info(String.format("%-25s%10s%10s",
-                                  "MessagingService", "n/a", pendingCommands + "," + pendingResponses));
-
-        // per-CF stats
-        logger.info(String.format("%-25s%20s%20s%20s", "ColumnFamily", "Memtable ops,data", "Row cache size/cap", "Key cache size/cap"));
-        for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
-        {
-            logger.info(String.format("%-25s%20s%20s%20s",
-                                      cfs.table.name + "." + cfs.columnFamily,
-                                      cfs.getMemtableColumnsCount() + "," + cfs.getMemtableDataSize(),
-                                      cfs.getRowCacheSize() + "/" + cfs.getRowCacheCapacity(),
-                                      cfs.getKeyCacheSize() + "/" + cfs.getKeyCacheCapacity()));
-        }
-    }
-    
-    
     // wrapper for sun class. this enables other jdks to compile this class.
     private static final class SunGcWrapper
     {
@@ -267,5 +209,4 @@ public class GCInspector
             return usageBeforeGc == null;
         }
     }
-    
 }

Added: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/StatusLogger.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/StatusLogger.java?rev=1068453&view=auto
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/StatusLogger.java (added)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/utils/StatusLogger.java Tue Feb  8 15:51:22 2011
@@ -0,0 +1,75 @@
+package org.apache.cassandra.utils;
+
+import java.lang.management.ManagementFactory;
+import java.util.Set;
+import javax.management.JMX;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import com.google.common.collect.Iterables;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.IExecutorMBean;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.CompactionManager;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.service.GCInspector;
+
+public class StatusLogger
+{
+    private static final Logger logger = LoggerFactory.getLogger(StatusLogger.class);
+
+    public static void log()
+    {
+        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
+        
+        // everything from o.a.c.concurrent
+        logger.info(String.format("%-25s%10s%10s", "Pool Name", "Active", "Pending"));
+        Set<ObjectName> request, internal;
+        try
+        {
+            request = server.queryNames(new ObjectName("org.apache.cassandra.request:type=*"), null);
+            internal = server.queryNames(new ObjectName("org.apache.cassandra.internal:type=*"), null);
+        }
+        catch (MalformedObjectNameException e)
+        {
+            throw new RuntimeException(e);
+        }
+        for (ObjectName objectName : Iterables.concat(request, internal))
+        {
+            String poolName = objectName.getKeyProperty("type");
+            IExecutorMBean threadPoolProxy = JMX.newMBeanProxy(server, objectName, IExecutorMBean.class);
+            logger.info(String.format("%-25s%10s%10s",
+                                      poolName, threadPoolProxy.getActiveCount(), threadPoolProxy.getPendingTasks()));
+        }
+        // one offs
+        logger.info(String.format("%-25s%10s%10s",
+                                  "CompactionManager", "n/a", CompactionManager.instance.getPendingTasks()));
+        int pendingCommands = 0;
+        for (int n : MessagingService.instance().getCommandPendingTasks().values())
+        {
+            pendingCommands += n;
+        }
+        int pendingResponses = 0;
+        for (int n : MessagingService.instance().getResponsePendingTasks().values())
+        {
+            pendingResponses += n;
+        }
+        logger.info(String.format("%-25s%10s%10s",
+                                  "MessagingService", "n/a", pendingCommands + "," + pendingResponses));
+
+        // per-CF stats
+        logger.info(String.format("%-25s%20s%20s%20s", "ColumnFamily", "Memtable ops,data", "Row cache size/cap", "Key cache size/cap"));
+        for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
+        {
+            logger.info(String.format("%-25s%20s%20s%20s",
+                                      cfs.table.name + "." + cfs.columnFamily,
+                                      cfs.getMemtableColumnsCount() + "," + cfs.getMemtableDataSize(),
+                                      cfs.getRowCacheSize() + "/" + cfs.getRowCacheCapacity(),
+                                      cfs.getKeyCacheSize() + "/" + cfs.getKeyCacheCapacity()));
+        }
+    }
+}