You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/12/02 09:24:31 UTC

git commit: Expose thread pool metrics for the native protocol server

Updated Branches:
  refs/heads/cassandra-1.2 a322b645e -> f39350453


Expose thread pool metrics for the native protocol server

patch by mishail; reviewed by slebresne for CASSANDRA-6234


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

Branch: refs/heads/cassandra-1.2
Commit: f393504539d1fd7ba70e9dabb70460f44a600e9d
Parents: a322b64
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Mon Dec 2 09:22:15 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Mon Dec 2 09:22:15 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../transport/RequestThreadPoolExecutor.java    | 31 ++++++++++++++++++--
 2 files changed, 29 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3935045/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f84ffec..2119f77 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -6,6 +6,7 @@
  * cqlsh: quote single quotes in strings inside collections (CASSANDRA-6172)
  * Improve gossip performance for typical messages (CASSANDRA-6409)
  * Throw IRE if a prepared has more markers than supported (CASSANDRA-5598)
+ * Expose Thread metrics for the native protocol server (CASSANDRA-6234)
 
 
 1.2.12

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3935045/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
index d266387..c2e6033 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -17,15 +17,15 @@
  */
 package org.apache.cassandra.transport;
 
-import java.util.concurrent.LinkedBlockingQueue;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 
 import org.jboss.netty.handler.execution.MemoryAwareThreadPoolExecutor;
 import org.jboss.netty.util.ObjectSizeEstimator;
-
 import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.metrics.ThreadPoolMetrics;
 
 public class RequestThreadPoolExecutor extends MemoryAwareThreadPoolExecutor
 {
@@ -33,6 +33,10 @@ public class RequestThreadPoolExecutor extends MemoryAwareThreadPoolExecutor
     // Number of request we accept to queue before blocking. We could allow this to be configured...
     private final static int MAX_QUEUED_REQUESTS = 128;
 
+    private final static String THREAD_FACTORY_ID = "Native-Transport-Requests";
+
+    private final ThreadPoolMetrics metrics;
+
     public RequestThreadPoolExecutor()
     {
         super(DatabaseDescriptor.getNativeTransportMaxThreads(),
@@ -40,7 +44,8 @@ public class RequestThreadPoolExecutor extends MemoryAwareThreadPoolExecutor
               MAX_QUEUED_REQUESTS,
               CORE_THREAD_TIMEOUT_SEC, TimeUnit.SECONDS,
               sizeEstimator(),
-              new NamedThreadFactory("Native-Transport-Requests"));
+              new NamedThreadFactory(THREAD_FACTORY_ID));
+        metrics = new ThreadPoolMetrics(this, "transport", THREAD_FACTORY_ID);
     }
 
     /*
@@ -74,4 +79,24 @@ public class RequestThreadPoolExecutor extends MemoryAwareThreadPoolExecutor
         super.afterExecute(r, t);
         DebuggableThreadPoolExecutor.logExceptionsAfterExecute(r, t);
     }
+
+    @Override
+    public void shutdown()
+    {
+        if (!isShutdown())
+        {
+            metrics.release();
+        }
+        super.shutdown();
+    }
+
+    @Override
+    public List<Runnable> shutdownNow()
+    {
+        if (!isShutdown())
+        {
+            metrics.release();
+        }
+        return super.shutdownNow();
+    }
 }