mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into geodistance-double-precision
This commit is contained in:
commit
c2209c997c
@ -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.
|
||||
|
@ -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.
|
||||
|
@ -106,6 +106,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> 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."},
|
||||
|
96
src/Functions/getClientHTTPHeader.cpp
Normal file
96
src/Functions/getClientHTTPHeader.cpp
Normal file
@ -0,0 +1,96 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionHelpers.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
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<DataTypeString>();
|
||||
}
|
||||
|
||||
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<String>()); 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<FunctionGetClientHTTPHeader>(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"}});
|
||||
}
|
||||
|
||||
}
|
@ -5,6 +5,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Core/ProtocolDefines.h>
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <Poco/Net/HTTPRequest.h>
|
||||
#include <unistd.h>
|
||||
|
||||
#include <Common/config_version.h>
|
||||
@ -255,7 +256,29 @@ String toString(ClientInfo::Interface interface)
|
||||
return "TCP_INTERSERVER";
|
||||
}
|
||||
|
||||
return std::format("Unknown {}!\n", static_cast<int>(interface));
|
||||
return std::format("Unknown server interface ({}).", static_cast<int>(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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -7,6 +7,12 @@
|
||||
#include <Common/VersionNumber.h>
|
||||
#include <boost/algorithm/string/trim.hpp>
|
||||
|
||||
|
||||
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<String, String> 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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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();
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
|
@ -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.
|
||||
|
@ -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)
|
||||
|
||||
|
37
tests/queries/0_stateless/03021_get_client_http_header.sh
Executable file
37
tests/queries/0_stateless/03021_get_client_http_header.sh
Executable file
@ -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))"
|
@ -1601,6 +1601,7 @@ geohashEncode
|
||||
geohashesInBox
|
||||
geoip
|
||||
geospatial
|
||||
getClientHTTPHeader
|
||||
getMacro
|
||||
getOSKernelVersion
|
||||
getServerPort
|
||||
|
Loading…
Reference in New Issue
Block a user