diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 0ca404274ec..82cc68133da 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -3132,3 +3132,17 @@ Result: │ (616.2931945826209,108.8825,115.6175) │ └───────────────────────────────────────┘ ``` + +## getClientHTTPHeader + +Get the value of an HTTP header. + +If there is no such header or the current request is not performed via the HTTP interface, the function returns an empty string. +Certain HTTP headers (e.g., `Authentication` and `X-ClickHouse-*`) are restricted. + +The function requires the setting `allow_get_client_http_header` to be enabled. +The setting is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. + +HTTP headers are case sensitive for this function. + +If the function is used in the context of a distributed query, it returns non-empty result only on the initiator node. diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5feaf485002..170fd4e9ca0 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -868,6 +868,8 @@ class IColumn; M(Bool, use_variant_as_common_type, false, "Use Variant as a result type for if/multiIf in case when there is no common type for arguments", 0) \ M(Bool, enable_order_by_all, true, "Enable sorting expression ORDER BY ALL.", 0) \ M(Bool, traverse_shadow_remote_data_paths, false, "Traverse shadow directory when query system.remote_data_paths", 0) \ + M(Bool, geo_distance_returns_float64_on_float64_arguments, true, "If all four arguments to `geoDistance`, `greatCircleDistance`, `greatCircleAngle` functions are Float64, return Float64 and use double precision for internal calculations. In previous ClickHouse versions, the functions always returned Float32.", 0) \ + M(Bool, allow_get_client_http_header, false, "Allow to use the function `getClientHTTPHeader` which lets to obtain a value of an the current HTTP request's header. It is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. Note that the `X-ClickHouse-*` and `Authentication` headers are always restricted and cannot be obtained with this function.", 0) \ \ /** Experimental functions */ \ M(Bool, allow_experimental_materialized_postgresql_table, false, "Allows to use the MaterializedPostgreSQL table engine. Disabled by default, because this feature is experimental", 0) \ @@ -902,7 +904,6 @@ class IColumn; M(Int64, ignore_cold_parts_seconds, 0, "Only available in ClickHouse Cloud. Exclude new data parts from SELECT queries until they're either pre-warmed (see cache_populated_by_fetch) or this many seconds old. Only for Replicated-/SharedMergeTree.", 0) \ M(Int64, prefer_warmed_unmerged_parts_seconds, 0, "Only available in ClickHouse Cloud. If a merged part is less than this many seconds old and is not pre-warmed (see cache_populated_by_fetch), but all its source parts are available and pre-warmed, SELECT queries will read from those parts instead. Only for ReplicatedMergeTree. Note that this only checks whether CacheWarmer processed the part; if the part was fetched into cache by something else, it'll still be considered cold until CacheWarmer gets to it; if it was warmed, then evicted from cache, it'll still be considered warm.", 0) \ M(Bool, iceberg_engine_ignore_schema_evolution, false, "Ignore schema evolution in Iceberg table engine and read all data using latest schema saved on table creation. Note that it can lead to incorrect result", 0) \ - M(Bool, geo_distance_returns_float64_on_float64_arguments, true, "If all four arguments to `geoDistance`, `greatCircleDistance`, `greatCircleAngle` functions are Float64, return Float64 and use double precision for internal calculations. In previous ClickHouse versions, the functions always returned Float32.", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS, move obsolete settings to OBSOLETE_SETTINGS and obsolete format settings to OBSOLETE_FORMAT_SETTINGS. diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index 5b9abd53b2a..6b31e9cd249 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -106,6 +106,7 @@ static std::map sett {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, + {"allow_get_client_http_header", false, false, "Introduced a new function."}, {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, diff --git a/src/Functions/getClientHTTPHeader.cpp b/src/Functions/getClientHTTPHeader.cpp new file mode 100644 index 00000000000..ebd070a90b9 --- /dev/null +++ b/src/Functions/getClientHTTPHeader.cpp @@ -0,0 +1,96 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int FUNCTION_NOT_ALLOWED; +} + +namespace +{ + +class FunctionGetClientHTTPHeader : public IFunction, WithContext +{ +public: + explicit FunctionGetClientHTTPHeader(ContextPtr context_) + : WithContext(context_) + { + if (!getContext()->getSettingsRef().allow_get_client_http_header) + throw Exception(ErrorCodes::FUNCTION_NOT_ALLOWED, "The function getClientHTTPHeader requires setting `allow_get_client_http_header` to be enabled."); + } + + String getName() const override { return "getClientHTTPHeader"; } + + bool useDefaultImplementationForConstants() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo &) const override { return false; } + + size_t getNumberOfArguments() const override + { + return 1; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isString(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "The argument of function {} must be String", getName()); + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override + { + const ClientInfo & client_info = getContext()->getClientInfo(); + + const auto & source = arguments[0].column; + auto result = result_type->createColumn(); + result->reserve(input_rows_count); + + for (size_t row = 0; row < input_rows_count; ++row) + { + Field header; + source->get(row, header); + if (auto it = client_info.http_headers.find(header.get()); it != client_info.http_headers.end()) + result->insert(it->second); + else + result->insertDefault(); + } + + return result; + } +}; + +} + +REGISTER_FUNCTION(GetClientHTTPHeader) +{ + factory.registerFunction("getClientHTTPHeader", + [](ContextPtr context) { return std::make_shared(context); }, + FunctionDocumentation{ + .description = R"( +Get the value of an HTTP header. + +If there is no such header or the current request is not performed via the HTTP interface, the function returns an empty string. +Certain HTTP headers (e.g., `Authentication` and `X-ClickHouse-*`) are restricted. + +The function requires the setting `allow_get_client_http_header` to be enabled. +The setting is not enabled by default for security reasons, because some headers, such as `Cookie`, could contain sensitive info. + +HTTP headers are case sensitive for this function. + +If the function is used in the context of a distributed query, it returns non-empty result only on the initiator node. +", + .syntax = "getClientHTTPHeader(name)", + .arguments = {{"name", "The HTTP header name (String)"}}, + .returned_value = "The value of the header (String).", + .categories{"Miscellaneous"}}); +} + +} diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index e4778edeb9c..ce1efb61cc0 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -255,7 +256,29 @@ String toString(ClientInfo::Interface interface) return "TCP_INTERSERVER"; } - return std::format("Unknown {}!\n", static_cast(interface)); + return std::format("Unknown server interface ({}).", static_cast(interface)); +} + +void ClientInfo::setFromHTTPRequest(const Poco::Net::HTTPRequest & request) +{ + http_method = ClientInfo::HTTPMethod::UNKNOWN; + if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) + http_method = ClientInfo::HTTPMethod::GET; + else if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) + http_method = ClientInfo::HTTPMethod::POST; + + http_user_agent = request.get("User-Agent", ""); + http_referer = request.get("Referer", ""); + forwarded_for = request.get("X-Forwarded-For", ""); + + for (const auto & header : request) + { + /// These headers can contain authentication info and shouldn't be accessible by the user. + String key_lowercase = Poco::toLower(header.first); + if (key_lowercase.starts_with("x-clickhouse") || key_lowercase == "authentication") + continue; + http_headers[header.first] = header.second; + } } } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index 70524333047..c2ed9f7ffa4 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -7,6 +7,12 @@ #include #include + +namespace Poco::Net +{ + class HTTPRequest; +} + namespace DB { @@ -93,6 +99,7 @@ public: HTTPMethod http_method = HTTPMethod::UNKNOWN; String http_user_agent; String http_referer; + std::unordered_map http_headers; /// For mysql and postgresql UInt64 connection_id = 0; @@ -135,6 +142,9 @@ public: /// Initialize parameters on client initiating query. void setInitialQuery(); + /// Initialize parameters related to HTTP request. + void setFromHTTPRequest(const Poco::Net::HTTPRequest & request); + bool clientVersionEquals(const ClientInfo & other, bool compare_patch) const; String getVersionStr() const; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index bde5c9c4c1b..086ed333366 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -4641,11 +4641,9 @@ void Context::setClientConnectionId(uint32_t connection_id_) client_info.connection_id = connection_id_; } -void Context::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Context::setHTTPClientInfo(const Poco::Net::HTTPRequest & request) { - client_info.http_method = http_method; - client_info.http_user_agent = http_user_agent; - client_info.http_referer = http_referer; + client_info.setFromHTTPRequest(request); need_recalculate_access = true; } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index f7bf8282d8a..8601d09621f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -642,7 +642,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHTTPClientInfo(const Poco::Net::HTTPRequest & request); void setForwardedFor(const String & forwarded_for); void setQueryKind(ClientInfo::QueryKind query_kind); void setQueryKindInitial(); diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index b52f8a507e3..9f64380ab43 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -429,18 +429,12 @@ void Session::setClientConnectionId(uint32_t connection_id) prepared_client_info->connection_id = connection_id; } -void Session::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer) +void Session::setHTTPClientInfo(const Poco::Net::HTTPRequest & request) { if (session_context) - { - session_context->setHTTPClientInfo(http_method, http_user_agent, http_referer); - } + session_context->setHTTPClientInfo(request); else - { - prepared_client_info->http_method = http_method; - prepared_client_info->http_user_agent = http_user_agent; - prepared_client_info->http_referer = http_referer; - } + prepared_client_info->setFromHTTPRequest(request); } void Session::setForwardedFor(const String & forwarded_for) diff --git a/src/Interpreters/Session.h b/src/Interpreters/Session.h index 334560a33c8..e6cb7ca20cd 100644 --- a/src/Interpreters/Session.h +++ b/src/Interpreters/Session.h @@ -65,7 +65,7 @@ public: void setClientInterface(ClientInfo::Interface interface); void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); void setClientConnectionId(uint32_t connection_id); - void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer); + void setHTTPClientInfo(const Poco::Net::HTTPRequest & request); void setForwardedFor(const String & forwarded_for); void setQuotaClientKey(const String & quota_key); void setConnectionClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version); diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp index c112eefec6c..fd9be999276 100644 --- a/src/Server/HTTPHandler.cpp +++ b/src/Server/HTTPHandler.cpp @@ -490,14 +490,7 @@ bool HTTPHandler::authenticateUser( /// Set client info. It will be used for quota accounting parameters in 'setUser' method. - ClientInfo::HTTPMethod http_method = ClientInfo::HTTPMethod::UNKNOWN; - if (request.getMethod() == HTTPServerRequest::HTTP_GET) - http_method = ClientInfo::HTTPMethod::GET; - else if (request.getMethod() == HTTPServerRequest::HTTP_POST) - http_method = ClientInfo::HTTPMethod::POST; - - session->setHTTPClientInfo(http_method, request.get("User-Agent", ""), request.get("Referer", "")); - session->setForwardedFor(request.get("X-Forwarded-For", "")); + session->setHTTPClientInfo(request); session->setQuotaClientKey(quota_key); /// Extract the last entry from comma separated list of forwarded_for addresses. diff --git a/tests/queries/0_stateless/03021_get_client_http_header.reference b/tests/queries/0_stateless/03021_get_client_http_header.reference new file mode 100644 index 00000000000..099bf1fa4e3 --- /dev/null +++ b/tests/queries/0_stateless/03021_get_client_http_header.reference @@ -0,0 +1,26 @@ +-- It works. +application/x-www-form-urlencoded +-- It supports non constant arguments. +application/x-www-form-urlencoded +*/* +-- Empty string for non-existent headers. +application/x-www-form-urlencoded + +-- I can use my own headers. +wtf +-- Some headers cannot be obtained. + + +-- The setting matters. +FUNCTION_NOT_ALLOWED +-- The setting is not enabled by default. +FUNCTION_NOT_ALLOWED +-- Are headers case-sentitive? +application/x-www-form-urlencoded + +-- Using it from non-HTTP does not make sense. + + +-- Does it work for distributed queries? (not yet, but maybe it will be needed later) + + diff --git a/tests/queries/0_stateless/03021_get_client_http_header.sh b/tests/queries/0_stateless/03021_get_client_http_header.sh new file mode 100755 index 00000000000..71486c725b4 --- /dev/null +++ b/tests/queries/0_stateless/03021_get_client_http_header.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo "-- It works." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('Content-Type')" + +echo "-- It supports non constant arguments." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Accept']))" + +echo "-- Empty string for non-existent headers." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Upyachka']))" + +echo "-- I can use my own headers." +${CLICKHOUSE_CURL} -H 'Upyachka: wtf' "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('Upyachka')" + +echo "-- Some headers cannot be obtained." +${CLICKHOUSE_CURL} -H 'X-ClickHouse-WTF: Secret' "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('X-ClickHouse-WTF')" +${CLICKHOUSE_CURL} -H 'Authentication: Secret' "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader('Authentication')" + +echo "-- The setting matters." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=0" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Accept']))" | grep -o -F 'FUNCTION_NOT_ALLOWED' + +echo "-- The setting is not enabled by default." +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'Accept']))" | grep -o -F 'FUNCTION_NOT_ALLOWED' + +echo "-- Are headers case-sentitive?" +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1" -d "SELECT getClientHTTPHeader(arrayJoin(['Content-Type', 'content-type']))" + +echo "-- Using it from non-HTTP does not make sense." +${CLICKHOUSE_CLIENT} --allow_get_client_http_header true --query "SELECT getClientHTTPHeader('Host')" +${CLICKHOUSE_LOCAL} --allow_get_client_http_header true --query "SELECT getClientHTTPHeader('Host')" + +echo "-- Does it work for distributed queries? (not yet, but maybe it will be needed later)" +${CLICKHOUSE_CURL} "${CLICKHOUSE_URL}&allow_get_client_http_header=1&prefer_localhost_replica=0" -d "SELECT getClientHTTPHeader(name) FROM clusterAllReplicas('test_cluster_one_shard_two_replicas', view(SELECT 'Content-Type' AS name))" diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index cd0cf3f1288..ee3ef1ae795 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1601,6 +1601,7 @@ geohashEncode geohashesInBox geoip geospatial +getClientHTTPHeader getMacro getOSKernelVersion getServerPort