From ea650d8d6c139ab7d6078c09da5c3cf72f2915fb Mon Sep 17 00:00:00 2001 From: Konstantin Shvachko Date: Fri, 26 Feb 2010 01:37:57 +0000 Subject: [PATCH] HADOOP-1849. Add undocumented configuration parameter for per handler call queue size in IPC Server. Contributed by Konstantin Shvachko. git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@916529 13f79535-47bb-0310-9956-ffa450edef68 --- CHANGES.txt | 19 +++++++++++-------- .../hadoop/fs/CommonConfigurationKeys.java | 9 +++++++++ src/java/org/apache/hadoop/ipc/Server.java | 18 +++++++----------- 3 files changed, 27 insertions(+), 19 deletions(-) diff --git a/CHANGES.txt b/CHANGES.txt index d080253f2d2..b5f8466d4c9 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1392,14 +1392,6 @@ Release 0.20.2 - 2010-2-16 HADOOP-6218. Adds a feature where TFile can be split by Record Sequence number. (Hong Tang and Raghu Angadi via ddas) - IMPROVEMENTS - - HADOOP-5611. Fix C++ libraries to build on Debian Lenny. (Todd Lipcon - via tomwhite) - - HADOOP-5612. Some c++ scripts are not chmodded before ant execution. - (Todd Lipcon via tomwhite) - BUG FIXES HADOOP-6231. Allow caching of filesystem instances to be disabled on a @@ -1424,6 +1416,17 @@ Release 0.20.2 - 2010-2-16 HADOOP-6498. IPC client bug may cause rpc call hang. (Ruyue Ma and hairong via hairong) + IMPROVEMENTS + + HADOOP-5611. Fix C++ libraries to build on Debian Lenny. (Todd Lipcon + via tomwhite) + + HADOOP-5612. Some c++ scripts are not chmodded before ant execution. + (Todd Lipcon via tomwhite) + + HADOOP-1849. Add undocumented configuration parameter for per handler + call queue size in IPC Server. (shv) + Release 0.20.1 - 2009-09-01 INCOMPATIBLE CHANGES diff --git a/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index 0852a7d6f78..8722e405156 100644 --- a/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -123,6 +123,15 @@ public class CommonConfigurationKeys { "ipc.server.max.response.size"; public static final int IPC_SERVER_RPC_MAX_RESPONSE_SIZE_DEFAULT = 1024*1024; + /** + * How many calls per handler are allowed in the queue. + */ + public static final String IPC_SERVER_HANDLER_QUEUE_SIZE_KEY = + "ipc.server.handler.queue.size"; + /** + * The default number of calls per handler in the queue. + */ + public static final int IPC_SERVER_HANDLER_QUEUE_SIZE_DEFAULT = 100; public static final String HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY = "hadoop.rpc.socket.factory.class.default"; diff --git a/src/java/org/apache/hadoop/ipc/Server.java b/src/java/org/apache/hadoop/ipc/Server.java index 1f02db48f1b..999a3802bb2 100644 --- a/src/java/org/apache/hadoop/ipc/Server.java +++ b/src/java/org/apache/hadoop/ipc/Server.java @@ -60,8 +60,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; - -import static org.apache.hadoop.fs.CommonConfigurationKeys.*; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; import org.apache.hadoop.ipc.metrics.RpcMetrics; @@ -98,12 +96,7 @@ public abstract class Server { // 3 : Introduce the protocol into the RPC connection header // 4 : Introduced SASL security layer public static final byte CURRENT_VERSION = 4; - - /** - * How many calls/handler are allowed in the queue. - */ - private static final int MAX_QUEUE_SIZE_PER_HANDLER = 100; - + /** * Initial and max size of response buffer */ @@ -1288,9 +1281,12 @@ public abstract class Server { this.paramClass = paramClass; this.handlerCount = handlerCount; this.socketSendBufferSize = 0; - this.maxQueueSize = handlerCount * MAX_QUEUE_SIZE_PER_HANDLER; - this.maxRespSize = conf.getInt(IPC_SERVER_RPC_MAX_RESPONSE_SIZE_KEY, - IPC_SERVER_RPC_MAX_RESPONSE_SIZE_DEFAULT); + this.maxQueueSize = handlerCount * conf.getInt( + CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_KEY, + CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_DEFAULT); + this.maxRespSize = conf.getInt( + CommonConfigurationKeys.IPC_SERVER_RPC_MAX_RESPONSE_SIZE_KEY, + CommonConfigurationKeys.IPC_SERVER_RPC_MAX_RESPONSE_SIZE_DEFAULT); this.callQueue = new LinkedBlockingQueue(maxQueueSize); this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000); this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);