This commit is contained in:
Ivan Lezhankin 2021-06-16 17:33:14 +03:00
parent b182d87d9c
commit ba08a580f8
14 changed files with 66 additions and 30 deletions

View File

@ -51,7 +51,7 @@ namespace
void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
LOG_TRACE(log, "Request URI: {}", request.getURI()); LOG_TRACE(log, "Request URI: {}", request.getURI());
HTMLForm params(getContext(), request); HTMLForm params(getContext()->getSettingsRef(), request);
if (!params.has("method")) if (!params.has("method"))
{ {

View File

@ -69,7 +69,7 @@ namespace
void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void ODBCColumnsInfoHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
HTMLForm params(getContext(), request, request.getStream()); HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
LOG_TRACE(log, "Request URI: {}", request.getURI()); LOG_TRACE(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message) auto process_error = [&response, this](const std::string & message)

View File

@ -21,7 +21,7 @@ namespace DB
{ {
void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void IdentifierQuoteHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
HTMLForm params(getContext(), request, request.getStream()); HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
LOG_TRACE(log, "Request URI: {}", request.getURI()); LOG_TRACE(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message) auto process_error = [&response, this](const std::string & message)

View File

@ -50,7 +50,7 @@ void ODBCHandler::processError(HTTPServerResponse & response, const std::string
void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void ODBCHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
HTMLForm params(getContext(), request); HTMLForm params(getContext()->getSettingsRef(), request);
LOG_TRACE(log, "Request URI: {}", request.getURI()); LOG_TRACE(log, "Request URI: {}", request.getURI());
if (mode == "read") if (mode == "read")

View File

@ -28,7 +28,7 @@ namespace
void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response) void SchemaAllowedHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
{ {
HTMLForm params(getContext(), request, request.getStream()); HTMLForm params(getContext()->getSettingsRef(), request, request.getStream());
LOG_TRACE(log, "Request URI: {}", request.getURI()); LOG_TRACE(log, "Request URI: {}", request.getURI());
auto process_error = [&response, this](const std::string & message) auto process_error = [&response, this](const std::string & message)

View File

@ -234,9 +234,9 @@ class IColumn;
M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ 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(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_uri_size, 1048576, "Maximum URI length of HTTP request", 0) \
M(UInt64, http_max_fields_number, 1000000, "Maximum number of fields in HTTP header", 0) \ M(UInt64, http_max_fields, 1000000, "Maximum number of fields in HTTP header", 0) \
M(UInt64, http_max_field_name_size, 1024, "Maximum length of field name 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, 8192, "Maximum length of field value 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, 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, 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) \ M(Bool, joined_subquery_requires_alias, true, "Force joined subqueries and table functions to have aliases for correct name qualification.", 0) \

View File

@ -1,6 +1,6 @@
#include <Server/HTTP/HTMLForm.h> #include <Server/HTTP/HTMLForm.h>
#include <Interpreters/Context.h> #include <Core/Settings.h>
#include <IO/EmptyReadBuffer.h> #include <IO/EmptyReadBuffer.h>
#include <IO/ReadBufferFromString.h> #include <IO/ReadBufferFromString.h>
#include <Server/HTTP/ReadHeaders.h> #include <Server/HTTP/ReadHeaders.h>
@ -36,39 +36,39 @@ const std::string HTMLForm::ENCODING_MULTIPART = "multipart/form-data";
const int HTMLForm::UNKNOWN_CONTENT_LENGTH = -1; const int HTMLForm::UNKNOWN_CONTENT_LENGTH = -1;
HTMLForm::HTMLForm(ContextPtr context) HTMLForm::HTMLForm(const Settings & settings)
: max_fields_number(context->getSettingsRef().http_max_fields_number) : max_fields_number(settings.http_max_fields)
, max_field_name_size(context->getSettingsRef().http_max_field_name_size) , max_field_name_size(settings.http_max_field_name_size)
, max_field_value_size(context->getSettingsRef().http_max_field_value_size) , max_field_value_size(settings.http_max_field_value_size)
, encoding(ENCODING_URL) , encoding(ENCODING_URL)
{ {
} }
HTMLForm::HTMLForm(ContextPtr context, const std::string & encoding_) : HTMLForm(context) HTMLForm::HTMLForm(const Settings & settings, const std::string & encoding_) : HTMLForm(settings)
{ {
encoding = encoding_; encoding = encoding_;
} }
HTMLForm::HTMLForm(ContextPtr context, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler) HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler)
: HTMLForm(context) : HTMLForm(settings)
{ {
load(request, requestBody, handler); load(request, requestBody, handler);
} }
HTMLForm::HTMLForm(ContextPtr context, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody) : HTMLForm(context) HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody) : HTMLForm(settings)
{ {
load(request, requestBody); load(request, requestBody);
} }
HTMLForm::HTMLForm(ContextPtr context, const Poco::Net::HTTPRequest & request) : HTMLForm(context, Poco::URI(request.getURI())) HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request) : HTMLForm(settings, Poco::URI(request.getURI()))
{ {
} }
HTMLForm::HTMLForm(ContextPtr context, const Poco::URI & uri) : HTMLForm(context) 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); readQuery(istr);

View File

@ -1,6 +1,5 @@
#pragma once #pragma once
#include <Interpreters/Context_fwd.h>
#include <IO/PeekableReadBuffer.h> #include <IO/PeekableReadBuffer.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
@ -13,6 +12,8 @@
namespace DB namespace DB
{ {
struct Settings;
class HTMLForm : public Poco::Net::NameValueCollection, private boost::noncopyable class HTMLForm : public Poco::Net::NameValueCollection, private boost::noncopyable
{ {
public: public:
@ -25,26 +26,26 @@ public:
/// Creates an empty HTMLForm and sets the /// Creates an empty HTMLForm and sets the
/// encoding to "application/x-www-form-urlencoded". /// encoding to "application/x-www-form-urlencoded".
explicit HTMLForm(ContextPtr context); explicit HTMLForm(const Settings & settings);
/// Creates an empty HTMLForm that uses the given encoding. /// 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". /// Encoding must be either "application/x-www-form-urlencoded" (which is the default) or "multipart/form-data".
explicit HTMLForm(ContextPtr context, const std::string & encoding); explicit HTMLForm(const Settings & settings, const std::string & encoding);
/// Creates a HTMLForm from the given HTTP request. /// Creates a HTMLForm from the given HTTP request.
/// Uploaded files are passed to the given PartHandler. /// Uploaded files are passed to the given PartHandler.
HTMLForm(ContextPtr context, 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. /// Creates a HTMLForm from the given HTTP request.
/// Uploaded files are silently discarded. /// Uploaded files are silently discarded.
HTMLForm(ContextPtr context, 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. /// 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). /// 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. /// For POST requests, you must use one of the constructors taking an additional input stream for the request body.
explicit HTMLForm(ContextPtr context, const Poco::Net::HTTPRequest & request); explicit HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request);
explicit HTMLForm(ContextPtr context, const Poco::URI & uri); explicit HTMLForm(const Settings & settings, const Poco::URI & uri);
template <typename T> template <typename T>
T getParsed(const std::string & key, T default_value) T getParsed(const std::string & key, T default_value)

View File

@ -17,7 +17,7 @@ namespace DB
{ {
HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session) HTTPServerRequest::HTTPServerRequest(ContextPtr context, HTTPServerResponse & response, Poco::Net::HTTPServerSession & session)
: max_uri_size(context->getSettingsRef().http_max_uri_size) : max_uri_size(context->getSettingsRef().http_max_uri_size)
, max_fields_number(context->getSettingsRef().http_max_fields_number) , max_fields_number(context->getSettingsRef().http_max_fields)
, max_field_name_size(context->getSettingsRef().http_max_field_name_size) , max_field_name_size(context->getSettingsRef().http_max_field_name_size)
, max_field_value_size(context->getSettingsRef().http_max_field_value_size) , max_field_value_size(context->getSettingsRef().http_max_field_value_size)
{ {

View File

@ -895,7 +895,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
if (request.getVersion() == HTTPServerRequest::HTTP_1_1) if (request.getVersion() == HTTPServerRequest::HTTP_1_1)
response.setChunkedTransferEncoding(true); response.setChunkedTransferEncoding(true);
HTMLForm params(request_context, request); HTMLForm params(request_context->getSettingsRef(), request);
with_stacktrace = params.getParsed<bool>("stacktrace", false); with_stacktrace = params.getParsed<bool>("stacktrace", false);
/// FIXME: maybe this check is already unnecessary. /// FIXME: maybe this check is already unnecessary.

View File

@ -50,7 +50,7 @@ std::pair<String, bool> InterserverIOHTTPHandler::checkAuthentication(HTTPServer
void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output) void InterserverIOHTTPHandler::processQuery(HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output)
{ {
HTMLForm params(server.context(), request); HTMLForm params(server.context()->getSettingsRef(), request);
LOG_TRACE(log, "Request URI: {}", request.getURI()); LOG_TRACE(log, "Request URI: {}", request.getURI());

View File

@ -26,7 +26,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
{ {
try try
{ {
HTMLForm params(getContext(), request); HTMLForm params(getContext()->getSettingsRef(), request);
/// Even if lag is small, output detailed information about the lag. /// Even if lag is small, output detailed information about the lag.
bool verbose = params.get("verbose", "") == "1"; bool verbose = params.get("verbose", "") == "1";

View File

@ -0,0 +1,8 @@
1
1
1
1
1
1
1
1

View 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.