Merge branch 'master' into geodistance-double-precision

This commit is contained in:
Alexey Milovidov 2024-03-25 17:49:32 +03:00 committed by GitHub
commit c2209c997c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
14 changed files with 219 additions and 25 deletions

View File

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

View File

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

View File

@ -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."},

View 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"}});
}
}

View File

@ -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;
}
}
}

View File

@ -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;

View File

@ -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;
}

View File

@ -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();

View File

@ -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)

View File

@ -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);

View File

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

View File

@ -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)

View 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))"

View File

@ -1601,6 +1601,7 @@ geohashEncode
geohashesInBox
geoip
geospatial
getClientHTTPHeader
getMacro
getOSKernelVersion
getServerPort