Author: jake
Date: Fri Oct 21 21:09:42 2011
New Revision: 1187540

URL: http://svn.apache.org/viewvc?rev=1187540&view=rev
Log:
Cache HSHA threads
Patch by Vijay2win; reviewed by tjake for CASSANDRA-3372
 

Modified:
    cassandra/branches/cassandra-1.0/CHANGES.txt
    
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/thrift/CassandraDaemon.java

Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1187540&r1=1187539&r2=1187540&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Fri Oct 21 21:09:42 2011
@@ -21,6 +21,8 @@
  * CompactionManager is now responsible for checking for new candidates
    post-task execution, enabling more consistent leveled compaction 
    (CASSANDRA-3391)
+ * Cache HSHA threads (CASSANDRA-3372)
+
 Merged from 0.8:
  * (CQL) update grammar to require key clause in DELETE statement
    (CASSANDRA-3349)

Modified: 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/thrift/CassandraDaemon.java?rev=1187540&r1=1187539&r2=1187540&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
 (original)
+++ 
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/thrift/CassandraDaemon.java
 Fri Oct 21 21:09:42 2011
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.service.AbstractCassandraDaemon;
@@ -191,9 +192,9 @@ public class CassandraDaemon extends org
                     // This is NIO selector service but the invocation will be 
Multi-Threaded with the Executor service.
                     ExecutorService executorService = new 
JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getRpcMinThreads(),
                                                                                
        DatabaseDescriptor.getRpcMaxThreads(),
-                                                                               
        DatabaseDescriptor.getRpcTimeout(), 
-                                                                               
        TimeUnit.MILLISECONDS,
-                                                                               
        new LinkedBlockingQueue<Runnable>(), 
+                                                                               
        60L, 
+                                                                               
        TimeUnit.SECONDS,
+                                                                               
        new SynchronousQueue<Runnable>(), 
                                                                                
        new NamedThreadFactory("RPC-Thread"), "RPC-THREAD-POOL");
                     TNonblockingServer.Args serverArgs = new 
TNonblockingServer.Args(serverTransport).inputTransportFactory(inTransportFactory)
                                                                                
        .outputTransportFactory(outTransportFactory)


Reply via email to