You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by sh...@apache.org on 2010/02/26 02:37:57 UTC
svn commit: r916529 - in /hadoop/common/trunk: CHANGES.txt
src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
src/java/org/apache/hadoop/ipc/Server.java
Author: shv
Date: Fri Feb 26 01:37:57 2010
New Revision: 916529
URL: http://svn.apache.org/viewvc?rev=916529&view=rev
Log:
HADOOP-1849. Add undocumented configuration parameter for per handler call queue size in IPC Server. Contributed by Konstantin Shvachko.
Modified:
hadoop/common/trunk/CHANGES.txt
hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java
Modified: hadoop/common/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/CHANGES.txt?rev=916529&r1=916528&r2=916529&view=diff
==============================================================================
--- hadoop/common/trunk/CHANGES.txt (original)
+++ hadoop/common/trunk/CHANGES.txt Fri Feb 26 01:37:57 2010
@@ -1392,14 +1392,6 @@
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 @@
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
Modified: hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java?rev=916529&r1=916528&r2=916529&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java Fri Feb 26 01:37:57 2010
@@ -123,6 +123,15 @@
"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";
Modified: hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java
URL: http://svn.apache.org/viewvc/hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java?rev=916529&r1=916528&r2=916529&view=diff
==============================================================================
--- hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java (original)
+++ hadoop/common/trunk/src/java/org/apache/hadoop/ipc/Server.java Fri Feb 26 01:37:57 2010
@@ -60,8 +60,6 @@
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 @@
// 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 @@
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<Call>(maxQueueSize);
this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);