mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add test
This commit is contained in:
parent
b182d87d9c
commit
ba08a580f8
@ -51,7 +51,7 @@ namespace
|
||||
void LibraryRequestHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response)
|
||||
{
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
HTMLForm params(getContext(), request);
|
||||
HTMLForm params(getContext()->getSettingsRef(), request);
|
||||
|
||||
if (!params.has("method"))
|
||||
{
|
||||
|
@ -69,7 +69,7 @@ namespace
|
||||
|
||||
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());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
|
@ -21,7 +21,7 @@ namespace DB
|
||||
{
|
||||
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());
|
||||
|
||||
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(getContext(), request);
|
||||
HTMLForm params(getContext()->getSettingsRef(), request);
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
if (mode == "read")
|
||||
|
@ -28,7 +28,7 @@ namespace
|
||||
|
||||
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());
|
||||
|
||||
auto process_error = [&response, this](const std::string & message)
|
||||
|
@ -234,9 +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_number, 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_value_size, 8192, "Maximum length of field value 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, 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,6 +1,6 @@
|
||||
#include <Server/HTTP/HTMLForm.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <IO/EmptyReadBuffer.h>
|
||||
#include <IO/ReadBufferFromString.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;
|
||||
|
||||
|
||||
HTMLForm::HTMLForm(ContextPtr context)
|
||||
: max_fields_number(context->getSettingsRef().http_max_fields_number)
|
||||
, max_field_name_size(context->getSettingsRef().http_max_field_name_size)
|
||||
, max_field_value_size(context->getSettingsRef().http_max_field_value_size)
|
||||
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(ContextPtr context, const std::string & encoding_) : HTMLForm(context)
|
||||
HTMLForm::HTMLForm(const Settings & settings, const std::string & encoding_) : HTMLForm(settings)
|
||||
{
|
||||
encoding = encoding_;
|
||||
}
|
||||
|
||||
|
||||
HTMLForm::HTMLForm(ContextPtr context, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler)
|
||||
: HTMLForm(context)
|
||||
HTMLForm::HTMLForm(const Settings & settings, const Poco::Net::HTTPRequest & request, ReadBuffer & requestBody, PartHandler & handler)
|
||||
: HTMLForm(settings)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
|
||||
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
|
||||
readQuery(istr);
|
||||
|
@ -1,6 +1,5 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
@ -13,6 +12,8 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
struct Settings;
|
||||
|
||||
class HTMLForm : public Poco::Net::NameValueCollection, private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
@ -25,26 +26,26 @@ public:
|
||||
|
||||
/// Creates an empty HTMLForm and sets the
|
||||
/// 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.
|
||||
/// 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.
|
||||
/// 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.
|
||||
/// 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.
|
||||
/// 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(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>
|
||||
T getParsed(const std::string & key, T default_value)
|
||||
|
@ -17,7 +17,7 @@ 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_number)
|
||||
, 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)
|
||||
{
|
||||
|
@ -895,7 +895,7 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
|
||||
if (request.getVersion() == HTTPServerRequest::HTTP_1_1)
|
||||
response.setChunkedTransferEncoding(true);
|
||||
|
||||
HTMLForm params(request_context, 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(server.context(), request);
|
||||
HTMLForm params(server.context()->getSettingsRef(), request);
|
||||
|
||||
LOG_TRACE(log, "Request URI: {}", request.getURI());
|
||||
|
||||
|
@ -26,7 +26,7 @@ void ReplicasStatusHandler::handleRequest(HTTPServerRequest & request, HTTPServe
|
||||
{
|
||||
try
|
||||
{
|
||||
HTMLForm params(getContext(), request);
|
||||
HTMLForm params(getContext()->getSettingsRef(), request);
|
||||
|
||||
/// Even if lag is small, output detailed information about the lag.
|
||||
bool verbose = params.get("verbose", "") == "1";
|
||||
|
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