Author: shv Date: Fri Feb 26 01:45:38 2010 New Revision: 916530 URL: http://svn.apache.org/viewvc?rev=916530&view=rev Log: HADOOP-1849. Merge -r 916528:916529 from trunk to branch-0.21.
Modified: hadoop/common/branches/branch-0.21/CHANGES.txt hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/ipc/Server.java Modified: hadoop/common/branches/branch-0.21/CHANGES.txt URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/CHANGES.txt?rev=916530&r1=916529&r2=916530&view=diff ============================================================================== --- hadoop/common/branches/branch-0.21/CHANGES.txt (original) +++ hadoop/common/branches/branch-0.21/CHANGES.txt Fri Feb 26 01:45:38 2010 @@ -1185,6 +1185,11 @@ HADOOP-6498. IPC client bug may cause rpc call hang. (Ruyue Ma and hairong via hairong) + IMPROVEMENTS + + 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.21/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java?rev=916530&r1=916529&r2=916530&view=diff ============================================================================== --- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java (original) +++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/fs/CommonConfigurationKeys.java Fri Feb 26 01:45:38 2010 @@ -119,6 +119,16 @@ public static final int IPC_CLIENT_IDLETHRESHOLD_DEFAULT = 4000; public static final String IPC_SERVER_TCPNODELAY_KEY = "ipc.server.tcpnodelay"; public static final boolean IPC_SERVER_TCPNODELAY_DEFAULT = false; + /** + * 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/branches/branch-0.21/src/java/org/apache/hadoop/ipc/Server.java URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/ipc/Server.java?rev=916530&r1=916529&r2=916530&view=diff ============================================================================== --- hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/ipc/Server.java (original) +++ hadoop/common/branches/branch-0.21/src/java/org/apache/hadoop/ipc/Server.java Fri Feb 26 01:45:38 2010 @@ -60,6 +60,7 @@ 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 org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; @@ -84,12 +85,7 @@ // 1 : Introduce ping and server does not throw away RPCs // 3 : Introduce the protocol into the RPC connection header public static final byte CURRENT_VERSION = 3; - - /** - * 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 */ @@ -1034,7 +1030,9 @@ this.paramClass = paramClass; this.handlerCount = handlerCount; this.socketSendBufferSize = 0; - this.maxQueueSize = handlerCount * MAX_QUEUE_SIZE_PER_HANDLER; + this.maxQueueSize = handlerCount * conf.getInt( + CommonConfigurationKeys.IPC_SERVER_HANDLER_QUEUE_SIZE_KEY, + CommonConfigurationKeys.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);