浏览代码

commit bf988b631fc79efe539d6d8897d902e19ef10a67
Author: Suresh Srinivas <sureshms@yahoo-inc.com>
Date: Mon Feb 22 14:35:53 2010 -0800

Cherry-pick merge of commit bcffa8467c55956cef0ae9c06a08de2a153ac958 (HADOOP:6577 from https://issues.apache.org/jira/secure/attachment/12436399/hadoop-6577.2.rel20.patch) from yahoo-hadoop-0.20 into yahoo-hadoop-0.20.1xx

+++ b/YAHOO-CHANGES.txt
+ HADOOP-6577. Add hidden configuration option "ipc.server.max.response.size"
+ to change the default 1 MB, the maximum size when large IPC handler
+ response buffer is reset. (suresh)
+


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077197 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 年之前
父节点
当前提交
52e2b101be

+ 11 - 6
src/core/org/apache/hadoop/ipc/Server.java

@@ -104,8 +104,10 @@ public abstract class Server {
    * Initial and max size of response buffer
    */
   static int INITIAL_RESP_BUF_SIZE = 10240;
-  static int MAX_RESP_BUF_SIZE = 1024*1024;
-    
+  static final String IPC_SERVER_RPC_MAX_RESPONSE_SIZE_KEY = 
+                        "ipc.server.max.response.size";
+  static final int IPC_SERVER_RPC_MAX_RESPONSE_SIZE_DEFAULT = 1024*1024;
+  
   public static final Log LOG = LogFactory.getLog(Server.class);
 
   private static final ThreadLocal<Server> SERVER = new ThreadLocal<Server>();
@@ -174,6 +176,7 @@ public abstract class Server {
   private SecretManager<TokenIdentifier> secretManager;
 
   private int maxQueueSize;
+  private final int maxRespSize;
   private int socketSendBufferSize;
   private final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
 
@@ -1201,10 +1204,10 @@ public abstract class Server {
             setupResponse(buf, call, 
                         (error == null) ? Status.SUCCESS : Status.ERROR, 
                         value, errorClass, error);
-            // Discard the large buf and reset it back to 
-            // smaller size to freeup heap
-            if (buf.size() > MAX_RESP_BUF_SIZE) {
-              LOG.warn("Large response size " + buf.size() + " for call " + 
+          // Discard the large buf and reset it back to 
+          // smaller size to freeup heap
+          if (buf.size() > maxRespSize) {
+            LOG.warn("Large response size " + buf.size() + " for call " + 
                 call.toString());
               buf = new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
             }
@@ -1249,6 +1252,8 @@ public abstract class Server {
     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.callQueue  = new LinkedBlockingQueue<Call>(maxQueueSize); 
     this.maxIdleTime = 2*conf.getInt("ipc.client.connection.maxidletime", 1000);
     this.maxConnectionsToNuke = conf.getInt("ipc.client.kill.max", 10);

+ 3 - 1
src/test/org/apache/hadoop/ipc/TestIPCServerResponder.java

@@ -116,8 +116,10 @@ public class TestIPCServerResponder extends TestCase {
 
   public void testResponseBuffer() throws Exception {
     Server.INITIAL_RESP_BUF_SIZE = 1;
-    Server.MAX_RESP_BUF_SIZE = 1;
+    conf.setInt(Server.IPC_SERVER_RPC_MAX_RESPONSE_SIZE_KEY,
+                1);
     testServerResponder(1, true, 1, 1, 5);
+    conf = new Configuration(); // reset configuration
   }
 
   public void testServerResponder() throws Exception {