cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r937118 - in /cassandra/trunk/src/java/org/apache/cassandra: concurrent/StageManager.java db/ColumnFamilyStore.java net/MessagingService.java service/StorageService.java
Date Fri, 23 Apr 2010 00:51:24 GMT
Author: jbellis
Date: Fri Apr 23 00:51:24 2010
New Revision: 937118

URL: http://svn.apache.org/viewvc?rev=937118&view=rev
Log:
set limit of 60s on "extra" threads
patch by jbellis; reviewed by rschildmeijer for CASSANDRA-805

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/concurrent/StageManager.java
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java
    cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/concurrent/StageManager.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/concurrent/StageManager.java?rev=937118&r1=937117&r2=937118&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/concurrent/StageManager.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/concurrent/StageManager.java Fri Apr 23
00:51:24 2010
@@ -49,6 +49,8 @@ public class StageManager
     private static final String LOADBALANCE_STAGE = "LOAD-BALANCER-STAGE";
     public static final String MIGRATION_STAGE = "MIGRATION-STAGE";
 
+    public static final long KEEPALIVE = 60; // seconds to keep "extra" threads alive for
when idle
+
     static
     {
         stages.put(MUTATION_STAGE, multiThreadedStage(MUTATION_STAGE, getConcurrentWriters()));
@@ -70,7 +72,7 @@ public class StageManager
 
         return new JMXEnabledThreadPoolExecutor(numThreads,
                                                 numThreads,
-                                                Integer.MAX_VALUE,
+                                                KEEPALIVE,
                                                 TimeUnit.SECONDS,
                                                 new LinkedBlockingQueue<Runnable>(DatabaseDescriptor.getStageQueueSize()),
                                                 new NamedThreadFactory(name));

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=937118&r1=937117&r2=937118&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Fri Apr 23 00:51:24
2010
@@ -37,6 +37,7 @@ import org.apache.commons.collections.It
 import com.google.common.collect.Iterables;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
@@ -44,7 +45,6 @@ import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Bounds;
-import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.*;
@@ -77,14 +77,14 @@ public class ColumnFamilyStore implement
     private static ExecutorService flushSorter_
             = new JMXEnabledThreadPoolExecutor(1,
                                                Runtime.getRuntime().availableProcessors(),
-                                               Integer.MAX_VALUE,
+                                               StageManager.KEEPALIVE,
                                                TimeUnit.SECONDS,
                                                new LinkedBlockingQueue<Runnable>(1
+ Runtime.getRuntime().availableProcessors()),
                                                new NamedThreadFactory("FLUSH-SORTER-POOL"));
     private static ExecutorService flushWriter_
             = new JMXEnabledThreadPoolExecutor(1,
                                                DatabaseDescriptor.getAllDataFileLocations().length,
-                                               Integer.MAX_VALUE,
+                                               StageManager.KEEPALIVE,
                                                TimeUnit.SECONDS,
                                                new LinkedBlockingQueue<Runnable>(1
+ 2 * DatabaseDescriptor.getAllDataFileLocations().length),
                                                new NamedThreadFactory("FLUSH-WRITER-POOL"));

Modified: cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java?rev=937118&r1=937117&r2=937118&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/net/MessagingService.java Fri Apr 23 00:51:24
2010
@@ -97,7 +97,7 @@ public class MessagingService implements
         // read executor puts messages to deserialize on this.
         messageDeserializerExecutor_ = new JMXEnabledThreadPoolExecutor(1,
                                                                         Runtime.getRuntime().availableProcessors(),
-                                                                        Integer.MAX_VALUE,
+                                                                        StageManager.KEEPALIVE,
                                                                         TimeUnit.SECONDS,
                                                                         new LinkedBlockingQueue<Runnable>(),
                                                                         new NamedThreadFactory("MESSAGE-DESERIALIZER-POOL"));

Modified: cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=937118&r1=937117&r2=937118&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java Fri Apr 23 00:51:24
2010
@@ -139,7 +139,7 @@ public class StorageService implements I
     /* This thread pool does consistency checks when the client doesn't care about consistency
*/
     private ExecutorService consistencyManager_ = new JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getConsistencyThreads(),
                                                                                    DatabaseDescriptor.getConsistencyThreads(),
-                                                                                   Integer.MAX_VALUE,
+                                                                                   StageManager.KEEPALIVE,
                                                                                    TimeUnit.SECONDS,
                                                                                    new LinkedBlockingQueue<Runnable>(),
                                                                                    new NamedThreadFactory("CONSISTENCY-MANAGER"));



Mime
View raw message