Somewhat better defaults for the binary protocol

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

Branch: refs/heads/cassandra-1.2
Commit: a0db8ffc53cdaf85fc991807f9173a8c1269796a
Parents: 452f596
Author: Sylvain Lebresne <sylv...@datastax.com>
Authored: Tue Nov 27 11:49:42 2012 +0100
Committer: Sylvain Lebresne <sylv...@datastax.com>
Committed: Thu Nov 29 10:16:33 2012 +0100

----------------------------------------------------------------------
 conf/cassandra.yaml                                |    9 ++++++---
 src/java/org/apache/cassandra/config/Config.java   |    3 ++-
 .../cassandra/config/DatabaseDescriptor.java       |    5 +++++
 .../transport/RequestThreadPoolExecutor.java       |   13 +++++++------
 .../org/apache/cassandra/transport/Server.java     |    2 ++
 .../apache/cassandra/transport/SimpleClient.java   |    2 ++
 6 files changed, 24 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a0db8ffc/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index a79e150..e077efe 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -310,9 +310,12 @@ listen_address: localhost
 start_native_transport: false
 # port for the CQL native transport to listen for clients on
 native_transport_port: 9042
-# The maximum of thread handling requests. The meaning is the same than
-# rpc_max_threads. The default is unlimited.
-#native_transport_max_threads: 2048
+# The minimum and maximum threads for handling requests when the native
+# transport is used. The meaning is those is similar to the one of
+# rpc_min_threads and rpc_max_threads, though the default differ slightly and
+# are the ones below:
+# native_transport_min_threads: 16
+# native_transport_max_threads: 128
 
 
 # Whether to start the thrift rpc server.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a0db8ffc/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java 
b/src/java/org/apache/cassandra/config/Config.java
index c605a3a..7de852d 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -86,7 +86,8 @@ public class Config
 
     public Boolean start_native_transport = false;
     public Integer native_transport_port = 8000;
-    public Integer native_transport_max_threads = Integer.MAX_VALUE;
+    public Integer native_transport_min_threads = 16;
+    public Integer native_transport_max_threads = 128;
 
     public Integer thrift_max_message_length_in_mb = 16;
     public Integer thrift_framed_transport_size_in_mb = 15;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a0db8ffc/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java 
b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index e615887..cb5ade0 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -986,6 +986,11 @@ public class DatabaseDescriptor
         return 
Integer.parseInt(System.getProperty("cassandra.native_transport_port", 
conf.native_transport_port.toString()));
     }
 
+    public static Integer getNativeTransportMinThreads()
+    {
+        return conf.native_transport_min_threads;
+    }
+
     public static Integer getNativeTransportMaxThreads()
     {
         return conf.native_transport_max_threads;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a0db8ffc/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 21ce926..83c7c29 100644
--- a/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
+++ b/src/java/org/apache/cassandra/transport/RequestThreadPoolExecutor.java
@@ -17,22 +17,23 @@
  */
 package org.apache.cassandra.transport;
 
+import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.TimeUnit;
 
-import org.jboss.netty.handler.execution.OrderedMemoryAwareThreadPoolExecutor;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.config.DatabaseDescriptor;
 
-public class RequestThreadPoolExecutor extends 
OrderedMemoryAwareThreadPoolExecutor
+public class RequestThreadPoolExecutor extends DebuggableThreadPoolExecutor
 {
     private final static int CORE_THREAD_TIMEOUT_SEC = 30;
 
     public RequestThreadPoolExecutor()
     {
-        super(DatabaseDescriptor.getNativeTransportMaxThreads(),
-              0, 0,
+        super(DatabaseDescriptor.getNativeTransportMinThreads(),
+              DatabaseDescriptor.getNativeTransportMaxThreads(),
               CORE_THREAD_TIMEOUT_SEC, TimeUnit.SECONDS,
+              new ArrayBlockingQueue(32), // Seems to help smooth latency 
compared to SynchronousQueue.
               new NamedThreadFactory("Native-Transport-Requests"));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a0db8ffc/src/java/org/apache/cassandra/transport/Server.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Server.java 
b/src/java/org/apache/cassandra/transport/Server.java
index e629ca4..c3fabfb 100644
--- a/src/java/org/apache/cassandra/transport/Server.java
+++ b/src/java/org/apache/cassandra/transport/Server.java
@@ -106,6 +106,8 @@ public class Server implements CassandraDaemon.Server
         factory = new 
NioServerSocketChannelFactory(Executors.newCachedThreadPool(), 
Executors.newCachedThreadPool());
         ServerBootstrap bootstrap = new ServerBootstrap(factory);
 
+        bootstrap.setOption("child.tcpNoDelay", true);
+
         // Set up the event pipeline factory.
         bootstrap.setPipelineFactory(new PipelineFactory(this));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a0db8ffc/src/java/org/apache/cassandra/transport/SimpleClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/SimpleClient.java 
b/src/java/org/apache/cassandra/transport/SimpleClient.java
index 7118bc1..580a3a0 100644
--- a/src/java/org/apache/cassandra/transport/SimpleClient.java
+++ b/src/java/org/apache/cassandra/transport/SimpleClient.java
@@ -92,6 +92,8 @@ public class SimpleClient
                             Executors.newCachedThreadPool(),
                             Executors.newCachedThreadPool()));
 
+        bootstrap.setOption("tcpNoDelay", true);
+
         // Configure the pipeline factory.
         bootstrap.setPipelineFactory(new PipelineFactory());
         ChannelFuture future = bootstrap.connect(new InetSocketAddress(host, 
port));

Reply via email to