Author: shv Date: Fri Feb 26 01:46:19 2010 New Revision: 916531 URL: http://svn.apache.org/viewvc?rev=916531&view=rev Log: HADOOP-1849. Merge -r 916528:916529 from trunk to branch-0.20.
Modified: hadoop/common/branches/branch-0.20/CHANGES.txt hadoop/common/branches/branch-0.20/src/core/org/apache/hadoop/ipc/Server.java Modified: hadoop/common/branches/branch-0.20/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/CHANGES.txt?rev=916531&r1=916530&r2=916531&view=diff ============================================================================== --- hadoop/common/branches/branch-0.20/CHANGES.txt (original) +++ hadoop/common/branches/branch-0.20/CHANGES.txt Fri Feb 26 01:46:19 2010 @@ -126,6 +126,9 @@ 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/branches/branch-0.20/src/core/org/apache/hadoop/ipc/Server.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.20/src/core/org/apache/hadoop/ipc/Server.java?rev=916531&r1=916530&r2=916531&view=diff ============================================================================== --- hadoop/common/branches/branch-0.20/src/core/org/apache/hadoop/ipc/Server.java (original) +++ hadoop/common/branches/branch-0.20/src/core/org/apache/hadoop/ipc/Server.java Fri Feb 26 01:46:19 2010 @@ -88,7 +88,9 @@ /** * How many calls/handler are allowed in the queue. */ - private static final int MAX_QUEUE_SIZE_PER_HANDLER = 100; + private static final int IPC_SERVER_HANDLER_QUEUE_SIZE_DEFAULT = 100; + private static final String IPC_SERVER_HANDLER_QUEUE_SIZE_KEY = + "ipc.server.handler.queue.size"; public static final Log LOG = LogFactory.getLog(Server.class); @@ -1016,7 +1018,9 @@ this.paramClass = paramClass; this.handlerCount = handlerCount; this.socketSendBufferSize = 0; - this.maxQueueSize = handlerCount * MAX_QUEUE_SIZE_PER_HANDLER; + this.maxQueueSize = handlerCount * conf.getInt( + IPC_SERVER_HANDLER_QUEUE_SIZE_KEY, + IPC_SERVER_HANDLER_QUEUE_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);