[NO ISSUE][CONF] Add Configuration For Max Http Request Size
- user model changes: no
- storage format changes: no
- interface changes: no
Details:
- Add configuration for max http request size and default
it to 50 MB.
- Close client connection if request exceeds max size.
- Add test case for large request.
Change-Id: I11153490fc022d3a94b5b6f7dc3e20204fb105a0
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2826
Reviewed-by: Murtadha Hubail <mhubail@apache.org>
Sonar-Qube: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <bamousaa@gmail.com>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index f4d24e2..5c05b2f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -94,6 +94,8 @@
import org.apache.hyracks.control.common.controllers.CCConfig;
import org.apache.hyracks.http.api.IServlet;
import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfig;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.WebManager;
import org.apache.hyracks.util.LoggingConfigUtil;
import org.apache.logging.log4j.Level;
@@ -217,8 +219,10 @@
}
protected HttpServer setupWebServer(ExternalProperties externalProperties) throws Exception {
+ final HttpServerConfig config =
+ HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
HttpServer webServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
- externalProperties.getWebInterfacePort());
+ externalProperties.getWebInterfacePort(), config);
webServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
webServer.addServlet(new ApiServlet(webServer.ctx(), new String[] { "/*" }, appCtx,
ccExtensionManager.getCompilationProvider(AQL), ccExtensionManager.getCompilationProvider(SQLPP),
@@ -227,8 +231,10 @@
}
protected HttpServer setupJSONAPIServer(ExternalProperties externalProperties) throws Exception {
- HttpServer jsonAPIServer =
- new HttpServer(webManager.getBosses(), webManager.getWorkers(), externalProperties.getAPIServerPort());
+ final HttpServerConfig config =
+ HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
+ HttpServer jsonAPIServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
+ externalProperties.getAPIServerPort(), config);
jsonAPIServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
jsonAPIServer.setAttribute(ASTERIX_APP_CONTEXT_INFO_ATTR, appCtx);
jsonAPIServer.setAttribute(ServletConstants.EXECUTOR_SERVICE_ATTR,
@@ -259,8 +265,10 @@
}
protected HttpServer setupQueryWebServer(ExternalProperties externalProperties) throws Exception {
+ final HttpServerConfig config =
+ HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
HttpServer queryWebServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
- externalProperties.getQueryWebInterfacePort());
+ externalProperties.getQueryWebInterfacePort(), config);
queryWebServer.setAttribute(HYRACKS_CONNECTION_ATTR, hcc);
ServiceLoader.load(IQueryWebServerRegistrant.class).iterator()
.forEachRemaining(c -> c.register(appCtx, queryWebServer));
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index b7b7c39..019f54d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -68,6 +68,8 @@
import org.apache.hyracks.control.nc.BaseNCApplication;
import org.apache.hyracks.control.nc.NodeControllerService;
import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfig;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.WebManager;
import org.apache.hyracks.util.LoggingConfigUtil;
import org.apache.logging.log4j.Level;
@@ -160,8 +162,11 @@
}
protected void configureServers() throws Exception {
+ final ExternalProperties externalProperties = getApplicationContext().getExternalProperties();
+ final HttpServerConfig config =
+ HttpServerConfigBuilder.custom().setMaxRequestSize(externalProperties.getMaxWebRequestSize()).build();
HttpServer apiServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(),
- getApplicationContext().getExternalProperties().getNcApiPort());
+ externalProperties.getNcApiPort(), config);
apiServer.setAttribute(ServletConstants.SERVICE_CONTEXT_ATTR, ncServiceCtx);
apiServer.addServlet(new StorageApiServlet(apiServer.ctx(), getApplicationContext(), Servlets.STORAGE));
webManager.add(apiServer);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
index cc4c573..572323a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
@@ -15,6 +15,7 @@
"default\.dir" : "target/io/dir/asterixdb",
"log\.level" : "INFO",
"max\.wait\.active\.cluster" : 60,
+ "max.web.request.size" : 52428800,
"messaging\.frame\.count" : 512,
"messaging\.frame\.size" : 4096,
"metadata\.callback\.port" : 0,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
index dd7dfb7..622f8fd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
@@ -15,6 +15,7 @@
"default\.dir" : "target/io/dir/asterixdb",
"log\.level" : "WARN",
"max\.wait\.active\.cluster" : 60,
+ "max.web.request.size" : 52428800,
"messaging\.frame\.count" : 512,
"messaging\.frame\.size" : 4096,
"metadata\.callback\.port" : 0,
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
index 71200c1..9ade575 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
@@ -15,6 +15,7 @@
"default\.dir" : "target/io/dir/asterixdb",
"log\.level" : "WARN",
"max\.wait\.active\.cluster" : 60,
+ "max.web.request.size" : 52428800,
"messaging\.frame\.count" : 512,
"messaging\.frame\.size" : 4096,
"metadata\.callback\.port" : 0,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
index 4bc1953..714bc92 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ExternalProperties.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.api.config.IOption;
import org.apache.hyracks.api.config.IOptionType;
import org.apache.hyracks.api.config.Section;
+import org.apache.hyracks.util.StorageUtil;
import org.apache.logging.log4j.Level;
public class ExternalProperties extends AbstractProperties {
@@ -42,7 +43,11 @@
"The max pending time (in seconds) for cluster startup. After the "
+ "threshold, if the cluster still is not up and running, it is considered unavailable"),
CC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the cluster controller process by managix"),
- NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix");
+ NC_JAVA_OPTS(STRING, "-Xmx1024m", "The JVM options passed to the node controller process(es) by managix"),
+ MAX_WEB_REQUEST_SIZE(
+ INTEGER,
+ StorageUtil.getIntSizeInBytes(50, StorageUtil.StorageUnit.MEGABYTE),
+ "The maximum accepted web request size in bytes");
private final IOptionType type;
private final Object defaultValue;
@@ -66,6 +71,7 @@
return Section.NC;
case LOG_LEVEL:
case MAX_WAIT_ACTIVE_CLUSTER:
+ case MAX_WEB_REQUEST_SIZE:
return Section.COMMON;
case CC_JAVA_OPTS:
case NC_JAVA_OPTS:
@@ -130,4 +136,8 @@
public int getNcApiPort() {
return accessor.getInt(Option.NC_API_PORT);
}
+
+ public int getMaxWebRequestSize() {
+ return accessor.getInt(Option.MAX_WEB_REQUEST_SIZE);
+ }
}
diff --git a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
index 6db7d6e..49f17a6 100644
--- a/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
+++ b/asterixdb/asterix-common/src/test/java/org/apache/asterix/test/server/RSSTestServer.java
@@ -19,6 +19,7 @@
package org.apache.asterix.test.server;
import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.WebManager;
public class RSSTestServer implements ITestServer {
@@ -27,7 +28,8 @@
public RSSTestServer(int port) {
webManager = new WebManager();
- HttpServer rssServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), port);
+ HttpServer rssServer = new HttpServer(webManager.getBosses(), webManager.getWorkers(), port,
+ HttpServerConfigBuilder.createDefault());
rssServer.addServlet(new RSSFeedServlet(null, new String[] { "/" }));
webManager.add(rssServer);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
index f413fe5..2938bd2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/web/WebServer.java
@@ -25,6 +25,7 @@
import org.apache.hyracks.control.cc.web.util.IJSONOutputFunction;
import org.apache.hyracks.control.cc.web.util.JSONOutputRequestHandler;
import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.StaticResourceServlet;
import org.apache.hyracks.http.server.WebManager;
@@ -40,7 +41,8 @@
listeningPort = port;
ctx = new ConcurrentHashMap<>();
webMgr = new WebManager();
- server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), listeningPort);
+ server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), listeningPort,
+ HttpServerConfigBuilder.createDefault());
addHandlers();
webMgr.add(server);
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestAggregator.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestAggregator.java
new file mode 100644
index 0000000..6e93523
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpRequestAggregator.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import java.util.List;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import io.netty.buffer.Unpooled;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelFutureListener;
+import io.netty.channel.ChannelHandlerContext;
+import io.netty.handler.codec.http.DefaultFullHttpResponse;
+import io.netty.handler.codec.http.FullHttpResponse;
+import io.netty.handler.codec.http.HttpMessage;
+import io.netty.handler.codec.http.HttpObject;
+import io.netty.handler.codec.http.HttpObjectAggregator;
+import io.netty.handler.codec.http.HttpRequest;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.netty.handler.codec.http.HttpVersion;
+
+@SuppressWarnings("squid:MaximumInheritanceDepth")
+public class HttpRequestAggregator extends HttpObjectAggregator {
+
+ private static final Logger LOGGER = LogManager.getLogger();
+ private static final FullHttpResponse TOO_LARGE_CLOSE = new DefaultFullHttpResponse(HttpVersion.HTTP_1_1,
+ HttpResponseStatus.REQUEST_ENTITY_TOO_LARGE, Unpooled.EMPTY_BUFFER);
+ private boolean failed = false;
+
+ public HttpRequestAggregator(int maxContentLength) {
+ super(maxContentLength);
+ }
+
+ public HttpRequestAggregator(int maxContentLength, boolean closeOnExpectationFailed) {
+ super(maxContentLength, closeOnExpectationFailed);
+ }
+
+ @Override
+ protected void decode(final ChannelHandlerContext ctx, HttpObject msg, List<Object> out) throws Exception {
+ if (!failed) {
+ super.decode(ctx, msg, out);
+ }
+ }
+
+ @Override
+ protected void handleOversizedMessage(final ChannelHandlerContext ctx, HttpMessage oversized) throws Exception {
+ failed = true;
+ LOGGER.warn("A large request encountered. Closing the channel");
+ if (oversized instanceof HttpRequest) {
+ // send back a 413 and close the connection
+ final ChannelFuture future = ctx.writeAndFlush(TOO_LARGE_CLOSE.retainedDuplicate());
+ future.addListener((ChannelFutureListener) channelFuture -> {
+ if (!channelFuture.isSuccess()) {
+ LOGGER.debug("Failed to send a 413 Request Entity Too Large.", channelFuture.cause());
+ }
+ ctx.close();
+ });
+ } else {
+ throw new IllegalStateException("Unknown large message of class " + oversized.getClass());
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
index bbe64ae..42b47fb 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServer.java
@@ -57,8 +57,6 @@
protected static final WriteBufferWaterMark WRITE_BUFFER_WATER_MARK =
new WriteBufferWaterMark(LOW_WRITE_BUFFER_WATER_MARK, HIGH_WRITE_BUFFER_WATER_MARK);
protected static final int RECEIVE_BUFFER_SIZE = 4096;
- protected static final int DEFAULT_NUM_EXECUTOR_THREADS = 16;
- protected static final int DEFAULT_REQUEST_QUEUE_SIZE = 256;
private static final Logger LOGGER = LogManager.getLogger();
private static final int FAILED = -1;
private static final int STOPPED = 0;
@@ -81,37 +79,30 @@
private volatile Thread recoveryThread;
private volatile Channel channel;
private Throwable cause;
+ private HttpServerConfig config;
- public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port) {
- this(bossGroup, workerGroup, port, DEFAULT_NUM_EXECUTOR_THREADS, DEFAULT_REQUEST_QUEUE_SIZE, null);
+ public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, HttpServerConfig config) {
+ this(bossGroup, workerGroup, port, config, null);
}
- public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port,
+ public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, HttpServerConfig config,
IChannelClosedHandler closeHandler) {
- this(bossGroup, workerGroup, port, DEFAULT_NUM_EXECUTOR_THREADS, DEFAULT_REQUEST_QUEUE_SIZE, closeHandler);
- }
-
- public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, int numExecutorThreads,
- int requestQueueSize) {
- this(bossGroup, workerGroup, port, numExecutorThreads, requestQueueSize, null);
- }
-
- public HttpServer(EventLoopGroup bossGroup, EventLoopGroup workerGroup, int port, int numExecutorThreads,
- int requestQueueSize, IChannelClosedHandler closeHandler) {
this.bossGroup = bossGroup;
this.workerGroup = workerGroup;
this.port = port;
this.closedHandler = closeHandler;
+ this.config = config;
ctx = new ConcurrentHashMap<>();
servlets = new ArrayList<>();
- workQueue = new LinkedBlockingQueue<>(requestQueueSize);
+ workQueue = new LinkedBlockingQueue<>(config.getRequestQueueSize());
+ int numExecutorThreads = config.getThreadCount();
executor = new ThreadPoolExecutor(numExecutorThreads, numExecutorThreads, 0L, TimeUnit.MILLISECONDS, workQueue,
runnable -> new Thread(runnable, "HttpExecutor(port:" + port + ")-" + threadId.getAndIncrement()));
long directMemoryBudget = numExecutorThreads * (long) HIGH_WRITE_BUFFER_WATER_MARK
- + numExecutorThreads * HttpServerInitializer.RESPONSE_CHUNK_SIZE;
+ + numExecutorThreads * config.getMaxResponseChunkSize();
LOGGER.log(Level.INFO, "The output direct memory budget for this server is " + directMemoryBudget + " bytes");
long inputBudgetEstimate =
- (long) HttpServerInitializer.MAX_REQUEST_INITIAL_LINE_LENGTH * (requestQueueSize + numExecutorThreads);
+ (long) config.getMaxRequestInitialLineLength() * (config.getRequestQueueSize() + numExecutorThreads);
inputBudgetEstimate = inputBudgetEstimate * 2;
LOGGER.log(Level.INFO,
"The \"estimated\" input direct memory budget for this server is " + inputBudgetEstimate + " bytes");
@@ -408,4 +399,8 @@
return "{\"class\":\"" + getClass().getSimpleName() + "\",\"port\":" + port + ",\"state\":\"" + getState()
+ "\"}";
}
+
+ public HttpServerConfig getConfig() {
+ return config;
+ }
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfig.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfig.java
new file mode 100644
index 0000000..443cc3c
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfig.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import org.apache.hyracks.util.annotations.ThreadSafe;
+
+@ThreadSafe
+public class HttpServerConfig {
+
+ private int maxRequestSize;
+ private int threadCount;
+ private int requestQueueSize;
+ private int maxRequestChunkSize;
+ private int maxResponseChunkSize;
+ private int maxRequestHeaderSize;
+ private int maxRequestInitialLineLength;
+
+ private HttpServerConfig() {
+ }
+
+ public static HttpServerConfig of(int threadCount, int requestQueueSize, int maxRequestSize,
+ int maxRequestInitialLineLength, int maxRequestHeaderSize, int maxRequestChunkSize,
+ int maxResponseChunkSize) {
+ final HttpServerConfig config = new HttpServerConfig();
+ config.threadCount = threadCount;
+ config.requestQueueSize = requestQueueSize;
+ config.maxRequestSize = maxRequestSize;
+ config.maxRequestInitialLineLength = maxRequestInitialLineLength;
+ config.maxRequestHeaderSize = maxRequestHeaderSize;
+ config.maxRequestChunkSize = maxRequestChunkSize;
+ config.maxResponseChunkSize = maxResponseChunkSize;
+ return config;
+ }
+
+ public int getMaxRequestSize() {
+ return maxRequestSize;
+ }
+
+ public int getThreadCount() {
+ return threadCount;
+ }
+
+ public int getRequestQueueSize() {
+ return requestQueueSize;
+ }
+
+ public int getMaxRequestChunkSize() {
+ return maxRequestChunkSize;
+ }
+
+ public int getMaxResponseChunkSize() {
+ return maxResponseChunkSize;
+ }
+
+ public int getMaxRequestHeaderSize() {
+ return maxRequestHeaderSize;
+ }
+
+ public int getMaxRequestInitialLineLength() {
+ return maxRequestInitialLineLength;
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfigBuilder.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfigBuilder.java
new file mode 100644
index 0000000..359d760
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerConfigBuilder.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.http.server;
+
+import org.apache.hyracks.util.annotations.NotThreadSafe;
+
+@NotThreadSafe
+public class HttpServerConfigBuilder {
+
+ private static final int MAX_REQUEST_CHUNK_SIZE = 262144;
+ private static final int MAX_REQUEST_HEADER_SIZE = 262144;
+ private static final int MAX_REQUEST_INITIAL_LINE_LENGTH = 131072;
+ private static final int RESPONSE_CHUNK_SIZE = 4096;
+ private static final int DEFAULT_THREAD_COUNT = 16;
+ private static final int DEFAULT_MAX_QUEUE_SIZE = 256;
+
+ private int maxRequestSize = Integer.MAX_VALUE;
+ private int threadCount = DEFAULT_THREAD_COUNT;
+ private int requestQueueSize = DEFAULT_MAX_QUEUE_SIZE;
+ private int maxRequestChunkSize = MAX_REQUEST_CHUNK_SIZE;
+ private int maxResponseChunkSize = RESPONSE_CHUNK_SIZE;
+ private int maxRequestHeaderSize = MAX_REQUEST_HEADER_SIZE;
+ private int maxRequestInitialLineLength = MAX_REQUEST_INITIAL_LINE_LENGTH;
+
+ private HttpServerConfigBuilder() {
+ }
+
+ public static HttpServerConfig createDefault() {
+ return new HttpServerConfigBuilder().build();
+ }
+
+ public static HttpServerConfigBuilder custom() {
+ return new HttpServerConfigBuilder();
+ }
+
+ public HttpServerConfigBuilder setMaxRequestSize(int maxRequestSize) {
+ this.maxRequestSize = maxRequestSize;
+ return this;
+ }
+
+ public HttpServerConfigBuilder setThreadCount(int threadCount) {
+ this.threadCount = threadCount;
+ return this;
+ }
+
+ public HttpServerConfigBuilder setRequestQueueSize(int requestQueueSize) {
+ this.requestQueueSize = requestQueueSize;
+ return this;
+ }
+
+ public HttpServerConfigBuilder setMaxRequestChunkSize(int maxRequestChunkSize) {
+ this.maxRequestChunkSize = maxRequestChunkSize;
+ return this;
+ }
+
+ public HttpServerConfigBuilder setMaxResponseChunkSize(int maxResponseChunkSize) {
+ this.maxResponseChunkSize = maxResponseChunkSize;
+ return this;
+ }
+
+ public HttpServerConfigBuilder setMaxRequestHeaderSize(int maxRequestHeaderSize) {
+ this.maxRequestHeaderSize = maxRequestHeaderSize;
+ return this;
+ }
+
+ public HttpServerConfigBuilder setMaxRequestInitialLineLength(int maxRequestInitialLineLength) {
+ this.maxRequestInitialLineLength = maxRequestInitialLineLength;
+ return this;
+ }
+
+ public HttpServerConfig build() {
+ return HttpServerConfig.of(threadCount, requestQueueSize, maxRequestSize, maxRequestInitialLineLength,
+ maxRequestHeaderSize, maxRequestChunkSize, maxResponseChunkSize);
+ }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
index bc173fd..e10da64 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/HttpServerInitializer.java
@@ -21,30 +21,35 @@
import io.netty.channel.ChannelInitializer;
import io.netty.channel.ChannelPipeline;
import io.netty.channel.socket.SocketChannel;
-import io.netty.handler.codec.http.HttpObjectAggregator;
import io.netty.handler.codec.http.HttpRequestDecoder;
import io.netty.handler.codec.http.HttpResponseEncoder;
public class HttpServerInitializer extends ChannelInitializer<SocketChannel> {
- public static final int MAX_REQUEST_CHUNK_SIZE = 262144;
- public static final int MAX_REQUEST_HEADER_SIZE = 262144;
- public static final int MAX_REQUEST_INITIAL_LINE_LENGTH = 131072;
- public static final int RESPONSE_CHUNK_SIZE = 4096;
private final HttpServer server;
+ private final int maxRequestSize;
+ private final int maxRequestInitialLineLength;
+ private final int maxRequestHeaderSize;
+ private final int maxRequestChunkSize;
+ private final int maxResponseChunkSize;
public HttpServerInitializer(HttpServer server) {
this.server = server;
+ final HttpServerConfig config = server.getConfig();
+ maxRequestSize = config.getMaxRequestSize();
+ maxRequestInitialLineLength = config.getMaxRequestInitialLineLength();
+ maxRequestHeaderSize = config.getMaxRequestHeaderSize();
+ maxRequestChunkSize = config.getMaxRequestChunkSize();
+ maxResponseChunkSize = config.getMaxResponseChunkSize();
}
@Override
public void initChannel(SocketChannel ch) {
ChannelPipeline p = ch.pipeline();
p.addLast(new HttpRequestCapacityController(server));
- p.addLast(new HttpRequestDecoder(MAX_REQUEST_INITIAL_LINE_LENGTH, MAX_REQUEST_HEADER_SIZE,
- MAX_REQUEST_CHUNK_SIZE));
+ p.addLast(new HttpRequestDecoder(maxRequestInitialLineLength, maxRequestHeaderSize, maxRequestChunkSize));
p.addLast(new HttpResponseEncoder());
- p.addLast(new HttpObjectAggregator(Integer.MAX_VALUE));
- p.addLast(server.createHttpHandler(RESPONSE_CHUNK_SIZE));
+ p.addLast(new HttpRequestAggregator(maxRequestSize));
+ p.addLast(server.createHttpHandler(maxResponseChunkSize));
}
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java
index 17f6f9a..2ab0229 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpRequestTask.java
@@ -41,8 +41,8 @@
protected final HttpUriRequest request;
- protected HttpRequestTask() throws URISyntaxException {
- request = post(null);
+ protected HttpRequestTask(int size) throws URISyntaxException {
+ request = post(null, size);
}
@Override
@@ -90,7 +90,7 @@
return builder.build();
}
- protected HttpUriRequest post(String query) throws URISyntaxException {
+ protected HttpUriRequest post(String query, int size) throws URISyntaxException {
URI uri = new URI(HttpServerTest.PROTOCOL, null, HttpServerTest.HOST, HttpServerTest.PORT, HttpServerTest.PATH,
query, null);
RequestBuilder builder = RequestBuilder.post(uri);
@@ -102,7 +102,11 @@
String statement = str.toString();
builder.setHeader("Content-type", "application/x-www-form-urlencoded");
builder.addParameter("statement", statement);
- builder.setEntity(new StringEntity(statement, StandardCharsets.UTF_8));
+ for (int i = 0; i < size; i++) {
+ str.append("This is a string statement that will be ignored");
+ str.append('\n');
+ }
+ builder.setEntity(new StringEntity(str.toString(), StandardCharsets.UTF_8));
builder.setCharset(StandardCharsets.UTF_8);
return builder.build();
}
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
index b39a141..8867bb5 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/test/java/org/apache/hyracks/http/test/HttpServerTest.java
@@ -34,6 +34,8 @@
import org.apache.hyracks.http.HttpTestUtil;
import org.apache.hyracks.http.server.HttpServer;
+import org.apache.hyracks.http.server.HttpServerConfig;
+import org.apache.hyracks.http.server.HttpServerConfigBuilder;
import org.apache.hyracks.http.server.InterruptOnCloseHandler;
import org.apache.hyracks.http.server.WebManager;
import org.apache.hyracks.http.servlet.ChattyServlet;
@@ -79,8 +81,9 @@
int numExecutors = 16;
int serverQueueSize = 16;
int numRequests = 128;
- HttpServer server =
- new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+ final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+ .setRequestQueueSize(serverQueueSize).build();
+ HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
server.addServlet(servlet);
webMgr.add(server);
@@ -135,8 +138,9 @@
int numExecutors = 2;
int serverQueueSize = 2;
int numPatches = 60;
- HttpServer server =
- new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+ final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+ .setRequestQueueSize(serverQueueSize).build();
+ HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
server.addServlet(servlet);
webMgr.add(server);
@@ -182,8 +186,9 @@
int serverQueueSize = 24;
HttpTestUtil.printMemUsage();
WebManager webMgr = new WebManager();
- HttpServer server =
- new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+ final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+ .setRequestQueueSize(serverQueueSize).build();
+ HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
ChattyServlet servlet = new ChattyServlet(server.ctx(), new String[] { PATH });
server.addServlet(servlet);
webMgr.add(server);
@@ -208,8 +213,9 @@
int numExecutors = 16;
int serverQueueSize = 16;
WebManager webMgr = new WebManager();
- HttpServer server =
- new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+ final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+ .setRequestQueueSize(serverQueueSize).build();
+ HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
server.addServlet(servlet);
webMgr.add(server);
@@ -245,8 +251,9 @@
int serverQueueSize = 16;
int numRequests = 1;
WebManager webMgr = new WebManager();
- HttpServer server =
- new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, serverQueueSize);
+ final HttpServerConfig config = HttpServerConfigBuilder.custom().setThreadCount(numExecutors)
+ .setRequestQueueSize(serverQueueSize).build();
+ HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
ChattyServlet servlet = new ChattyServlet(server.ctx(), new String[] { PATH });
server.addServlet(servlet);
webMgr.add(server);
@@ -301,8 +308,10 @@
WebManager webMgr = new WebManager();
int numExecutors = 1;
int queueSize = 1;
- HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, numExecutors, queueSize,
- InterruptOnCloseHandler.INSTANCE);
+ final HttpServerConfig config =
+ HttpServerConfigBuilder.custom().setThreadCount(numExecutors).setRequestQueueSize(queueSize).build();
+ HttpServer server =
+ new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config, InterruptOnCloseHandler.INSTANCE);
SleepyServlet servlet = new SleepyServlet(server.ctx(), new String[] { PATH });
server.addServlet(servlet);
webMgr.add(server);
@@ -335,6 +344,30 @@
}
}
+ @Test
+ public void testLargeRequest() throws Exception {
+ WebManager webMgr = new WebManager();
+ // Server with max allowed request size = 512K
+ final HttpServerConfig config = HttpServerConfigBuilder.custom().setMaxRequestSize(512 * 1024).build();
+ HttpServer server = new HttpServer(webMgr.getBosses(), webMgr.getWorkers(), PORT, config);
+ ChattyServlet servlet = new ChattyServlet(server.ctx(), new String[] { PATH });
+ server.addServlet(servlet);
+ webMgr.add(server);
+ webMgr.start();
+ Exception failure = null;
+ try {
+ request(1, 32000);
+ for (Future<Void> thread : FUTURES) {
+ thread.get();
+ }
+ } catch (Exception e) {
+ failure = e;
+ } finally {
+ webMgr.stop();
+ }
+ Assert.assertNotNull(failure);
+ }
+
public static void setPrivateField(Object obj, String filedName, Object value) throws Exception {
Field f = obj.getClass().getDeclaredField(filedName);
f.setAccessible(true);
@@ -342,8 +375,12 @@
}
private void request(int count) throws URISyntaxException {
+ request(count, 32);
+ }
+
+ private void request(int count, int size) throws URISyntaxException {
for (int i = 0; i < count; i++) {
- HttpRequestTask requestTask = new HttpRequestTask();
+ HttpRequestTask requestTask = new HttpRequestTask(size);
Future<Void> next = executor.submit(requestTask);
FUTURES.add(next);
TASKS.add(requestTask);