mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #66067 from vitlibar/move-http-handler-stuff-to-separate-files
Move some of HTTPHandler stuff to separate source files in order to reuse it in PrometheusRequestHandler
This commit is contained in:
commit
6d0bf97476
@ -79,7 +79,7 @@ namespace Net
|
||||
/// Returns the value of the first name-value pair with the given name.
|
||||
/// If no value with the given name has been found, the defaultValue is returned.
|
||||
|
||||
const std::vector<std::reference_wrapper<const std::string>> getAll(const std::string & name) const;
|
||||
std::vector<std::string> getAll(const std::string & name) const;
|
||||
/// Returns all values of all name-value pairs with the given name.
|
||||
///
|
||||
/// Returns an empty vector if there are no name-value pairs with the given name.
|
||||
|
@ -102,9 +102,9 @@ const std::string& NameValueCollection::get(const std::string& name, const std::
|
||||
return defaultValue;
|
||||
}
|
||||
|
||||
const std::vector<std::reference_wrapper<const std::string>> NameValueCollection::getAll(const std::string& name) const
|
||||
std::vector<std::string> NameValueCollection::getAll(const std::string& name) const
|
||||
{
|
||||
std::vector<std::reference_wrapper<const std::string>> values;
|
||||
std::vector<std::string> values;
|
||||
for (ConstIterator it = _map.find(name); it != _map.end(); it++)
|
||||
if (it->first == name)
|
||||
values.push_back(it->second);
|
||||
|
@ -29,33 +29,12 @@ BlockIO InterpreterSetRoleQuery::execute()
|
||||
|
||||
void InterpreterSetRoleQuery::setRole(const ASTSetRoleQuery & query)
|
||||
{
|
||||
auto & access_control = getContext()->getAccessControl();
|
||||
auto session_context = getContext()->getSessionContext();
|
||||
auto user = session_context->getUser();
|
||||
|
||||
if (query.kind == ASTSetRoleQuery::Kind::SET_ROLE_DEFAULT)
|
||||
{
|
||||
session_context->setCurrentRolesDefault();
|
||||
}
|
||||
else
|
||||
{
|
||||
RolesOrUsersSet roles_from_query{*query.roles, access_control};
|
||||
std::vector<UUID> new_current_roles;
|
||||
if (roles_from_query.all)
|
||||
{
|
||||
new_current_roles = user->granted_roles.findGranted(roles_from_query);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & id : roles_from_query.getMatchingIDs())
|
||||
{
|
||||
if (!user->granted_roles.isGranted(id))
|
||||
throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role should be granted to set current");
|
||||
new_current_roles.emplace_back(id);
|
||||
}
|
||||
}
|
||||
session_context->setCurrentRoles(new_current_roles);
|
||||
}
|
||||
session_context->setCurrentRoles(RolesOrUsersSet{*query.roles, session_context->getAccessControl()});
|
||||
}
|
||||
|
||||
|
||||
|
@ -726,7 +726,10 @@ try
|
||||
|
||||
/// Access rights must be checked for the user who executed the initial INSERT query.
|
||||
if (key.user_id)
|
||||
insert_context->setUser(*key.user_id, key.current_roles);
|
||||
{
|
||||
insert_context->setUser(*key.user_id);
|
||||
insert_context->setCurrentRoles(key.current_roles);
|
||||
}
|
||||
|
||||
insert_context->setSettings(key.settings);
|
||||
|
||||
|
@ -58,6 +58,7 @@
|
||||
#include <Access/EnabledRowPolicies.h>
|
||||
#include <Access/QuotaUsage.h>
|
||||
#include <Access/User.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Access/SettingsProfile.h>
|
||||
#include <Access/SettingsProfilesInfo.h>
|
||||
#include <Access/SettingsConstraintsAndProfileIDs.h>
|
||||
@ -190,6 +191,7 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
||||
extern const int CLUSTER_DOESNT_EXIST;
|
||||
extern const int SET_NON_GRANTED_ROLE;
|
||||
}
|
||||
|
||||
#define SHUTDOWN(log, desc, ptr, method) do \
|
||||
@ -1303,7 +1305,7 @@ ConfigurationPtr Context::getUsersConfig()
|
||||
return shared->users_config;
|
||||
}
|
||||
|
||||
void Context::setUser(const UUID & user_id_, const std::optional<const std::vector<UUID>> & current_roles_)
|
||||
void Context::setUser(const UUID & user_id_)
|
||||
{
|
||||
/// Prepare lists of user's profiles, constraints, settings, roles.
|
||||
/// NOTE: AccessControl::read<User>() and other AccessControl's functions may require some IO work,
|
||||
@ -1312,8 +1314,8 @@ void Context::setUser(const UUID & user_id_, const std::optional<const std::vect
|
||||
auto & access_control = getAccessControl();
|
||||
auto user = access_control.read<User>(user_id_);
|
||||
|
||||
auto new_current_roles = current_roles_ ? user->granted_roles.findGranted(*current_roles_) : user->granted_roles.findGranted(user->default_roles);
|
||||
auto enabled_roles = access_control.getEnabledRolesInfo(new_current_roles, {});
|
||||
auto default_roles = user->granted_roles.findGranted(user->default_roles);
|
||||
auto enabled_roles = access_control.getEnabledRolesInfo(default_roles, {});
|
||||
auto enabled_profiles = access_control.getEnabledSettingsInfo(user_id_, user->settings, enabled_roles->enabled_roles, enabled_roles->settings_from_enabled_roles);
|
||||
const auto & database = user->default_database;
|
||||
|
||||
@ -1327,7 +1329,7 @@ void Context::setUser(const UUID & user_id_, const std::optional<const std::vect
|
||||
/// so we shouldn't check constraints here.
|
||||
setCurrentProfilesWithLock(*enabled_profiles, /* check_constraints= */ false, lock);
|
||||
|
||||
setCurrentRolesWithLock(new_current_roles, lock);
|
||||
setCurrentRolesWithLock(default_roles, lock);
|
||||
|
||||
/// It's optional to specify the DEFAULT DATABASE in the user's definition.
|
||||
if (!database.empty())
|
||||
@ -1362,25 +1364,66 @@ std::optional<UUID> Context::getUserID() const
|
||||
return user_id;
|
||||
}
|
||||
|
||||
void Context::setCurrentRolesWithLock(const std::vector<UUID> & current_roles_, const std::lock_guard<ContextSharedMutex> &)
|
||||
void Context::setCurrentRolesWithLock(const std::vector<UUID> & new_current_roles, const std::lock_guard<ContextSharedMutex> &)
|
||||
{
|
||||
if (current_roles_.empty())
|
||||
if (new_current_roles.empty())
|
||||
current_roles = nullptr;
|
||||
else
|
||||
current_roles = std::make_shared<std::vector<UUID>>(current_roles_);
|
||||
current_roles = std::make_shared<std::vector<UUID>>(new_current_roles);
|
||||
need_recalculate_access = true;
|
||||
}
|
||||
|
||||
void Context::setCurrentRoles(const std::vector<UUID> & current_roles_)
|
||||
void Context::setCurrentRolesImpl(const std::vector<UUID> & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr<const User> & user)
|
||||
{
|
||||
std::lock_guard lock(mutex);
|
||||
setCurrentRolesWithLock(current_roles_, lock);
|
||||
if (skip_if_not_granted)
|
||||
{
|
||||
auto filtered_role_ids = user->granted_roles.findGranted(new_current_roles);
|
||||
std::lock_guard lock{mutex};
|
||||
setCurrentRolesWithLock(filtered_role_ids, lock);
|
||||
return;
|
||||
}
|
||||
if (throw_if_not_granted)
|
||||
{
|
||||
for (const auto & role_id : new_current_roles)
|
||||
{
|
||||
if (!user->granted_roles.isGranted(role_id))
|
||||
{
|
||||
auto role_name = getAccessControl().tryReadName(role_id);
|
||||
throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role {} should be granted to set as a current", role_name.value_or(toString(role_id)));
|
||||
}
|
||||
}
|
||||
}
|
||||
std::lock_guard lock2{mutex};
|
||||
setCurrentRolesWithLock(new_current_roles, lock2);
|
||||
}
|
||||
|
||||
void Context::setCurrentRoles(const std::vector<UUID> & new_current_roles, bool check_grants)
|
||||
{
|
||||
setCurrentRolesImpl(new_current_roles, /* throw_if_not_granted= */ check_grants, /* skip_if_not_granted= */ !check_grants, getUser());
|
||||
}
|
||||
|
||||
void Context::setCurrentRoles(const RolesOrUsersSet & new_current_roles, bool check_grants)
|
||||
{
|
||||
if (new_current_roles.all)
|
||||
{
|
||||
auto user = getUser();
|
||||
setCurrentRolesImpl(user->granted_roles.findGranted(new_current_roles), /* throw_if_not_granted= */ false, /* skip_if_not_granted= */ false, user);
|
||||
}
|
||||
else
|
||||
{
|
||||
setCurrentRoles(new_current_roles.getMatchingIDs(), check_grants);
|
||||
}
|
||||
}
|
||||
|
||||
void Context::setCurrentRoles(const Strings & new_current_roles, bool check_grants)
|
||||
{
|
||||
setCurrentRoles(getAccessControl().getIDs<Role>(new_current_roles), check_grants);
|
||||
}
|
||||
|
||||
void Context::setCurrentRolesDefault()
|
||||
{
|
||||
auto user = getUser();
|
||||
setCurrentRoles(user->granted_roles.findGranted(user->default_roles));
|
||||
setCurrentRolesImpl(user->granted_roles.findGranted(user->default_roles), /* throw_if_not_granted= */ false, /* skip_if_not_granted= */ false, user);
|
||||
}
|
||||
|
||||
std::vector<UUID> Context::getCurrentRoles() const
|
||||
|
@ -61,6 +61,7 @@ class AccessFlags;
|
||||
struct AccessRightsElement;
|
||||
class AccessRightsElements;
|
||||
enum class RowPolicyFilterType : uint8_t;
|
||||
struct RolesOrUsersSet;
|
||||
class EmbeddedDictionaries;
|
||||
class ExternalDictionariesLoader;
|
||||
class ExternalUserDefinedExecutableFunctionsLoader;
|
||||
@ -607,13 +608,15 @@ public:
|
||||
|
||||
/// Sets the current user assuming that he/she is already authenticated.
|
||||
/// WARNING: This function doesn't check password!
|
||||
void setUser(const UUID & user_id_, const std::optional<const std::vector<UUID>> & current_roles_ = {});
|
||||
void setUser(const UUID & user_id_);
|
||||
UserPtr getUser() const;
|
||||
|
||||
std::optional<UUID> getUserID() const;
|
||||
String getUserName() const;
|
||||
|
||||
void setCurrentRoles(const std::vector<UUID> & current_roles_);
|
||||
void setCurrentRoles(const Strings & new_current_roles, bool check_grants = true);
|
||||
void setCurrentRoles(const std::vector<UUID> & new_current_roles, bool check_grants = true);
|
||||
void setCurrentRoles(const RolesOrUsersSet & new_current_roles, bool check_grants = true);
|
||||
void setCurrentRolesDefault();
|
||||
std::vector<UUID> getCurrentRoles() const;
|
||||
std::vector<UUID> getEnabledRoles() const;
|
||||
@ -1347,7 +1350,7 @@ private:
|
||||
|
||||
void setCurrentProfilesWithLock(const SettingsProfilesInfo & profiles_info, bool check_constraints, const std::lock_guard<ContextSharedMutex> & lock);
|
||||
|
||||
void setCurrentRolesWithLock(const std::vector<UUID> & current_roles_, const std::lock_guard<ContextSharedMutex> & lock);
|
||||
void setCurrentRolesWithLock(const std::vector<UUID> & new_current_roles, const std::lock_guard<ContextSharedMutex> & lock);
|
||||
|
||||
void setSettingWithLock(std::string_view name, const String & value, const std::lock_guard<ContextSharedMutex> & lock);
|
||||
|
||||
@ -1380,6 +1383,7 @@ private:
|
||||
void initGlobal();
|
||||
|
||||
void setUserID(const UUID & user_id_);
|
||||
void setCurrentRolesImpl(const std::vector<UUID> & new_current_roles, bool throw_if_not_granted, bool skip_if_not_granted, const std::shared_ptr<const User> & user);
|
||||
|
||||
template <typename... Args>
|
||||
void checkAccessImpl(const Args &... args) const;
|
||||
|
265
src/Server/HTTP/authenticateUserByHTTP.cpp
Normal file
265
src/Server/HTTP/authenticateUserByHTTP.cpp
Normal file
@ -0,0 +1,265 @@
|
||||
#include <Server/HTTP/authenticateUserByHTTP.h>
|
||||
|
||||
#include <Access/Authentication.h>
|
||||
#include <Access/Common/SSLCertificateSubjects.h>
|
||||
#include <Access/Credentials.h>
|
||||
#include <Access/ExternalAuthenticators.h>
|
||||
#include <Common/Base64.h>
|
||||
#include <Server/HTTP/HTTPServerRequest.h>
|
||||
#include <Server/HTTP/HTMLForm.h>
|
||||
#include <Server/HTTP/HTTPServerResponse.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/Session.h>
|
||||
|
||||
#include <Poco/Net/HTTPBasicCredentials.h>
|
||||
|
||||
#if USE_SSL
|
||||
#include <Poco/Net/X509Certificate.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int AUTHENTICATION_FAILED;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
}
|
||||
|
||||
|
||||
namespace
|
||||
{
|
||||
/// Throws an exception that multiple authorization schemes are used simultaneously.
|
||||
[[noreturn]] void throwMultipleAuthenticationMethods(std::string_view method1, std::string_view method2)
|
||||
{
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: it is not allowed to use {} and {} simultaneously", method1, method2);
|
||||
}
|
||||
|
||||
/// Checks that a specified user name is not empty, and throws an exception if it's empty.
|
||||
void checkUserNameNotEmpty(const String & user_name, std::string_view method)
|
||||
{
|
||||
if (user_name.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Got an empty user name from {}", method);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
bool authenticateUserByHTTP(
|
||||
const HTTPServerRequest & request,
|
||||
const HTMLForm & params,
|
||||
HTTPServerResponse & response,
|
||||
Session & session,
|
||||
std::unique_ptr<Credentials> & request_credentials,
|
||||
ContextPtr global_context,
|
||||
LoggerPtr log)
|
||||
{
|
||||
/// Get the credentials created by the previous call of authenticateUserByHTTP() while handling the previous HTTP request.
|
||||
auto current_credentials = std::move(request_credentials);
|
||||
|
||||
/// The user and password can be passed by headers (similar to X-Auth-*),
|
||||
/// which is used by load balancers to pass authentication information.
|
||||
std::string user = request.get("X-ClickHouse-User", "");
|
||||
std::string password = request.get("X-ClickHouse-Key", "");
|
||||
std::string quota_key = request.get("X-ClickHouse-Quota", "");
|
||||
bool has_auth_headers = !user.empty() || !password.empty();
|
||||
|
||||
/// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name
|
||||
/// extracted from the SSL certificate used for this connection instead of checking password.
|
||||
bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on");
|
||||
|
||||
/// User name and password can be passed using HTTP Basic auth or query parameters
|
||||
/// (both methods are insecure).
|
||||
bool has_http_credentials = request.hasCredentials();
|
||||
bool has_credentials_in_query_params = params.has("user") || params.has("password");
|
||||
|
||||
std::string spnego_challenge;
|
||||
SSLCertificateSubjects certificate_subjects;
|
||||
|
||||
if (has_ssl_certificate_auth)
|
||||
{
|
||||
#if USE_SSL
|
||||
/// For SSL certificate authentication we extract the user name from the "X-ClickHouse-User" HTTP header.
|
||||
checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers");
|
||||
|
||||
/// It is prohibited to mix different authorization schemes.
|
||||
if (!password.empty())
|
||||
throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via password");
|
||||
if (has_http_credentials)
|
||||
throwMultipleAuthenticationMethods("SSL certificate authentication", "Authorization HTTP header");
|
||||
if (has_credentials_in_query_params)
|
||||
throwMultipleAuthenticationMethods("SSL certificate authentication", "authentication via parameters");
|
||||
|
||||
if (request.havePeerCertificate())
|
||||
certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate());
|
||||
|
||||
if (certificate_subjects.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name");
|
||||
#else
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"SSL certificate authentication disabled because ClickHouse was built without SSL library");
|
||||
#endif
|
||||
}
|
||||
else if (has_auth_headers)
|
||||
{
|
||||
checkUserNameNotEmpty(user, "X-ClickHouse HTTP headers");
|
||||
|
||||
/// It is prohibited to mix different authorization schemes.
|
||||
if (has_http_credentials)
|
||||
throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "Authorization HTTP header");
|
||||
if (has_credentials_in_query_params)
|
||||
throwMultipleAuthenticationMethods("X-ClickHouse HTTP headers", "authentication via parameters");
|
||||
}
|
||||
else if (has_http_credentials)
|
||||
{
|
||||
/// It is prohibited to mix different authorization schemes.
|
||||
if (has_credentials_in_query_params)
|
||||
throwMultipleAuthenticationMethods("Authorization HTTP header", "authentication via parameters");
|
||||
|
||||
std::string scheme;
|
||||
std::string auth_info;
|
||||
request.getCredentials(scheme, auth_info);
|
||||
|
||||
if (Poco::icompare(scheme, "Basic") == 0)
|
||||
{
|
||||
Poco::Net::HTTPBasicCredentials credentials(auth_info);
|
||||
user = credentials.getUsername();
|
||||
password = credentials.getPassword();
|
||||
checkUserNameNotEmpty(user, "Authorization HTTP header");
|
||||
}
|
||||
else if (Poco::icompare(scheme, "Negotiate") == 0)
|
||||
{
|
||||
spnego_challenge = auth_info;
|
||||
|
||||
if (spnego_challenge.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: SPNEGO challenge is empty");
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If the user name is not set we assume it's the 'default' user.
|
||||
user = params.get("user", "default");
|
||||
password = params.get("password", "");
|
||||
checkUserNameNotEmpty(user, "authentication via parameters");
|
||||
}
|
||||
|
||||
if (!certificate_subjects.empty())
|
||||
{
|
||||
chassert(!user.empty());
|
||||
if (!current_credentials)
|
||||
current_credentials = std::make_unique<SSLCertificateCredentials>(user, std::move(certificate_subjects));
|
||||
|
||||
auto * certificate_credentials = dynamic_cast<SSLCertificateCredentials *>(current_credentials.get());
|
||||
if (!certificate_credentials)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected SSL certificate authorization scheme");
|
||||
}
|
||||
else if (!spnego_challenge.empty())
|
||||
{
|
||||
if (!current_credentials)
|
||||
current_credentials = global_context->makeGSSAcceptorContext();
|
||||
|
||||
auto * gss_acceptor_context = dynamic_cast<GSSAcceptorContext *>(current_credentials.get());
|
||||
if (!gss_acceptor_context)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected");
|
||||
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wunreachable-code"
|
||||
const auto spnego_response = base64Encode(gss_acceptor_context->processToken(base64Decode(spnego_challenge), log));
|
||||
#pragma clang diagnostic pop
|
||||
|
||||
if (!spnego_response.empty())
|
||||
response.set("WWW-Authenticate", "Negotiate " + spnego_response);
|
||||
|
||||
if (!gss_acceptor_context->isFailed() && !gss_acceptor_context->isReady())
|
||||
{
|
||||
if (spnego_response.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: 'Negotiate' HTTP Authorization failure");
|
||||
|
||||
response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED);
|
||||
response.send();
|
||||
/// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request.
|
||||
request_credentials = std::move(current_credentials);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else // I.e., now using user name and password strings ("Basic").
|
||||
{
|
||||
if (!current_credentials)
|
||||
current_credentials = std::make_unique<BasicCredentials>();
|
||||
|
||||
auto * basic_credentials = dynamic_cast<BasicCredentials *>(current_credentials.get());
|
||||
if (!basic_credentials)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme");
|
||||
|
||||
chassert(!user.empty());
|
||||
basic_credentials->setUserName(user);
|
||||
basic_credentials->setPassword(password);
|
||||
}
|
||||
|
||||
if (params.has("quota_key"))
|
||||
{
|
||||
if (!quota_key.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Invalid authentication: it is not allowed "
|
||||
"to use quota key as HTTP header and as parameter simultaneously");
|
||||
|
||||
quota_key = params.get("quota_key");
|
||||
}
|
||||
|
||||
/// Set client info. It will be used for quota accounting parameters in 'setUser' method.
|
||||
|
||||
session.setHTTPClientInfo(request);
|
||||
session.setQuotaClientKey(quota_key);
|
||||
|
||||
/// Extract the last entry from comma separated list of forwarded_for addresses.
|
||||
/// Only the last proxy can be trusted (if any).
|
||||
String forwarded_address = session.getClientInfo().getLastForwardedFor();
|
||||
try
|
||||
{
|
||||
if (!forwarded_address.empty() && global_context->getConfigRef().getBool("auth_use_forwarded_address", false))
|
||||
session.authenticate(*current_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port()));
|
||||
else
|
||||
session.authenticate(*current_credentials, request.clientAddress());
|
||||
}
|
||||
catch (const Authentication::Require<BasicCredentials> & required_credentials)
|
||||
{
|
||||
current_credentials = std::make_unique<BasicCredentials>();
|
||||
|
||||
if (required_credentials.getRealm().empty())
|
||||
response.set("WWW-Authenticate", "Basic");
|
||||
else
|
||||
response.set("WWW-Authenticate", "Basic realm=\"" + required_credentials.getRealm() + "\"");
|
||||
|
||||
response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED);
|
||||
response.send();
|
||||
/// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request.
|
||||
request_credentials = std::move(current_credentials);
|
||||
return false;
|
||||
}
|
||||
catch (const Authentication::Require<GSSAcceptorContext> & required_credentials)
|
||||
{
|
||||
current_credentials = global_context->makeGSSAcceptorContext();
|
||||
|
||||
if (required_credentials.getRealm().empty())
|
||||
response.set("WWW-Authenticate", "Negotiate");
|
||||
else
|
||||
response.set("WWW-Authenticate", "Negotiate realm=\"" + required_credentials.getRealm() + "\"");
|
||||
|
||||
response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED);
|
||||
response.send();
|
||||
/// Keep the credentials for next HTTP request. A client can handle HTTP_UNAUTHORIZED and send us more credentials with the next HTTP request.
|
||||
request_credentials = std::move(current_credentials);
|
||||
return false;
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
31
src/Server/HTTP/authenticateUserByHTTP.h
Normal file
31
src/Server/HTTP/authenticateUserByHTTP.h
Normal file
@ -0,0 +1,31 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class HTTPServerRequest;
|
||||
class HTMLForm;
|
||||
class HTTPServerResponse;
|
||||
class Session;
|
||||
class Credentials;
|
||||
|
||||
/// Authenticates a user via HTTP protocol and initializes a session.
|
||||
/// Usually retrieves the name and the password for that user from either the request's headers or from the query parameters.
|
||||
/// Returns true when the user successfully authenticated,
|
||||
/// the session instance will be configured accordingly, and the request_credentials instance will be dropped.
|
||||
/// Returns false when the user is not authenticated yet, and the HTTP_UNAUTHORIZED response is sent with the "WWW-Authenticate" header,
|
||||
/// in this case the `request_credentials` instance must be preserved until the next request or until any exception.
|
||||
/// Throws an exception if authentication failed.
|
||||
bool authenticateUserByHTTP(
|
||||
const HTTPServerRequest & request,
|
||||
const HTMLForm & params,
|
||||
HTTPServerResponse & response,
|
||||
Session & session,
|
||||
std::unique_ptr<Credentials> & request_credentials,
|
||||
ContextPtr global_context,
|
||||
LoggerPtr log);
|
||||
|
||||
}
|
158
src/Server/HTTP/exceptionCodeToHTTPStatus.cpp
Normal file
158
src/Server/HTTP/exceptionCodeToHTTPStatus.cpp
Normal file
@ -0,0 +1,158 @@
|
||||
#include <Server/HTTP/exceptionCodeToHTTPStatus.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int CANNOT_PARSE_TEXT;
|
||||
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
||||
extern const int CANNOT_PARSE_QUOTED_STRING;
|
||||
extern const int CANNOT_PARSE_DATE;
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_NUMBER;
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
extern const int CANNOT_PARSE_IPV4;
|
||||
extern const int CANNOT_PARSE_IPV6;
|
||||
extern const int CANNOT_PARSE_UUID;
|
||||
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
||||
extern const int CANNOT_SCHEDULE_TASK;
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_COMPILE_REGEXP;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int THERE_IS_NO_COLUMN;
|
||||
extern const int UNKNOWN_ELEMENT_IN_AST;
|
||||
extern const int UNKNOWN_TYPE_OF_AST_NODE;
|
||||
extern const int TOO_DEEP_AST;
|
||||
extern const int TOO_BIG_AST;
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE;
|
||||
|
||||
extern const int SYNTAX_ERROR;
|
||||
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int TYPE_MISMATCH;
|
||||
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int UNKNOWN_FUNCTION;
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
extern const int UNKNOWN_TYPE;
|
||||
extern const int UNKNOWN_STORAGE;
|
||||
extern const int UNKNOWN_DATABASE;
|
||||
extern const int UNKNOWN_SETTING;
|
||||
extern const int UNKNOWN_DIRECTION_OF_SORTING;
|
||||
extern const int UNKNOWN_AGGREGATE_FUNCTION;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int UNKNOWN_DATABASE_ENGINE;
|
||||
extern const int UNKNOWN_TYPE_OF_QUERY;
|
||||
extern const int UNKNOWN_ROLE;
|
||||
|
||||
extern const int QUERY_IS_TOO_LARGE;
|
||||
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
|
||||
extern const int UNKNOWN_USER;
|
||||
extern const int WRONG_PASSWORD;
|
||||
extern const int REQUIRED_PASSWORD;
|
||||
extern const int AUTHENTICATION_FAILED;
|
||||
extern const int SET_NON_GRANTED_ROLE;
|
||||
|
||||
extern const int HTTP_LENGTH_REQUIRED;
|
||||
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
}
|
||||
|
||||
|
||||
Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code)
|
||||
{
|
||||
using namespace Poco::Net;
|
||||
|
||||
if (exception_code == ErrorCodes::REQUIRED_PASSWORD)
|
||||
{
|
||||
return HTTPResponse::HTTP_UNAUTHORIZED;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::UNKNOWN_USER ||
|
||||
exception_code == ErrorCodes::WRONG_PASSWORD ||
|
||||
exception_code == ErrorCodes::AUTHENTICATION_FAILED ||
|
||||
exception_code == ErrorCodes::SET_NON_GRANTED_ROLE)
|
||||
{
|
||||
return HTTPResponse::HTTP_FORBIDDEN;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::BAD_ARGUMENTS ||
|
||||
exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_TEXT ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_DATE ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_DATETIME ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_NUMBER ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_IPV4 ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_IPV6 ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_UUID ||
|
||||
exception_code == ErrorCodes::DUPLICATE_COLUMN ||
|
||||
exception_code == ErrorCodes::ILLEGAL_COLUMN ||
|
||||
exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST ||
|
||||
exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE ||
|
||||
exception_code == ErrorCodes::THERE_IS_NO_COLUMN ||
|
||||
exception_code == ErrorCodes::TOO_DEEP_AST ||
|
||||
exception_code == ErrorCodes::TOO_BIG_AST ||
|
||||
exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE ||
|
||||
exception_code == ErrorCodes::SYNTAX_ERROR ||
|
||||
exception_code == ErrorCodes::INCORRECT_DATA ||
|
||||
exception_code == ErrorCodes::TYPE_MISMATCH ||
|
||||
exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE)
|
||||
{
|
||||
return HTTPResponse::HTTP_BAD_REQUEST;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::UNKNOWN_TABLE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_FUNCTION ||
|
||||
exception_code == ErrorCodes::UNKNOWN_IDENTIFIER ||
|
||||
exception_code == ErrorCodes::UNKNOWN_TYPE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_STORAGE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_DATABASE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_SETTING ||
|
||||
exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING ||
|
||||
exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION ||
|
||||
exception_code == ErrorCodes::UNKNOWN_FORMAT ||
|
||||
exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY ||
|
||||
exception_code == ErrorCodes::UNKNOWN_ROLE)
|
||||
{
|
||||
return HTTPResponse::HTTP_NOT_FOUND;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE)
|
||||
{
|
||||
return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::NOT_IMPLEMENTED)
|
||||
{
|
||||
return HTTPResponse::HTTP_NOT_IMPLEMENTED;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::SOCKET_TIMEOUT ||
|
||||
exception_code == ErrorCodes::CANNOT_OPEN_FILE)
|
||||
{
|
||||
return HTTPResponse::HTTP_SERVICE_UNAVAILABLE;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED)
|
||||
{
|
||||
return HTTPResponse::HTTP_LENGTH_REQUIRED;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::TIMEOUT_EXCEEDED)
|
||||
{
|
||||
return HTTPResponse::HTTP_REQUEST_TIMEOUT;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK)
|
||||
{
|
||||
return HTTPResponse::HTTP_SERVICE_UNAVAILABLE;
|
||||
}
|
||||
|
||||
return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR;
|
||||
}
|
||||
|
||||
}
|
11
src/Server/HTTP/exceptionCodeToHTTPStatus.h
Normal file
11
src/Server/HTTP/exceptionCodeToHTTPStatus.h
Normal file
@ -0,0 +1,11 @@
|
||||
#pragma once
|
||||
#include <Poco/Net/HTTPResponse.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Converts Exception code to HTTP status code.
|
||||
Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code);
|
||||
|
||||
}
|
80
src/Server/HTTP/sendExceptionToHTTPClient.cpp
Normal file
80
src/Server/HTTP/sendExceptionToHTTPClient.cpp
Normal file
@ -0,0 +1,80 @@
|
||||
#include <Server/HTTP/sendExceptionToHTTPClient.h>
|
||||
|
||||
#include <Server/HTTP/HTTPServerRequest.h>
|
||||
#include <Server/HTTP/WriteBufferFromHTTPServerResponse.h>
|
||||
#include <Server/HTTP/exceptionCodeToHTTPStatus.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int HTTP_LENGTH_REQUIRED;
|
||||
extern const int REQUIRED_PASSWORD;
|
||||
}
|
||||
|
||||
|
||||
void sendExceptionToHTTPClient(
|
||||
const String & exception_message,
|
||||
int exception_code,
|
||||
HTTPServerRequest & request,
|
||||
HTTPServerResponse & response,
|
||||
WriteBufferFromHTTPServerResponse * out,
|
||||
LoggerPtr log)
|
||||
{
|
||||
setHTTPResponseStatusAndHeadersForException(exception_code, request, response, out, log);
|
||||
|
||||
if (!out)
|
||||
{
|
||||
/// If nothing was sent yet.
|
||||
WriteBufferFromHTTPServerResponse out_for_message{response, request.getMethod() == HTTPRequest::HTTP_HEAD, DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT};
|
||||
|
||||
out_for_message.writeln(exception_message);
|
||||
out_for_message.finalize();
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If buffer has data, and that data wasn't sent yet, then no need to send that data
|
||||
bool data_sent = (out->count() != out->offset());
|
||||
|
||||
if (!data_sent)
|
||||
out->position() = out->buffer().begin();
|
||||
|
||||
out->writeln(exception_message);
|
||||
out->finalize();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void setHTTPResponseStatusAndHeadersForException(
|
||||
int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, WriteBufferFromHTTPServerResponse * out, LoggerPtr log)
|
||||
{
|
||||
if (out)
|
||||
out->setExceptionCode(exception_code);
|
||||
else
|
||||
response.set("X-ClickHouse-Exception-Code", toString<int>(exception_code));
|
||||
|
||||
/// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body
|
||||
/// to avoid reading part of the current request body in the next request.
|
||||
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive()
|
||||
&& exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (!request.getStream().eof())
|
||||
request.getStream().ignoreAll();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Cannot read remaining request body during exception handling");
|
||||
response.setKeepAlive(false);
|
||||
}
|
||||
}
|
||||
|
||||
if (exception_code == ErrorCodes::REQUIRED_PASSWORD)
|
||||
response.requireAuthentication("ClickHouse server HTTP API");
|
||||
else
|
||||
response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code));
|
||||
}
|
||||
}
|
27
src/Server/HTTP/sendExceptionToHTTPClient.h
Normal file
27
src/Server/HTTP/sendExceptionToHTTPClient.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/logger_useful.h>
|
||||
#include <base/types.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
class HTTPServerRequest;
|
||||
class HTTPServerResponse;
|
||||
class WriteBufferFromHTTPServerResponse;
|
||||
|
||||
/// Sends an exception to HTTP client. This function doesn't handle its own exceptions so it needs to be wrapped in try-catch.
|
||||
/// Argument `out` may be either created from `response` or be nullptr (if it wasn't created before the exception).
|
||||
void sendExceptionToHTTPClient(
|
||||
const String & exception_message,
|
||||
int exception_code,
|
||||
HTTPServerRequest & request,
|
||||
HTTPServerResponse & response,
|
||||
WriteBufferFromHTTPServerResponse * out,
|
||||
LoggerPtr log);
|
||||
|
||||
/// Sets "X-ClickHouse-Exception-Code" header and the correspondent HTTP status in the response for an exception.
|
||||
/// This is a part of what sendExceptionToHTTPClient() does.
|
||||
void setHTTPResponseStatusAndHeadersForException(
|
||||
int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, WriteBufferFromHTTPServerResponse * out, LoggerPtr log);
|
||||
}
|
24
src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp
Normal file
24
src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.cpp
Normal file
@ -0,0 +1,24 @@
|
||||
#include <Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h>
|
||||
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Server/HTTP/HTTPServerRequest.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void setReadOnlyIfHTTPMethodIdempotent(ContextMutablePtr context, const String & http_method)
|
||||
{
|
||||
/// Anything else beside HTTP POST should be readonly queries.
|
||||
if (http_method != HTTPServerRequest::HTTP_POST)
|
||||
{
|
||||
/// 'readonly' setting values mean:
|
||||
/// readonly = 0 - any query is allowed, client can change any setting.
|
||||
/// readonly = 1 - only readonly queries are allowed, client can't change settings.
|
||||
/// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'.
|
||||
if (context->getSettingsRef().readonly == 0)
|
||||
context->setSetting("readonly", 2);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
12
src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h
Normal file
12
src/Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h
Normal file
@ -0,0 +1,12 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/Context_fwd.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Sets readonly = 2 if the current HTTP method is not HTTP POST and if readonly is not set already.
|
||||
void setReadOnlyIfHTTPMethodIdempotent(ContextMutablePtr context, const String & http_method);
|
||||
|
||||
}
|
@ -1,11 +1,6 @@
|
||||
#include <Server/HTTPHandler.h>
|
||||
|
||||
#include <Access/Authentication.h>
|
||||
#include <Access/Credentials.h>
|
||||
#include <Access/AccessControl.h>
|
||||
#include <Access/ExternalAuthenticators.h>
|
||||
#include <Access/Role.h>
|
||||
#include <Access/User.h>
|
||||
#include <Compression/CompressedReadBuffer.h>
|
||||
#include <Compression/CompressedWriteBuffer.h>
|
||||
#include <Core/ExternalTable.h>
|
||||
@ -37,20 +32,14 @@
|
||||
#include <base/getFQDNOrHostName.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <Server/HTTP/HTTPResponse.h>
|
||||
#include <Server/HTTP/authenticateUserByHTTP.h>
|
||||
#include <Server/HTTP/sendExceptionToHTTPClient.h>
|
||||
#include <Server/HTTP/setReadOnlyIfHTTPMethodIdempotent.h>
|
||||
#include <boost/container/flat_set.hpp>
|
||||
|
||||
#include <Access/Common/SSLCertificateSubjects.h>
|
||||
#include "config.h"
|
||||
|
||||
#include <Poco/Base64Decoder.h>
|
||||
#include <Poco/Base64Encoder.h>
|
||||
#include <Poco/Net/HTTPBasicCredentials.h>
|
||||
#include <Poco/Net/HTTPMessage.h>
|
||||
#include <Poco/Net/HTTPStream.h>
|
||||
#include <Poco/MemoryStream.h>
|
||||
#include <Poco/StreamCopier.h>
|
||||
#include <Poco/String.h>
|
||||
#include <Poco/Net/SocketAddress.h>
|
||||
|
||||
#include "config.h"
|
||||
|
||||
#include <algorithm>
|
||||
#include <chrono>
|
||||
@ -60,78 +49,19 @@
|
||||
#include <unordered_map>
|
||||
#include <utility>
|
||||
|
||||
#if USE_SSL
|
||||
#include <Poco/Net/X509Certificate.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int CANNOT_COMPILE_REGEXP;
|
||||
extern const int CANNOT_OPEN_FILE;
|
||||
extern const int CANNOT_PARSE_TEXT;
|
||||
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
||||
extern const int CANNOT_PARSE_QUOTED_STRING;
|
||||
extern const int CANNOT_PARSE_DATE;
|
||||
extern const int CANNOT_PARSE_DATETIME;
|
||||
extern const int CANNOT_PARSE_NUMBER;
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
extern const int CANNOT_PARSE_IPV4;
|
||||
extern const int CANNOT_PARSE_IPV6;
|
||||
extern const int CANNOT_PARSE_UUID;
|
||||
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
|
||||
extern const int CANNOT_SCHEDULE_TASK;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int THERE_IS_NO_COLUMN;
|
||||
extern const int UNKNOWN_ELEMENT_IN_AST;
|
||||
extern const int UNKNOWN_TYPE_OF_AST_NODE;
|
||||
extern const int TOO_DEEP_AST;
|
||||
extern const int TOO_BIG_AST;
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE;
|
||||
|
||||
extern const int SYNTAX_ERROR;
|
||||
|
||||
extern const int INCORRECT_DATA;
|
||||
extern const int TYPE_MISMATCH;
|
||||
|
||||
extern const int UNKNOWN_TABLE;
|
||||
extern const int UNKNOWN_FUNCTION;
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
extern const int UNKNOWN_TYPE;
|
||||
extern const int UNKNOWN_STORAGE;
|
||||
extern const int UNKNOWN_DATABASE;
|
||||
extern const int UNKNOWN_SETTING;
|
||||
extern const int UNKNOWN_DIRECTION_OF_SORTING;
|
||||
extern const int UNKNOWN_AGGREGATE_FUNCTION;
|
||||
extern const int UNKNOWN_FORMAT;
|
||||
extern const int UNKNOWN_DATABASE_ENGINE;
|
||||
extern const int UNKNOWN_TYPE_OF_QUERY;
|
||||
extern const int UNKNOWN_ROLE;
|
||||
extern const int NO_ELEMENTS_IN_CONFIG;
|
||||
|
||||
extern const int QUERY_IS_TOO_LARGE;
|
||||
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int SOCKET_TIMEOUT;
|
||||
|
||||
extern const int UNKNOWN_USER;
|
||||
extern const int WRONG_PASSWORD;
|
||||
extern const int REQUIRED_PASSWORD;
|
||||
extern const int AUTHENTICATION_FAILED;
|
||||
extern const int SET_NON_GRANTED_ROLE;
|
||||
|
||||
extern const int INVALID_SESSION_TIMEOUT;
|
||||
extern const int HTTP_LENGTH_REQUIRED;
|
||||
extern const int SUPPORT_IS_DISABLED;
|
||||
|
||||
extern const int TIMEOUT_EXCEEDED;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -173,115 +103,6 @@ void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::Laye
|
||||
}
|
||||
}
|
||||
|
||||
static String base64Decode(const String & encoded)
|
||||
{
|
||||
String decoded;
|
||||
Poco::MemoryInputStream istr(encoded.data(), encoded.size());
|
||||
Poco::Base64Decoder decoder(istr);
|
||||
Poco::StreamCopier::copyToString(decoder, decoded);
|
||||
return decoded;
|
||||
}
|
||||
|
||||
static String base64Encode(const String & decoded)
|
||||
{
|
||||
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
ostr.exceptions(std::ios::failbit);
|
||||
Poco::Base64Encoder encoder(ostr);
|
||||
encoder.rdbuf()->setLineLength(0);
|
||||
encoder << decoded;
|
||||
encoder.close();
|
||||
return ostr.str();
|
||||
}
|
||||
|
||||
static Poco::Net::HTTPResponse::HTTPStatus exceptionCodeToHTTPStatus(int exception_code)
|
||||
{
|
||||
using namespace Poco::Net;
|
||||
|
||||
if (exception_code == ErrorCodes::REQUIRED_PASSWORD)
|
||||
{
|
||||
return HTTPResponse::HTTP_UNAUTHORIZED;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::UNKNOWN_USER ||
|
||||
exception_code == ErrorCodes::WRONG_PASSWORD ||
|
||||
exception_code == ErrorCodes::AUTHENTICATION_FAILED ||
|
||||
exception_code == ErrorCodes::SET_NON_GRANTED_ROLE)
|
||||
{
|
||||
return HTTPResponse::HTTP_FORBIDDEN;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::BAD_ARGUMENTS ||
|
||||
exception_code == ErrorCodes::CANNOT_COMPILE_REGEXP ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_TEXT ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_DATE ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_DATETIME ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_NUMBER ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_IPV4 ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_IPV6 ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED ||
|
||||
exception_code == ErrorCodes::CANNOT_PARSE_UUID ||
|
||||
exception_code == ErrorCodes::DUPLICATE_COLUMN ||
|
||||
exception_code == ErrorCodes::ILLEGAL_COLUMN ||
|
||||
exception_code == ErrorCodes::UNKNOWN_ELEMENT_IN_AST ||
|
||||
exception_code == ErrorCodes::UNKNOWN_TYPE_OF_AST_NODE ||
|
||||
exception_code == ErrorCodes::THERE_IS_NO_COLUMN ||
|
||||
exception_code == ErrorCodes::TOO_DEEP_AST ||
|
||||
exception_code == ErrorCodes::TOO_BIG_AST ||
|
||||
exception_code == ErrorCodes::UNEXPECTED_AST_STRUCTURE ||
|
||||
exception_code == ErrorCodes::SYNTAX_ERROR ||
|
||||
exception_code == ErrorCodes::INCORRECT_DATA ||
|
||||
exception_code == ErrorCodes::TYPE_MISMATCH ||
|
||||
exception_code == ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE)
|
||||
{
|
||||
return HTTPResponse::HTTP_BAD_REQUEST;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::UNKNOWN_TABLE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_FUNCTION ||
|
||||
exception_code == ErrorCodes::UNKNOWN_IDENTIFIER ||
|
||||
exception_code == ErrorCodes::UNKNOWN_TYPE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_STORAGE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_DATABASE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_SETTING ||
|
||||
exception_code == ErrorCodes::UNKNOWN_DIRECTION_OF_SORTING ||
|
||||
exception_code == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION ||
|
||||
exception_code == ErrorCodes::UNKNOWN_FORMAT ||
|
||||
exception_code == ErrorCodes::UNKNOWN_DATABASE_ENGINE ||
|
||||
exception_code == ErrorCodes::UNKNOWN_TYPE_OF_QUERY ||
|
||||
exception_code == ErrorCodes::UNKNOWN_ROLE)
|
||||
{
|
||||
return HTTPResponse::HTTP_NOT_FOUND;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::QUERY_IS_TOO_LARGE)
|
||||
{
|
||||
return HTTPResponse::HTTP_REQUESTENTITYTOOLARGE;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::NOT_IMPLEMENTED)
|
||||
{
|
||||
return HTTPResponse::HTTP_NOT_IMPLEMENTED;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::SOCKET_TIMEOUT ||
|
||||
exception_code == ErrorCodes::CANNOT_OPEN_FILE)
|
||||
{
|
||||
return HTTPResponse::HTTP_SERVICE_UNAVAILABLE;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::HTTP_LENGTH_REQUIRED)
|
||||
{
|
||||
return HTTPResponse::HTTP_LENGTH_REQUIRED;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::TIMEOUT_EXCEEDED)
|
||||
{
|
||||
return HTTPResponse::HTTP_REQUEST_TIMEOUT;
|
||||
}
|
||||
else if (exception_code == ErrorCodes::CANNOT_SCHEDULE_TASK)
|
||||
{
|
||||
return HTTPResponse::HTTP_SERVICE_UNAVAILABLE;
|
||||
}
|
||||
|
||||
return HTTPResponse::HTTP_INTERNAL_SERVER_ERROR;
|
||||
}
|
||||
|
||||
|
||||
static std::chrono::steady_clock::duration parseSessionTimeout(
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const HTMLForm & params)
|
||||
@ -358,204 +179,9 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name, const HTTP
|
||||
HTTPHandler::~HTTPHandler() = default;
|
||||
|
||||
|
||||
bool HTTPHandler::authenticateUser(
|
||||
HTTPServerRequest & request,
|
||||
HTMLForm & params,
|
||||
HTTPServerResponse & response)
|
||||
bool HTTPHandler::authenticateUser(HTTPServerRequest & request, HTMLForm & params, HTTPServerResponse & response)
|
||||
{
|
||||
using namespace Poco::Net;
|
||||
|
||||
/// The user and password can be passed by headers (similar to X-Auth-*),
|
||||
/// which is used by load balancers to pass authentication information.
|
||||
std::string user = request.get("X-ClickHouse-User", "");
|
||||
std::string password = request.get("X-ClickHouse-Key", "");
|
||||
std::string quota_key = request.get("X-ClickHouse-Quota", "");
|
||||
|
||||
/// The header 'X-ClickHouse-SSL-Certificate-Auth: on' enables checking the common name
|
||||
/// extracted from the SSL certificate used for this connection instead of checking password.
|
||||
bool has_ssl_certificate_auth = (request.get("X-ClickHouse-SSL-Certificate-Auth", "") == "on");
|
||||
bool has_auth_headers = !user.empty() || !password.empty() || has_ssl_certificate_auth;
|
||||
|
||||
/// User name and password can be passed using HTTP Basic auth or query parameters
|
||||
/// (both methods are insecure).
|
||||
bool has_http_credentials = request.hasCredentials();
|
||||
bool has_credentials_in_query_params = params.has("user") || params.has("password");
|
||||
|
||||
std::string spnego_challenge;
|
||||
SSLCertificateSubjects certificate_subjects;
|
||||
|
||||
if (has_auth_headers)
|
||||
{
|
||||
/// It is prohibited to mix different authorization schemes.
|
||||
if (has_http_credentials)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: it is not allowed "
|
||||
"to use SSL certificate authentication and Authorization HTTP header simultaneously");
|
||||
if (has_credentials_in_query_params)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: it is not allowed "
|
||||
"to use SSL certificate authentication and authentication via parameters simultaneously simultaneously");
|
||||
|
||||
if (has_ssl_certificate_auth)
|
||||
{
|
||||
#if USE_SSL
|
||||
if (!password.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: it is not allowed "
|
||||
"to use SSL certificate authentication and authentication via password simultaneously");
|
||||
|
||||
if (request.havePeerCertificate())
|
||||
certificate_subjects = extractSSLCertificateSubjects(request.peerCertificate());
|
||||
|
||||
if (certificate_subjects.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: SSL certificate authentication requires nonempty certificate's Common Name or Subject Alternative Name");
|
||||
#else
|
||||
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED,
|
||||
"SSL certificate authentication disabled because ClickHouse was built without SSL library");
|
||||
#endif
|
||||
}
|
||||
}
|
||||
else if (has_http_credentials)
|
||||
{
|
||||
/// It is prohibited to mix different authorization schemes.
|
||||
if (has_credentials_in_query_params)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED,
|
||||
"Invalid authentication: it is not allowed "
|
||||
"to use Authorization HTTP header and authentication via parameters simultaneously");
|
||||
|
||||
std::string scheme;
|
||||
std::string auth_info;
|
||||
request.getCredentials(scheme, auth_info);
|
||||
|
||||
if (Poco::icompare(scheme, "Basic") == 0)
|
||||
{
|
||||
HTTPBasicCredentials credentials(auth_info);
|
||||
user = credentials.getUsername();
|
||||
password = credentials.getPassword();
|
||||
}
|
||||
else if (Poco::icompare(scheme, "Negotiate") == 0)
|
||||
{
|
||||
spnego_challenge = auth_info;
|
||||
|
||||
if (spnego_challenge.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: SPNEGO challenge is empty");
|
||||
}
|
||||
else
|
||||
{
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: '{}' HTTP Authorization scheme is not supported", scheme);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If the user name is not set we assume it's the 'default' user.
|
||||
user = params.get("user", "default");
|
||||
password = params.get("password", "");
|
||||
}
|
||||
|
||||
if (!certificate_subjects.empty())
|
||||
{
|
||||
if (!request_credentials)
|
||||
request_credentials = std::make_unique<SSLCertificateCredentials>(user, std::move(certificate_subjects));
|
||||
|
||||
auto * certificate_credentials = dynamic_cast<SSLCertificateCredentials *>(request_credentials.get());
|
||||
if (!certificate_credentials)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected SSL certificate authorization scheme");
|
||||
}
|
||||
else if (!spnego_challenge.empty())
|
||||
{
|
||||
if (!request_credentials)
|
||||
request_credentials = server.context()->makeGSSAcceptorContext();
|
||||
|
||||
auto * gss_acceptor_context = dynamic_cast<GSSAcceptorContext *>(request_credentials.get());
|
||||
if (!gss_acceptor_context)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: unexpected 'Negotiate' HTTP Authorization scheme expected");
|
||||
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wunreachable-code"
|
||||
const auto spnego_response = base64Encode(gss_acceptor_context->processToken(base64Decode(spnego_challenge), log));
|
||||
#pragma clang diagnostic pop
|
||||
|
||||
if (!spnego_response.empty())
|
||||
response.set("WWW-Authenticate", "Negotiate " + spnego_response);
|
||||
|
||||
if (!gss_acceptor_context->isFailed() && !gss_acceptor_context->isReady())
|
||||
{
|
||||
if (spnego_response.empty())
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: 'Negotiate' HTTP Authorization failure");
|
||||
|
||||
response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED);
|
||||
response.send();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else // I.e., now using user name and password strings ("Basic").
|
||||
{
|
||||
if (!request_credentials)
|
||||
request_credentials = std::make_unique<BasicCredentials>();
|
||||
|
||||
auto * basic_credentials = dynamic_cast<BasicCredentials *>(request_credentials.get());
|
||||
if (!basic_credentials)
|
||||
throw Exception(ErrorCodes::AUTHENTICATION_FAILED, "Invalid authentication: expected 'Basic' HTTP Authorization scheme");
|
||||
|
||||
basic_credentials->setUserName(user);
|
||||
basic_credentials->setPassword(password);
|
||||
}
|
||||
|
||||
if (params.has("quota_key"))
|
||||
{
|
||||
if (!quota_key.empty())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Invalid authentication: it is not allowed "
|
||||
"to use quota key as HTTP header and as parameter simultaneously");
|
||||
|
||||
quota_key = params.get("quota_key");
|
||||
}
|
||||
|
||||
/// Set client info. It will be used for quota accounting parameters in 'setUser' method.
|
||||
|
||||
session->setHTTPClientInfo(request);
|
||||
session->setQuotaClientKey(quota_key);
|
||||
|
||||
/// Extract the last entry from comma separated list of forwarded_for addresses.
|
||||
/// Only the last proxy can be trusted (if any).
|
||||
String forwarded_address = session->getClientInfo().getLastForwardedFor();
|
||||
try
|
||||
{
|
||||
if (!forwarded_address.empty() && server.config().getBool("auth_use_forwarded_address", false))
|
||||
session->authenticate(*request_credentials, Poco::Net::SocketAddress(forwarded_address, request.clientAddress().port()));
|
||||
else
|
||||
session->authenticate(*request_credentials, request.clientAddress());
|
||||
}
|
||||
catch (const Authentication::Require<BasicCredentials> & required_credentials)
|
||||
{
|
||||
request_credentials = std::make_unique<BasicCredentials>();
|
||||
|
||||
if (required_credentials.getRealm().empty())
|
||||
response.set("WWW-Authenticate", "Basic");
|
||||
else
|
||||
response.set("WWW-Authenticate", "Basic realm=\"" + required_credentials.getRealm() + "\"");
|
||||
|
||||
response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED);
|
||||
response.send();
|
||||
return false;
|
||||
}
|
||||
catch (const Authentication::Require<GSSAcceptorContext> & required_credentials)
|
||||
{
|
||||
request_credentials = server.context()->makeGSSAcceptorContext();
|
||||
|
||||
if (required_credentials.getRealm().empty())
|
||||
response.set("WWW-Authenticate", "Negotiate");
|
||||
else
|
||||
response.set("WWW-Authenticate", "Negotiate realm=\"" + required_credentials.getRealm() + "\"");
|
||||
|
||||
response.setStatusAndReason(HTTPResponse::HTTP_UNAUTHORIZED);
|
||||
response.send();
|
||||
return false;
|
||||
}
|
||||
|
||||
request_credentials.reset();
|
||||
return true;
|
||||
return authenticateUserByHTTP(request, params, response, *session, request_credentials, server.context(), log);
|
||||
}
|
||||
|
||||
|
||||
@ -727,10 +353,22 @@ void HTTPHandler::processQuery(
|
||||
|
||||
std::unique_ptr<ReadBuffer> in;
|
||||
|
||||
static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role",
|
||||
"buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session"};
|
||||
auto roles = params.getAll("role");
|
||||
if (!roles.empty())
|
||||
context->setCurrentRoles(roles);
|
||||
|
||||
Names reserved_param_suffixes;
|
||||
std::string database = request.get("X-ClickHouse-Database", params.get("database", ""));
|
||||
if (!database.empty())
|
||||
context->setCurrentDatabase(database);
|
||||
|
||||
std::string default_format = request.get("X-ClickHouse-Format", params.get("default_format", ""));
|
||||
if (!default_format.empty())
|
||||
context->setDefaultFormat(default_format);
|
||||
|
||||
/// Anything else beside HTTP POST should be readonly queries.
|
||||
setReadOnlyIfHTTPMethodIdempotent(context, request.getMethod());
|
||||
|
||||
bool has_external_data = startsWith(request.getContentType(), "multipart/form-data");
|
||||
|
||||
auto param_could_be_skipped = [&] (const String & name)
|
||||
{
|
||||
@ -738,87 +376,36 @@ void HTTPHandler::processQuery(
|
||||
if (name.empty())
|
||||
return true;
|
||||
|
||||
/// Some parameters (database, default_format, everything used in the code above) do not
|
||||
/// belong to the Settings class.
|
||||
static const NameSet reserved_param_names{"compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", "role",
|
||||
"buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check", "client_protocol_version", "close_session",
|
||||
"database", "default_format"};
|
||||
|
||||
if (reserved_param_names.contains(name))
|
||||
return true;
|
||||
|
||||
for (const String & suffix : reserved_param_suffixes)
|
||||
/// For external data we also want settings.
|
||||
if (has_external_data)
|
||||
{
|
||||
if (endsWith(name, suffix))
|
||||
return true;
|
||||
/// Skip unneeded parameters to avoid confusing them later with context settings or query parameters.
|
||||
/// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings.
|
||||
static const Names reserved_param_suffixes = {"_format", "_types", "_structure"};
|
||||
for (const String & suffix : reserved_param_suffixes)
|
||||
{
|
||||
if (endsWith(name, suffix))
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
};
|
||||
|
||||
auto roles = params.getAll("role");
|
||||
if (!roles.empty())
|
||||
{
|
||||
const auto & access_control = context->getAccessControl();
|
||||
const auto & user = context->getUser();
|
||||
std::vector<UUID> roles_ids(roles.size());
|
||||
for (size_t i = 0; i < roles.size(); i++)
|
||||
{
|
||||
auto role_id = access_control.getID<Role>(roles[i]);
|
||||
if (user->granted_roles.isGranted(role_id))
|
||||
roles_ids[i] = role_id;
|
||||
else
|
||||
throw Exception(ErrorCodes::SET_NON_GRANTED_ROLE, "Role {} should be granted to set as a current", roles[i].get());
|
||||
}
|
||||
context->setCurrentRoles(roles_ids);
|
||||
}
|
||||
|
||||
/// Settings can be overridden in the query.
|
||||
/// Some parameters (database, default_format, everything used in the code above) do not
|
||||
/// belong to the Settings class.
|
||||
|
||||
/// 'readonly' setting values mean:
|
||||
/// readonly = 0 - any query is allowed, client can change any setting.
|
||||
/// readonly = 1 - only readonly queries are allowed, client can't change settings.
|
||||
/// readonly = 2 - only readonly queries are allowed, client can change any setting except 'readonly'.
|
||||
|
||||
/// In theory if initially readonly = 0, the client can change any setting and then set readonly
|
||||
/// to some other value.
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
/// Anything else beside HTTP POST should be readonly queries.
|
||||
if (request.getMethod() != HTTPServerRequest::HTTP_POST)
|
||||
{
|
||||
if (settings.readonly == 0)
|
||||
context->setSetting("readonly", 2);
|
||||
}
|
||||
|
||||
bool has_external_data = startsWith(request.getContentType(), "multipart/form-data");
|
||||
|
||||
if (has_external_data)
|
||||
{
|
||||
/// Skip unneeded parameters to avoid confusing them later with context settings or query parameters.
|
||||
reserved_param_suffixes.reserve(3);
|
||||
/// It is a bug and ambiguity with `date_time_input_format` and `low_cardinality_allow_in_native_format` formats/settings.
|
||||
reserved_param_suffixes.emplace_back("_format");
|
||||
reserved_param_suffixes.emplace_back("_types");
|
||||
reserved_param_suffixes.emplace_back("_structure");
|
||||
}
|
||||
|
||||
std::string database = request.get("X-ClickHouse-Database", "");
|
||||
std::string default_format = request.get("X-ClickHouse-Format", "");
|
||||
|
||||
SettingsChanges settings_changes;
|
||||
for (const auto & [key, value] : params)
|
||||
{
|
||||
if (key == "database")
|
||||
{
|
||||
if (database.empty())
|
||||
database = value;
|
||||
}
|
||||
else if (key == "default_format")
|
||||
{
|
||||
if (default_format.empty())
|
||||
default_format = value;
|
||||
}
|
||||
else if (param_could_be_skipped(key))
|
||||
{
|
||||
}
|
||||
else
|
||||
if (!param_could_be_skipped(key))
|
||||
{
|
||||
/// Other than query parameters are treated as settings.
|
||||
if (!customizeQueryParam(context, key, value))
|
||||
@ -826,15 +413,9 @@ void HTTPHandler::processQuery(
|
||||
}
|
||||
}
|
||||
|
||||
if (!database.empty())
|
||||
context->setCurrentDatabase(database);
|
||||
|
||||
if (!default_format.empty())
|
||||
context->setDefaultFormat(default_format);
|
||||
|
||||
/// For external data we also want settings
|
||||
context->checkSettingsConstraints(settings_changes, SettingSource::QUERY);
|
||||
context->applySettingsChanges(settings_changes);
|
||||
const auto & settings = context->getSettingsRef();
|
||||
|
||||
/// Set the query id supplied by the user, if any, and also update the OpenTelemetry fields.
|
||||
context->setCurrentQueryId(params.get("query_id", request.get("X-ClickHouse-Query-Id", "")));
|
||||
@ -936,7 +517,7 @@ void HTTPHandler::processQuery(
|
||||
{
|
||||
bool with_stacktrace = (params.getParsed<bool>("stacktrace", false) && server.config().getBool("enable_http_stacktrace", true));
|
||||
ExecutionStatus status = ExecutionStatus::fromCurrentException("", with_stacktrace);
|
||||
formatExceptionForClient(status.code, request, response, used_output);
|
||||
setHTTPResponseStatusAndHeadersForException(status.code, request, response, used_output.out_holder.get(), log);
|
||||
current_output_format.setException(status.message);
|
||||
current_output_format.finalize();
|
||||
used_output.exception_is_written = true;
|
||||
@ -970,7 +551,7 @@ void HTTPHandler::trySendExceptionToClient(
|
||||
const std::string & s, int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output)
|
||||
try
|
||||
{
|
||||
formatExceptionForClient(exception_code, request, response, used_output);
|
||||
setHTTPResponseStatusAndHeadersForException(exception_code, request, response, used_output.out_holder.get(), log);
|
||||
|
||||
if (!used_output.out_holder && !used_output.exception_is_written)
|
||||
{
|
||||
@ -1032,38 +613,6 @@ catch (...)
|
||||
used_output.cancel();
|
||||
}
|
||||
|
||||
void HTTPHandler::formatExceptionForClient(int exception_code, HTTPServerRequest & request, HTTPServerResponse & response, Output & used_output)
|
||||
{
|
||||
if (used_output.out_holder)
|
||||
used_output.out_holder->setExceptionCode(exception_code);
|
||||
else
|
||||
response.set("X-ClickHouse-Exception-Code", toString<int>(exception_code));
|
||||
|
||||
/// FIXME: make sure that no one else is reading from the same stream at the moment.
|
||||
|
||||
/// If HTTP method is POST and Keep-Alive is turned on, we should try to read the whole request body
|
||||
/// to avoid reading part of the current request body in the next request.
|
||||
if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST && response.getKeepAlive()
|
||||
&& exception_code != ErrorCodes::HTTP_LENGTH_REQUIRED)
|
||||
{
|
||||
try
|
||||
{
|
||||
if (!request.getStream().eof())
|
||||
request.getStream().ignoreAll();
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(log, "Cannot read remaining request body during exception handling");
|
||||
response.setKeepAlive(false);
|
||||
}
|
||||
}
|
||||
|
||||
if (exception_code == ErrorCodes::REQUIRED_PASSWORD)
|
||||
response.requireAuthentication("ClickHouse server HTTP API");
|
||||
else
|
||||
response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code));
|
||||
}
|
||||
|
||||
void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse & response, const ProfileEvents::Event & write_event)
|
||||
{
|
||||
setThreadName("HTTPHandler");
|
||||
|
@ -173,12 +173,6 @@ private:
|
||||
HTTPServerResponse & response,
|
||||
Output & used_output);
|
||||
|
||||
void formatExceptionForClient(
|
||||
int exception_code,
|
||||
HTTPServerRequest & request,
|
||||
HTTPServerResponse & response,
|
||||
Output & used_output);
|
||||
|
||||
static void pushDelayedResults(Output & used_output);
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user