mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #25296 from abyss7/http-issues
Add settings for HTTP header limitations
This commit is contained in:
commit
04be5437d9
@ -51,7 +51,7 @@ namespace
|
||||
void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
HTMLForm params(request);
|
||||
HTMLForm params(getContext()->getSettingsRef(), request);
|
||||
|
||||
if (!params.has("method"))
|
||||
{
|
||||
|
@ -69,7 +69,7 @@ namespace
|
||||
|
||||
void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
HTMLForm params(request, request.getStream());
|
||||
HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
|
@ -21,7 +21,7 @@ namespace DB
|
||||
{
|
||||
void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
HTMLForm params(request, request.getStream());
|
||||
HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
|
@ -50,7 +50,7 @@ void ODBCHandler::processError(HTTPServerResponse & response, const std::string
|
||||
|
||||
void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
HTMLForm params(request);
|
||||
HTMLForm params(getContext()->getSettingsRef(), request);
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
if (mode == "read")
|
||||
|
@ -29,7 +29,7 @@ namespace
|
||||
|
||||
void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
HTMLForm params(request, request.getStream());
|
||||
HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
|
@ -234,6 +234,9 @@ class IColumn;
|
||||
M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \
|
||||
M(Seconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \
|
||||
M(UInt64, http_max_uri_size, 1048576, "Maximum URI length of HTTP request", 0) \
|
||||
M(UInt64, http_max_fields, 1000000, "Maximum number of fields in HTTP header", 0) \
|
||||
M(UInt64, http_max_field_name_size, 1048576, "Maximum length of field name in HTTP header", 0) \
|
||||
M(UInt64, http_max_field_value_size, 1048576, "Maximum length of field value in HTTP header", 0) \
|
||||
M(Bool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \
|
||||
M(Bool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \
|
||||
M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include <Server/HTTP/HTMLForm.h>
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/EmptyReadBuffer.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <Server/HTTP/ReadHeaders.h>
|
||||
@ -35,38 +36,41 @@ const std::string HTMLForm::ENCODING_MULTIPART = "multipart/form-data";
|
||||
const int HTMLForm::UNKNOWN_CONTENT_LENGTH = -1;
|
||||
|
||||
|
||||
HTMLForm::HTMLForm() : field_limit(DFL_FIELD_LIMIT), value_length_limit(DFL_MAX_VALUE_LENGTH), encoding(ENCODING_URL)
|
||||
HTMLForm::HTMLForm(const Settings & settings)
|
||||
: max_fields_number(settings.http_max_fields)
|
||||
, max_field_name_size(settings.http_max_field_name_size)
|
||||
, max_field_value_size(settings.http_max_field_value_size)
|
||||
, encoding(ENCODING_URL)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
HTMLForm::HTMLForm(const std::string & encoding_)
|
||||
: field_limit(DFL_FIELD_LIMIT), value_length_limit(DFL_MAX_VALUE_LENGTH), encoding(encoding_)
|
||||
HTMLForm::HTMLForm(const Settings & settings, const std::string & encoding_) : HTMLForm(settings)
|
||||
{
|
||||
encoding = encoding_;
|
||||
}
|
||||
|
||||
|
||||
HTMLForm::HTMLForm(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler)
|
||||
: field_limit(DFL_FIELD_LIMIT), value_length_limit(DFL_MAX_VALUE_LENGTH)
|
||||
HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler)
|
||||
: HTMLForm(settings)
|
||||
{
|
||||
load(request, requestBody, handler);
|
||||
}
|
||||
|
||||
|
||||
HTMLForm::HTMLForm(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody)
|
||||
: field_limit(DFL_FIELD_LIMIT), value_length_limit(DFL_MAX_VALUE_LENGTH)
|
||||
HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody) : HTMLForm(settings)
|
||||
{
|
||||
load(request, requestBody);
|
||||
}
|
||||
|
||||
|
||||
HTMLForm::HTMLForm(const Poco::Net::HTTPRequest & request) : HTMLForm(Poco::URI(request.getURI()))
|
||||
HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request) : HTMLForm(settings, Poco::URI(request.getURI()))
|
||||
{
|
||||
}
|
||||
|
||||
HTMLForm::HTMLForm(const Poco::URI & uri) : field_limit(DFL_FIELD_LIMIT), value_length_limit(DFL_MAX_VALUE_LENGTH)
|
||||
HTMLForm::HTMLForm(const Settings & settings, const Poco::URI & uri) : HTMLForm(settings)
|
||||
{
|
||||
ReadBufferFromString istr(uri.getRawQuery()); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
ReadBufferFromString istr(uri.getRawQuery()); // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
readQuery(istr);
|
||||
}
|
||||
|
||||
@ -123,7 +127,7 @@ void HTMLForm::readQuery(ReadBuffer & in)
|
||||
|
||||
while (true)
|
||||
{
|
||||
if (field_limit > 0 && fields == field_limit)
|
||||
if (max_fields_number > 0 && fields == max_fields_number)
|
||||
throw Poco::Net::HTMLFormException("Too many form fields");
|
||||
|
||||
std::string name;
|
||||
@ -133,7 +137,7 @@ void HTMLForm::readQuery(ReadBuffer & in)
|
||||
{
|
||||
if (ch == '+')
|
||||
ch = ' ';
|
||||
if (name.size() < MAX_NAME_LENGTH)
|
||||
if (name.size() < max_field_name_size)
|
||||
name += ch;
|
||||
else
|
||||
throw Poco::Net::HTMLFormException("Field name too long");
|
||||
@ -145,7 +149,7 @@ void HTMLForm::readQuery(ReadBuffer & in)
|
||||
{
|
||||
if (ch == '+')
|
||||
ch = ' ';
|
||||
if (value.size() < value_length_limit)
|
||||
if (value.size() < max_field_value_size)
|
||||
value += ch;
|
||||
else
|
||||
throw Poco::Net::HTMLFormException("Field value too long");
|
||||
@ -185,11 +189,11 @@ void HTMLForm::readMultipart(ReadBuffer & in_, PartHandler & handler)
|
||||
/// Read each part until next boundary (or last boundary)
|
||||
while (!in.eof())
|
||||
{
|
||||
if (field_limit && fields > field_limit)
|
||||
if (max_fields_number && fields > max_fields_number)
|
||||
throw Poco::Net::HTMLFormException("Too many form fields");
|
||||
|
||||
Poco::Net::MessageHeader header;
|
||||
readHeaders(header, in);
|
||||
readHeaders(header, in, max_fields_number, max_field_name_size, max_field_value_size);
|
||||
skipToNextLineOrEOF(in);
|
||||
|
||||
NameValueCollection params;
|
||||
@ -209,7 +213,7 @@ void HTMLForm::readMultipart(ReadBuffer & in_, PartHandler & handler)
|
||||
|
||||
while (in.read(ch))
|
||||
{
|
||||
if (value.size() > value_length_limit)
|
||||
if (value.size() > max_field_value_size)
|
||||
throw Poco::Net::HTMLFormException("Field value too long");
|
||||
value += ch;
|
||||
}
|
||||
|
@ -12,6 +12,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
|
||||
class HTMLForm : public Poco::Net::NameValueCollection, private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
@ -19,31 +21,31 @@ public:
|
||||
|
||||
enum Options
|
||||
{
|
||||
OPT_USE_CONTENT_LENGTH = 0x01 // don't use Chunked Transfer-Encoding for multipart requests.
|
||||
OPT_USE_CONTENT_LENGTH = 0x01, /// don't use Chunked Transfer-Encoding for multipart requests.
|
||||
};
|
||||
|
||||
/// Creates an empty HTMLForm and sets the
|
||||
/// encoding to "application/x-www-form-urlencoded".
|
||||
HTMLForm();
|
||||
explicit HTMLForm(const Settings & settings);
|
||||
|
||||
/// Creates an empty HTMLForm that uses the given encoding.
|
||||
/// Encoding must be either "application/x-www-form-urlencoded" (which is the default) or "multipart/form-data".
|
||||
explicit HTMLForm(const std::string & encoding);
|
||||
explicit HTMLForm(const Settings & settings, const std::string & encoding);
|
||||
|
||||
/// Creates a HTMLForm from the given HTTP request.
|
||||
/// Uploaded files are passed to the given PartHandler.
|
||||
HTMLForm(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler);
|
||||
HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler);
|
||||
|
||||
/// Creates a HTMLForm from the given HTTP request.
|
||||
/// Uploaded files are silently discarded.
|
||||
HTMLForm(const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody);
|
||||
HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody);
|
||||
|
||||
/// Creates a HTMLForm from the given HTTP request.
|
||||
/// The request must be a GET request and the form data must be in the query string (URL encoded).
|
||||
/// For POST requests, you must use one of the constructors taking an additional input stream for the request body.
|
||||
explicit HTMLForm(const Poco::Net::HTTPRequest & request);
|
||||
explicit HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request);
|
||||
|
||||
explicit HTMLForm(const Poco::URI & uri);
|
||||
explicit HTMLForm(const Settings & settings, const Poco::URI & uri);
|
||||
|
||||
template <typename T>
|
||||
T getParsed(const std::string & key, T default_value)
|
||||
@ -76,13 +78,6 @@ private:
|
||||
/// This buffer provides data line by line to check for boundary line in a convenient way.
|
||||
class MultipartReadBuffer;
|
||||
|
||||
enum Limits
|
||||
{
|
||||
DFL_FIELD_LIMIT = 100,
|
||||
MAX_NAME_LENGTH = 1024,
|
||||
DFL_MAX_VALUE_LENGTH = 256 * 1024
|
||||
};
|
||||
|
||||
struct Part
|
||||
{
|
||||
std::string name;
|
||||
@ -91,8 +86,8 @@ private:
|
||||
|
||||
using PartVec = std::vector<Part>;
|
||||
|
||||
size_t field_limit;
|
||||
size_t value_length_limit;
|
||||
const size_t max_fields_number, max_field_name_size, max_field_value_size;
|
||||
|
||||
std::string encoding;
|
||||
std::string boundary;
|
||||
PartVec parts;
|
||||
|
@ -17,6 +17,9 @@ namespace DB
|
||||
{
|
||||
HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
|
||||
: max_uri_size(context->getSettingsRef().http_max_uri_size)
|
||||
, max_fields_number(context->getSettingsRef().http_max_fields)
|
||||
, max_field_name_size(context->getSettingsRef().http_max_field_name_size)
|
||||
, max_field_value_size(context->getSettingsRef().http_max_field_value_size)
|
||||
{
|
||||
response.attachRequest(this);
|
||||
|
||||
@ -110,7 +113,7 @@ void HTTPServerRequest::readRequest(ReadBuffer & in)
|
||||
|
||||
skipToNextLineOrEOF(in);
|
||||
|
||||
readHeaders(*this, in);
|
||||
readHeaders(*this, in, max_fields_number, max_field_name_size, max_field_value_size);
|
||||
|
||||
skipToNextLineOrEOF(in);
|
||||
|
||||
|
@ -40,14 +40,14 @@ private:
|
||||
/// Limits for basic sanity checks when reading a header
|
||||
enum Limits
|
||||
{
|
||||
MAX_NAME_LENGTH = 256,
|
||||
MAX_VALUE_LENGTH = 8192,
|
||||
MAX_METHOD_LENGTH = 32,
|
||||
MAX_VERSION_LENGTH = 8,
|
||||
MAX_FIELDS_NUMBER = 100,
|
||||
};
|
||||
|
||||
const size_t max_uri_size;
|
||||
const size_t max_fields_number;
|
||||
const size_t max_field_name_size;
|
||||
const size_t max_field_value_size;
|
||||
|
||||
std::unique_ptr<ReadBuffer> stream;
|
||||
Poco::Net::SocketImpl * socket;
|
||||
|
@ -8,10 +8,6 @@ namespace DB
|
||||
class ReadBuffer;
|
||||
|
||||
void readHeaders(
|
||||
Poco::Net::MessageHeader & headers,
|
||||
ReadBuffer & in,
|
||||
size_t max_fields_number = 100,
|
||||
size_t max_name_length = 256,
|
||||
size_t max_value_length = 8192);
|
||||
Poco::Net::MessageHeader & headers, ReadBuffer & in, size_t max_fields_number, size_t max_name_length, size_t max_value_length);
|
||||
|
||||
}
|
||||
|
@ -895,7 +895,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
|
||||
if (request.getVersion() == HTTPServerRequest::HTTP_1_1)
|
||||
response.setChunkedTransferEncoding(true);
|
||||
|
||||
HTMLForm params(request);
|
||||
HTMLForm params(request_context->getSettingsRef(), request);
|
||||
with_stacktrace = params.getParsed<bool>("stacktrace", false);
|
||||
|
||||
/// FIXME: maybe this check is already unnecessary.
|
||||
|
@ -50,7 +50,7 @@ std::pair<String, bool> InterserverIOHTTPHandler::checkAuthentication(HTTPServer
|
||||
|
||||
void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output)
|
||||
{
|
||||
HTMLForm params(request);
|
||||
HTMLForm params(server.context()->getSettingsRef(), request);
|
||||
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/InterserverCredentials.h>
|
||||
#include <Server/HTTP/HTTPRequestHandler.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Interpreters/InterserverCredentials.h>
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
|
||||
|
@ -18,23 +18,20 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
ReplicasStatusHandler::ReplicasStatusHandler(IServer & server)
|
||||
: context(server.context())
|
||||
ReplicasStatusHandler::ReplicasStatusHandler(IServer & server) : WithContext(server.context())
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
try
|
||||
{
|
||||
HTMLForm params(request);
|
||||
HTMLForm params(getContext()->getSettingsRef(), request);
|
||||
|
||||
/// Even if lag is small, output detailed information about the lag.
|
||||
bool verbose = params.get("verbose", "") == "1";
|
||||
|
||||
const MergeTreeSettings & settings = context->getReplicatedMergeTreeSettings();
|
||||
const MergeTreeSettings & settings = getContext()->getReplicatedMergeTreeSettings();
|
||||
|
||||
bool ok = true;
|
||||
WriteBufferFromOwnString message;
|
||||
@ -48,7 +45,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
if (!db.second->canContainMergeTreeTables())
|
||||
continue;
|
||||
|
||||
for (auto iterator = db.second->getTablesIterator(context); iterator->isValid(); iterator->next())
|
||||
for (auto iterator = db.second->getTablesIterator(getContext()); iterator->isValid(); iterator->next())
|
||||
{
|
||||
const auto & table = iterator->table();
|
||||
if (!table)
|
||||
@ -73,7 +70,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
}
|
||||
}
|
||||
|
||||
const auto & config = context->getConfigRef();
|
||||
const auto & config = getContext()->getConfigRef();
|
||||
setResponseDefaultHeaders(response, config.getUInt("keep_alive_timeout", 10));
|
||||
|
||||
if (!ok)
|
||||
|
@ -9,11 +9,8 @@ class Context;
|
||||
class IServer;
|
||||
|
||||
/// Replies "Ok.\n" if all replicas on this server don't lag too much. Otherwise output lag information.
|
||||
class ReplicasStatusHandler : public HTTPRequestHandler
|
||||
class ReplicasStatusHandler : public HTTPRequestHandler, WithContext
|
||||
{
|
||||
private:
|
||||
ContextPtr context;
|
||||
|
||||
public:
|
||||
explicit ReplicasStatusHandler(IServer & server_);
|
||||
|
||||
|
8
tests/queries/0_stateless/01903_http_fields.reference
Normal file
8
tests/queries/0_stateless/01903_http_fields.reference
Normal file
@ -0,0 +1,8 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
27
tests/queries/0_stateless/01903_http_fields.sh
Executable file
27
tests/queries/0_stateless/01903_http_fields.sh
Executable file
@ -0,0 +1,27 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
DEFAULT_MAX_NAME_SIZE=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.settings WHERE name='http_max_field_name_size'")
|
||||
DEFAULT_MAX_VALUE_SIZE=$($CLICKHOUSE_CLIENT -q "SELECT value FROM system.settings WHERE name='http_max_field_value_size'")
|
||||
|
||||
python3 -c "print('a'*($DEFAULT_MAX_NAME_SIZE-2) + ';')" > $CLICKHOUSE_TMP/short_name.txt
|
||||
python3 -c "print('a'*($DEFAULT_MAX_NAME_SIZE+1) + ';')" > $CLICKHOUSE_TMP/long_name.txt
|
||||
python3 -c "print('a'*($DEFAULT_MAX_NAME_SIZE-2) + ': ' + 'b'*($DEFAULT_MAX_VALUE_SIZE-2))" > $CLICKHOUSE_TMP/short_short.txt
|
||||
python3 -c "print('a'*($DEFAULT_MAX_NAME_SIZE-2) + ': ' + 'b'*($DEFAULT_MAX_VALUE_SIZE+1))" > $CLICKHOUSE_TMP/short_long.txt
|
||||
python3 -c "print('a'*($DEFAULT_MAX_NAME_SIZE+1) + ': ' + 'b'*($DEFAULT_MAX_VALUE_SIZE-2))" > $CLICKHOUSE_TMP/long_short.txt
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H @$CLICKHOUSE_TMP/short_name.txt -d 'SELECT 1'
|
||||
${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}" -H @$CLICKHOUSE_TMP/long_name.txt -d 'SELECT 1' 2>&1 | grep -Fc '400 Bad Request'
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}" -H @$CLICKHOUSE_TMP/short_short.txt -d 'SELECT 1'
|
||||
${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}" -H @$CLICKHOUSE_TMP/short_long.txt -d 'SELECT 1' 2>&1 | grep -Fc '400 Bad Request'
|
||||
${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}" -H @$CLICKHOUSE_TMP/long_short.txt -d 'SELECT 1' 2>&1 | grep -Fc '400 Bad Request'
|
||||
|
||||
# Session and query settings shouldn't affect the HTTP field's name or value sizes.
|
||||
${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}&http_max_field_name_size=$(($DEFAULT_MAX_NAME_SIZE+10))" -H @$CLICKHOUSE_TMP/long_name.txt -d 'SELECT 1' 2>&1 | grep -Fc '400 Bad Request'
|
||||
${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}&http_max_field_value_size=$(($DEFAULT_MAX_VALUE_SIZE+10))" -H @$CLICKHOUSE_TMP/short_long.txt -d 'SELECT 1' 2>&1 | grep -Fc '400 Bad Request'
|
||||
${CLICKHOUSE_CURL} -sSv "${CLICKHOUSE_URL}&http_max_field_name_size=$(($DEFAULT_MAX_NAME_SIZE+10))" -H @$CLICKHOUSE_TMP/long_short.txt -d 'SELECT 1' 2>&1 | grep -Fc '400 Bad Request'
|
||||
|
||||
# TODO: test that session context doesn't affect these settings either.
|
Loading…
Reference in New Issue
Block a user