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. Contributed by Suresh Srinivas.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@912056 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d099c1c78a
commit
9871771bb5
|
@ -54,10 +54,14 @@ Trunk (unreleased changes)
|
|||
HADOOP-6510. Adds a way for superusers to impersonate other users
|
||||
in a secure environment. (Jitendra Nath Pandey via ddas)
|
||||
|
||||
HADOOP-6421 Adds Symbolic links to FileContext, AbstractFileSystem.
|
||||
HADOOP-6421. Adds Symbolic links to FileContext, AbstractFileSystem.
|
||||
It also adds a limited implementation for the local file system
|
||||
(RawLocalFs) that allows local symlinks. (Eli Collins via Sanjay Radia)
|
||||
|
||||
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)
|
||||
|
||||
IMPROVEMENTS
|
||||
|
||||
HADOOP-6283. Improve the exception messages thrown by
|
||||
|
|
|
@ -119,6 +119,10 @@ public class CommonConfigurationKeys {
|
|||
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;
|
||||
public static final String IPC_SERVER_RPC_MAX_RESPONSE_SIZE_KEY =
|
||||
"ipc.server.max.response.size";
|
||||
public static final int IPC_SERVER_RPC_MAX_RESPONSE_SIZE_DEFAULT =
|
||||
1024*1024;
|
||||
|
||||
public static final String HADOOP_RPC_SOCKET_FACTORY_CLASS_DEFAULT_KEY =
|
||||
"hadoop.rpc.socket.factory.class.default";
|
||||
|
|
|
@ -59,6 +59,7 @@ import javax.security.sasl.SaslServer;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
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;
|
||||
|
@ -104,8 +105,7 @@ 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;
|
||||
|
||||
|
||||
public static final Log LOG = LogFactory.getLog(Server.class);
|
||||
|
||||
private static final ThreadLocal<Server> SERVER = new ThreadLocal<Server>();
|
||||
|
@ -174,6 +174,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
|
||||
|
||||
|
@ -1207,7 +1208,7 @@ public abstract class Server {
|
|||
: 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) {
|
||||
if (buf.size() > maxRespSize) {
|
||||
LOG.warn("Large response size " + buf.size() + " for call "
|
||||
+ call.toString());
|
||||
buf = new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
|
||||
|
@ -1253,6 +1254,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);
|
||||
|
|
|
@ -27,6 +27,7 @@ import junit.framework.TestCase;
|
|||
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.io.BytesWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
|
@ -116,8 +117,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(CommonConfigurationKeys.IPC_SERVER_RPC_MAX_RESPONSE_SIZE_KEY,
|
||||
1);
|
||||
testServerResponder(1, true, 1, 1, 5);
|
||||
conf = new Configuration(); // reset configuration
|
||||
}
|
||||
|
||||
public void testServerResponder() throws Exception {
|
||||
|
|
Loading…
Reference in New Issue