Merge branch 'master' into processors-3

This commit is contained in:
Nikolai Kochetov 2019-10-12 19:24:02 +03:00
commit 757488369f
53 changed files with 929 additions and 456 deletions

View File

@ -164,6 +164,7 @@ macro(add_object_library name common_path)
endif ()
endmacro()
add_object_library(clickhouse_access src/Access)
add_object_library(clickhouse_core src/Core)
add_object_library(clickhouse_compression src/Compression)
add_object_library(clickhouse_datastreams src/DataStreams)

View File

@ -19,8 +19,8 @@
#include <Common/ClickHouseRevision.h>
#include <Common/ThreadStatus.h>
#include <Common/config_version.h>
#include <Common/quoteString.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <IO/UseSSL.h>
#include <Parsers/parseQuery.h>
@ -221,14 +221,6 @@ catch (const Exception & e)
}
inline String getQuotedString(const String & s)
{
WriteBufferFromOwnString buf;
writeQuotedString(s, buf);
return buf.str();
}
std::string LocalServer::getInitialCreateTableQuery()
{
if (!config().has("table-structure"))
@ -241,7 +233,7 @@ std::string LocalServer::getInitialCreateTableQuery()
if (!config().has("table-file") || config().getString("table-file") == "-") /// Use Unix tools stdin naming convention
table_file = "stdin";
else /// Use regular file
table_file = getQuotedString(config().getString("table-file"));
table_file = quoteString(config().getString("table-file"));
return
"CREATE TABLE " + table_name +

View File

@ -46,7 +46,7 @@ MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & so
, connection_id(connection_id_)
, public_key(public_key_)
, private_key(private_key_)
, auth_plugin(new Authentication::Native41())
, auth_plugin(new MySQLProtocol::Authentication::Native41())
{
server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF;
if (ssl_enabled)
@ -231,8 +231,8 @@ void MySQLHandler::authenticate(const String & user_name, const String & auth_pl
{
// For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when possible (if password is specified using double SHA1). Otherwise SHA256 plugin is used.
auto user = connection_context.getUser(user_name);
if (user->password_double_sha1_hex.empty())
auth_plugin = std::make_unique<Authentication::Sha256Password>(public_key, private_key, log);
if (user->authentication.getType() != DB::Authentication::DOUBLE_SHA1_PASSWORD)
auth_plugin = std::make_unique<MySQLProtocol::Authentication::Sha256Password>(public_key, private_key, log);
try {
std::optional<String> auth_response = auth_plugin_name == auth_plugin->getName() ? std::make_optional<String>(initial_auth_response) : std::nullopt;

View File

@ -0,0 +1,397 @@
#include <Access/AllowedClientHosts.h>
#include <Common/Exception.h>
#include <common/SimpleCache.h>
#include <Common/StringUtils/StringUtils.h>
#include <IO/ReadHelpers.h>
#include <Poco/Net/SocketAddress.h>
#include <Poco/RegularExpression.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <boost/range/algorithm/find.hpp>
#include <boost/range/algorithm/find_first_of.hpp>
namespace DB
{
namespace ErrorCodes
{
extern const int DNS_ERROR;
extern const int IP_ADDRESS_NOT_ALLOWED;
}
namespace
{
using IPAddress = Poco::Net::IPAddress;
const AllowedClientHosts::IPSubnet ALL_ADDRESSES = AllowedClientHosts::IPSubnet{IPAddress{IPAddress::IPv6}, IPAddress{IPAddress::IPv6}};
IPAddress toIPv6(const IPAddress & addr)
{
if (addr.family() == IPAddress::IPv6)
return addr;
return IPAddress("::FFFF:" + addr.toString());
}
IPAddress maskToIPv6(const IPAddress & mask)
{
if (mask.family() == IPAddress::IPv6)
return mask;
return IPAddress(96, IPAddress::IPv6) | toIPv6(mask);
}
bool isAddressOfHostImpl(const IPAddress & address, const String & host)
{
IPAddress addr_v6 = toIPv6(address);
/// Resolve by hand, because Poco don't use AI_ALL flag but we need it.
addrinfo * ai = nullptr;
SCOPE_EXIT(
{
if (ai)
freeaddrinfo(ai);
});
addrinfo hints;
memset(&hints, 0, sizeof(hints));
hints.ai_family = AF_UNSPEC;
hints.ai_flags |= AI_V4MAPPED | AI_ALL;
int ret = getaddrinfo(host.c_str(), nullptr, &hints, &ai);
if (0 != ret)
throw Exception("Cannot getaddrinfo: " + std::string(gai_strerror(ret)), ErrorCodes::DNS_ERROR);
for (; ai != nullptr; ai = ai->ai_next)
{
if (ai->ai_addrlen && ai->ai_addr)
{
if (ai->ai_family == AF_INET6)
{
if (addr_v6 == IPAddress(
&reinterpret_cast<sockaddr_in6*>(ai->ai_addr)->sin6_addr, sizeof(in6_addr),
reinterpret_cast<sockaddr_in6*>(ai->ai_addr)->sin6_scope_id))
{
return true;
}
}
else if (ai->ai_family == AF_INET)
{
if (addr_v6 == toIPv6(IPAddress(&reinterpret_cast<sockaddr_in *>(ai->ai_addr)->sin_addr, sizeof(in_addr))))
{
return true;
}
}
}
}
return false;
}
/// Cached version of isAddressOfHostImpl(). We need to cache DNS requests.
bool isAddressOfHost(const IPAddress & address, const String & host)
{
static SimpleCache<decltype(isAddressOfHostImpl), isAddressOfHostImpl> cache;
return cache(address, host);
}
String getHostByAddressImpl(const IPAddress & address)
{
Poco::Net::SocketAddress sock_addr(address, 0);
/// Resolve by hand, because Poco library doesn't have such functionality.
char host[1024];
int gai_errno = getnameinfo(sock_addr.addr(), sock_addr.length(), host, sizeof(host), nullptr, 0, NI_NAMEREQD);
if (0 != gai_errno)
throw Exception("Cannot getnameinfo: " + std::string(gai_strerror(gai_errno)), ErrorCodes::DNS_ERROR);
/// Check that PTR record is resolved back to client address
if (!isAddressOfHost(address, host))
throw Exception("Host " + String(host) + " isn't resolved back to " + address.toString(), ErrorCodes::DNS_ERROR);
return host;
}
/// Cached version of getHostByAddressImpl(). We need to cache DNS requests.
String getHostByAddress(const IPAddress & address)
{
static SimpleCache<decltype(getHostByAddressImpl), &getHostByAddressImpl> cache;
return cache(address);
}
}
String AllowedClientHosts::IPSubnet::toString() const
{
unsigned int prefix_length = mask.prefixLength();
if (IPAddress{prefix_length, mask.family()} == mask)
return prefix.toString() + "/" + std::to_string(prefix_length);
return prefix.toString() + "/" + mask.toString();
}
AllowedClientHosts::AllowedClientHosts()
{
}
AllowedClientHosts::AllowedClientHosts(AllAddressesTag)
{
addAllAddresses();
}
AllowedClientHosts::~AllowedClientHosts() = default;
AllowedClientHosts::AllowedClientHosts(const AllowedClientHosts & src)
{
*this = src;
}
AllowedClientHosts & AllowedClientHosts::operator =(const AllowedClientHosts & src)
{
addresses = src.addresses;
subnets = src.subnets;
host_names = src.host_names;
host_regexps = src.host_regexps;
compiled_host_regexps.clear();
return *this;
}
AllowedClientHosts::AllowedClientHosts(AllowedClientHosts && src)
{
*this = src;
}
AllowedClientHosts & AllowedClientHosts::operator =(AllowedClientHosts && src)
{
addresses = std::move(src.addresses);
subnets = std::move(src.subnets);
host_names = std::move(src.host_names);
host_regexps = std::move(src.host_regexps);
compiled_host_regexps = std::move(src.compiled_host_regexps);
return *this;
}
void AllowedClientHosts::clear()
{
addresses.clear();
subnets.clear();
host_names.clear();
host_regexps.clear();
compiled_host_regexps.clear();
}
bool AllowedClientHosts::empty() const
{
return addresses.empty() && subnets.empty() && host_names.empty() && host_regexps.empty();
}
void AllowedClientHosts::addAddress(const IPAddress & address)
{
IPAddress addr_v6 = toIPv6(address);
if (boost::range::find(addresses, addr_v6) == addresses.end())
addresses.push_back(addr_v6);
}
void AllowedClientHosts::addAddress(const String & address)
{
addAddress(IPAddress{address});
}
void AllowedClientHosts::addSubnet(const IPSubnet & subnet)
{
IPSubnet subnet_v6;
subnet_v6.prefix = toIPv6(subnet.prefix);
subnet_v6.mask = maskToIPv6(subnet.mask);
if (subnet_v6.mask == IPAddress(128, IPAddress::IPv6))
{
addAddress(subnet_v6.prefix);
return;
}
subnet_v6.prefix = subnet_v6.prefix & subnet_v6.mask;
if (boost::range::find(subnets, subnet_v6) == subnets.end())
subnets.push_back(subnet_v6);
}
void AllowedClientHosts::addSubnet(const IPAddress & prefix, const IPAddress & mask)
{
addSubnet(IPSubnet{prefix, mask});
}
void AllowedClientHosts::addSubnet(const IPAddress & prefix, size_t num_prefix_bits)
{
addSubnet(prefix, IPAddress(num_prefix_bits, prefix.family()));
}
void AllowedClientHosts::addSubnet(const String & subnet)
{
size_t slash = subnet.find('/');
if (slash == String::npos)
{
addAddress(subnet);
return;
}
IPAddress prefix{String{subnet, 0, slash}};
String mask(subnet, slash + 1, subnet.length() - slash - 1);
if (std::all_of(mask.begin(), mask.end(), isNumericASCII))
addSubnet(prefix, parseFromString<UInt8>(mask));
else
addSubnet(prefix, IPAddress{mask});
}
void AllowedClientHosts::addHostName(const String & host_name)
{
if (boost::range::find(host_names, host_name) == host_names.end())
host_names.push_back(host_name);
}
void AllowedClientHosts::addHostRegexp(const String & host_regexp)
{
if (boost::range::find(host_regexps, host_regexp) == host_regexps.end())
host_regexps.push_back(host_regexp);
}
void AllowedClientHosts::addAllAddresses()
{
clear();
addSubnet(ALL_ADDRESSES);
}
bool AllowedClientHosts::containsAllAddresses() const
{
return (boost::range::find(subnets, ALL_ADDRESSES) != subnets.end())
|| (boost::range::find(host_regexps, ".*") != host_regexps.end())
|| (boost::range::find(host_regexps, "$") != host_regexps.end());
}
bool AllowedClientHosts::contains(const IPAddress & address) const
{
return containsImpl(address, String(), nullptr);
}
void AllowedClientHosts::checkContains(const IPAddress & address, const String & user_name) const
{
String error;
if (!containsImpl(address, user_name, &error))
throw Exception(error, ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
}
bool AllowedClientHosts::containsImpl(const IPAddress & address, const String & user_name, String * error) const
{
if (error)
error->clear();
/// Check `ip_addresses`.
IPAddress addr_v6 = toIPv6(address);
if (boost::range::find(addresses, addr_v6) != addresses.end())
return true;
/// Check `ip_subnets`.
for (const auto & subnet : subnets)
if ((addr_v6 & subnet.mask) == subnet.prefix)
return true;
/// Check `hosts`.
for (const String & host_name : host_names)
{
try
{
if (isAddressOfHost(address, host_name))
return true;
}
catch (Exception & e)
{
if (e.code() != ErrorCodes::DNS_ERROR)
e.rethrow();
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
&Logger::get("AddressPatterns"),
"Failed to check if the allowed client hosts contain address " << address.toString() << ". " << e.displayText()
<< ", code = " << e.code());
}
}
/// Check `host_regexps`.
if (!host_regexps.empty())
{
compileRegexps();
try
{
String resolved_host = getHostByAddress(address);
for (const auto & compiled_regexp : compiled_host_regexps)
{
if (compiled_regexp && compiled_regexp->match(resolved_host))
return true;
}
}
catch (Exception & e)
{
if (e.code() != ErrorCodes::DNS_ERROR)
e.rethrow();
/// Try to ignore DNS errors: if host cannot be resolved, skip it and try next.
LOG_WARNING(
&Logger::get("AddressPatterns"),
"Failed to check if the allowed client hosts contain address " << address.toString() << ". " << e.displayText()
<< ", code = " << e.code());
}
}
if (error)
{
if (user_name.empty())
*error = "It's not allowed to connect from address " + address.toString();
else
*error = "User " + user_name + " is not allowed to connect from address " + address.toString();
}
return false;
}
void AllowedClientHosts::compileRegexps() const
{
if (compiled_host_regexps.size() == host_regexps.size())
return;
size_t old_size = compiled_host_regexps.size();
compiled_host_regexps.reserve(host_regexps.size());
for (size_t i = old_size; i != host_regexps.size(); ++i)
compiled_host_regexps.emplace_back(std::make_unique<Poco::RegularExpression>(host_regexps[i]));
}
bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs)
{
return (lhs.addresses == rhs.addresses) && (lhs.subnets == rhs.subnets) && (lhs.host_names == rhs.host_names)
&& (lhs.host_regexps == rhs.host_regexps);
}
}

View File

@ -0,0 +1,103 @@
#pragma once
#include <Core/Types.h>
#include <Poco/Net/IPAddress.h>
#include <memory>
#include <vector>
namespace Poco
{
class RegularExpression;
}
namespace DB
{
/// Represents lists of hosts an user is allowed to connect to server from.
class AllowedClientHosts
{
public:
using IPAddress = Poco::Net::IPAddress;
struct IPSubnet
{
IPAddress prefix;
IPAddress mask;
String toString() const;
friend bool operator ==(const IPSubnet & lhs, const IPSubnet & rhs) { return (lhs.prefix == rhs.prefix) && (lhs.mask == rhs.mask); }
friend bool operator !=(const IPSubnet & lhs, const IPSubnet & rhs) { return !(lhs == rhs); }
};
struct AllAddressesTag {};
AllowedClientHosts();
explicit AllowedClientHosts(AllAddressesTag);
~AllowedClientHosts();
AllowedClientHosts(const AllowedClientHosts & src);
AllowedClientHosts & operator =(const AllowedClientHosts & src);
AllowedClientHosts(AllowedClientHosts && src);
AllowedClientHosts & operator =(AllowedClientHosts && src);
/// Removes all contained addresses. This will disallow all addresses.
void clear();
bool empty() const;
/// Allows exact IP address.
/// For example, 213.180.204.3 or 2a02:6b8::3
void addAddress(const IPAddress & address);
void addAddress(const String & address);
/// Allows an IP subnet.
void addSubnet(const IPSubnet & subnet);
void addSubnet(const String & subnet);
/// Allows an IP subnet.
/// For example, 312.234.1.1/255.255.255.0 or 2a02:6b8::3/FFFF:FFFF:FFFF:FFFF::
void addSubnet(const IPAddress & prefix, const IPAddress & mask);
/// Allows an IP subnet.
/// For example, 10.0.0.1/8 or 2a02:6b8::3/64
void addSubnet(const IPAddress & prefix, size_t num_prefix_bits);
/// Allows all addresses.
void addAllAddresses();
/// Allows an exact host. The `contains()` function will check that the provided address equals to one of that host's addresses.
void addHostName(const String & host_name);
/// Allows a regular expression for the host.
void addHostRegexp(const String & host_regexp);
const std::vector<IPAddress> & getAddresses() const { return addresses; }
const std::vector<IPSubnet> & getSubnets() const { return subnets; }
const std::vector<String> & getHostNames() const { return host_names; }
const std::vector<String> & getHostRegexps() const { return host_regexps; }
/// Checks if the provided address is in the list. Returns false if not.
bool contains(const IPAddress & address) const;
/// Checks if any address is allowed.
bool containsAllAddresses() const;
/// Checks if the provided address is in the list. Throws an exception if not.
/// `username` is only used for generating an error message if the address isn't in the list.
void checkContains(const IPAddress & address, const String & user_name = String()) const;
friend bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs);
friend bool operator !=(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs) { return !(lhs == rhs); }
private:
bool containsImpl(const IPAddress & address, const String & user_name, String * error) const;
void compileRegexps() const;
std::vector<IPAddress> addresses;
std::vector<IPSubnet> subnets;
std::vector<String> host_names;
std::vector<String> host_regexps;
mutable std::vector<std::unique_ptr<Poco::RegularExpression>> compiled_host_regexps;
};
}

View File

@ -0,0 +1,207 @@
#include <Access/Authentication.h>
#include <Common/Exception.h>
#include <common/StringRef.h>
#include <Core/Defines.h>
#include <Poco/SHA1Engine.h>
#include <boost/algorithm/hex.hpp>
#include "config_core.h"
#if USE_SSL
# include <openssl/sha.h>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int SUPPORT_IS_DISABLED;
extern const int REQUIRED_PASSWORD;
extern const int WRONG_PASSWORD;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
namespace
{
using Digest = Authentication::Digest;
Digest encodePlainText(const StringRef & text)
{
return Digest(text.data, text.data + text.size);
}
Digest encodeSHA256(const StringRef & text)
{
#if USE_SSL
Digest hash;
hash.resize(32);
SHA256_CTX ctx;
SHA256_Init(&ctx);
SHA256_Update(&ctx, reinterpret_cast<const UInt8 *>(text.data), text.size);
SHA256_Final(hash.data(), &ctx);
return hash;
#else
UNUSED(text);
throw DB::Exception("SHA256 passwords support is disabled, because ClickHouse was built without SSL library", DB::ErrorCodes::SUPPORT_IS_DISABLED);
#endif
}
Digest encodeSHA1(const StringRef & text)
{
Poco::SHA1Engine engine;
engine.update(text.data, text.size);
return engine.digest();
}
Digest encodeSHA1(const Digest & text)
{
return encodeSHA1(StringRef{reinterpret_cast<const char *>(text.data()), text.size()});
}
Digest encodeDoubleSHA1(const StringRef & text)
{
return encodeSHA1(encodeSHA1(text));
}
}
Authentication::Authentication(Authentication::Type type_)
: type(type_)
{
}
void Authentication::setPassword(const String & password_)
{
switch (type)
{
case NO_PASSWORD:
throw Exception("Cannot specify password for the 'NO_PASSWORD' authentication type", ErrorCodes::LOGICAL_ERROR);
case PLAINTEXT_PASSWORD:
setPasswordHashBinary(encodePlainText(password_));
return;
case SHA256_PASSWORD:
setPasswordHashBinary(encodeSHA256(password_));
return;
case DOUBLE_SHA1_PASSWORD:
setPasswordHashBinary(encodeDoubleSHA1(password_));
return;
}
throw Exception("Unknown authentication type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
}
String Authentication::getPassword() const
{
if (type != PLAINTEXT_PASSWORD)
throw Exception("Cannot decode the password", ErrorCodes::LOGICAL_ERROR);
return String(password_hash.data(), password_hash.data() + password_hash.size());
}
void Authentication::setPasswordHashHex(const String & hash)
{
Digest digest;
digest.resize(hash.size() / 2);
boost::algorithm::unhex(hash.begin(), hash.end(), digest.data());
setPasswordHashBinary(digest);
}
String Authentication::getPasswordHashHex() const
{
String hex;
hex.resize(password_hash.size() * 2);
boost::algorithm::hex(password_hash.begin(), password_hash.end(), hex.data());
return hex;
}
void Authentication::setPasswordHashBinary(const Digest & hash)
{
switch (type)
{
case NO_PASSWORD:
throw Exception("Cannot specify password for the 'NO_PASSWORD' authentication type", ErrorCodes::LOGICAL_ERROR);
case PLAINTEXT_PASSWORD:
{
password_hash = hash;
return;
}
case SHA256_PASSWORD:
{
if (hash.size() != 32)
throw Exception(
"Password hash for the 'SHA256_PASSWORD' authentication type has length " + std::to_string(hash.size())
+ " but must be exactly 32 bytes.",
ErrorCodes::BAD_ARGUMENTS);
password_hash = hash;
return;
}
case DOUBLE_SHA1_PASSWORD:
{
if (hash.size() != 20)
throw Exception(
"Password hash for the 'DOUBLE_SHA1_PASSWORD' authentication type has length " + std::to_string(hash.size())
+ " but must be exactly 20 bytes.",
ErrorCodes::BAD_ARGUMENTS);
password_hash = hash;
return;
}
}
throw Exception("Unknown authentication type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
}
bool Authentication::isCorrectPassword(const String & password_) const
{
switch (type)
{
case NO_PASSWORD:
return true;
case PLAINTEXT_PASSWORD:
return password_ == StringRef{reinterpret_cast<const char *>(password_hash.data()), password_hash.size()};
case SHA256_PASSWORD:
return encodeSHA256(password_) == password_hash;
case DOUBLE_SHA1_PASSWORD:
{
auto first_sha1 = encodeSHA1(password_);
/// If it was MySQL compatibility server, then first_sha1 already contains double SHA1.
if (first_sha1 == password_hash)
return true;
return encodeSHA1(first_sha1) == password_hash;
}
}
throw Exception("Unknown authentication type: " + std::to_string(static_cast<int>(type)), ErrorCodes::LOGICAL_ERROR);
}
void Authentication::checkPassword(const String & password_, const String & user_name) const
{
if (isCorrectPassword(password_))
return;
auto info_about_user_name = [&user_name]() { return user_name.empty() ? String() : " for user " + user_name; };
if (password_.empty() && (type != NO_PASSWORD))
throw Exception("Password required" + info_about_user_name(), ErrorCodes::REQUIRED_PASSWORD);
throw Exception("Wrong password" + info_about_user_name(), ErrorCodes::WRONG_PASSWORD);
}
bool operator ==(const Authentication & lhs, const Authentication & rhs)
{
return (lhs.type == rhs.type) && (lhs.password_hash == rhs.password_hash);
}
}

View File

@ -0,0 +1,66 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
/// Authentication type and encrypted password for checking when an user logins.
class Authentication
{
public:
enum Type
{
/// User doesn't have to enter password.
NO_PASSWORD,
/// Password is stored as is.
PLAINTEXT_PASSWORD,
/// Password is encrypted in SHA256 hash.
SHA256_PASSWORD,
/// SHA1(SHA1(password)).
/// This kind of hash is used by the `mysql_native_password` authentication plugin.
DOUBLE_SHA1_PASSWORD,
};
using Digest = std::vector<UInt8>;
Authentication(Authentication::Type type = NO_PASSWORD);
Authentication(const Authentication & src) = default;
Authentication & operator =(const Authentication & src) = default;
Authentication(Authentication && src) = default;
Authentication & operator =(Authentication && src) = default;
Type getType() const { return type; }
/// Sets the password and encrypt it using the authentication type set in the constructor.
void setPassword(const String & password);
/// Returns the password. Allowed to use only for Type::PLAINTEXT_PASSWORD.
String getPassword() const;
/// Sets the password as a string of hexadecimal digits.
void setPasswordHashHex(const String & hash);
String getPasswordHashHex() const;
/// Sets the password in binary form.
void setPasswordHashBinary(const Digest & hash);
const Digest & getPasswordHashBinary() const { return password_hash; }
/// Checks if the provided password is correct. Returns false if not.
bool isCorrectPassword(const String & password) const;
/// Checks if the provided password is correct. Throws an exception if not.
/// `user_name` is only used for generating an error message if the password is incorrect.
void checkPassword(const String & password, const String & user_name = String()) const;
friend bool operator ==(const Authentication & lhs, const Authentication & rhs);
friend bool operator !=(const Authentication & lhs, const Authentication & rhs) { return !(lhs == rhs); }
private:
Type type = Type::NO_PASSWORD;
Digest password_hash;
};
}

View File

View File

@ -1,3 +1,4 @@
add_subdirectory (Access)
add_subdirectory (Columns)
add_subdirectory (Common)
add_subdirectory (Core)

View File

@ -1,6 +1,6 @@
#include <Common/DiskSpaceMonitor.h>
#include <Common/escapeForFileName.h>
#include <IO/WriteHelpers.h>
#include <Common/quoteString.h>
#include <set>

View File

@ -0,0 +1,37 @@
#include <Common/quoteString.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{
String quoteString(const StringRef & x)
{
String res(x.size, '\0');
WriteBufferFromString wb(res);
writeQuotedString(x, wb);
return res;
}
String backQuote(const StringRef & x)
{
String res(x.size, '\0');
{
WriteBufferFromString wb(res);
writeBackQuotedString(x, wb);
}
return res;
}
String backQuoteIfNeed(const StringRef & x)
{
String res(x.size, '\0');
{
WriteBufferFromString wb(res);
writeProbablyBackQuotedString(x, wb);
}
return res;
}
}

View File

@ -0,0 +1,17 @@
#pragma once
#include <Core/Types.h>
#include <common/StringRef.h>
namespace DB
{
/// Quote the string.
String quoteString(const StringRef & x);
/// Quote the identifier with backquotes.
String backQuote(const StringRef & x);
/// Quote the identifier with backquotes, if required.
String backQuoteIfNeed(const StringRef & x);
}

View File

@ -919,10 +919,10 @@ public:
auto user = context.getUser(user_name);
if (user->password_double_sha1_hex.empty())
if (user->authentication.getType() != DB::Authentication::DOUBLE_SHA1_PASSWORD)
throw Exception("Cannot use " + getName() + " auth plugin for user " + user_name + " since its password isn't specified using double SHA1.", ErrorCodes::UNKNOWN_EXCEPTION);
Poco::SHA1Engine::Digest double_sha1_value = Poco::DigestEngine::digestFromHex(user->password_double_sha1_hex);
Poco::SHA1Engine::Digest double_sha1_value = user->authentication.getPasswordHashBinary();
assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE);
Poco::SHA1Engine engine;

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnsCommon.h>
#include <Columns/ColumnsNumber.h>
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#include <Common/FieldVisitors.h>

View File

@ -2,6 +2,7 @@
#include <Interpreters/castColumn.h>
#include <Columns/ColumnConst.h>
#include <Common/assert_cast.h>
#include <Common/quoteString.h>
#include <Parsers/IAST.h>

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Databases/DatabasesCommon.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTCreateQuery.h>

View File

@ -66,26 +66,4 @@ void writeException(const Exception & e, WriteBuffer & buf, bool with_stack_trac
if (has_nested)
writeException(Exception(Exception::CreateFromPoco, *e.nested()), buf, with_stack_trace);
}
String backQuoteIfNeed(const String & x)
{
String res(x.size(), '\0');
{
WriteBufferFromString wb(res);
writeProbablyBackQuotedString(x, wb);
}
return res;
}
String backQuote(const String & x)
{
String res(x.size(), '\0');
{
WriteBufferFromString wb(res);
writeBackQuotedString(x, wb);
}
return res;
}
}

View File

@ -410,36 +410,36 @@ inline void writeQuotedString(const StringRef & ref, WriteBuffer & buf)
writeAnyQuotedString<'\''>(ref, buf);
}
inline void writeDoubleQuotedString(const String & s, WriteBuffer & buf)
inline void writeDoubleQuotedString(const StringRef & s, WriteBuffer & buf)
{
writeAnyQuotedString<'"'>(s, buf);
}
/// Outputs a string in backquotes.
inline void writeBackQuotedString(const String & s, WriteBuffer & buf)
inline void writeBackQuotedString(const StringRef & s, WriteBuffer & buf)
{
writeAnyQuotedString<'`'>(s, buf);
}
/// Outputs a string in backquotes for MySQL.
inline void writeBackQuotedStringMySQL(const String & s, WriteBuffer & buf)
inline void writeBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf)
{
writeChar('`', buf);
writeAnyEscapedString<'`', true>(s.data(), s.data() + s.size(), buf);
writeAnyEscapedString<'`', true>(s.data, s.data + s.size, buf);
writeChar('`', buf);
}
/// The same, but quotes apply only if there are characters that do not match the identifier without quotes.
template <typename F>
inline void writeProbablyQuotedStringImpl(const String & s, WriteBuffer & buf, F && write_quoted_string)
inline void writeProbablyQuotedStringImpl(const StringRef & s, WriteBuffer & buf, F && write_quoted_string)
{
if (s.empty() || !isValidIdentifierBegin(s[0]))
if (!s.size || !isValidIdentifierBegin(s.data[0]))
write_quoted_string(s, buf);
else
{
const char * pos = s.data() + 1;
const char * end = s.data() + s.size();
const char * pos = s.data + 1;
const char * end = s.data + s.size;
for (; pos < end; ++pos)
if (!isWordCharASCII(*pos))
break;
@ -450,19 +450,19 @@ inline void writeProbablyQuotedStringImpl(const String & s, WriteBuffer & buf, F
}
}
inline void writeProbablyBackQuotedString(const String & s, WriteBuffer & buf)
inline void writeProbablyBackQuotedString(const StringRef & s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const String & s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); });
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedString(s_, buf_); });
}
inline void writeProbablyDoubleQuotedString(const String & s, WriteBuffer & buf)
inline void writeProbablyDoubleQuotedString(const StringRef & s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const String & s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); });
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeDoubleQuotedString(s_, buf_); });
}
inline void writeProbablyBackQuotedStringMySQL(const String & s, WriteBuffer & buf)
inline void writeProbablyBackQuotedStringMySQL(const StringRef & s, WriteBuffer & buf)
{
writeProbablyQuotedStringImpl(s, buf, [](const String & s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); });
writeProbablyQuotedStringImpl(s, buf, [](const StringRef & s_, WriteBuffer & buf_) { return writeBackQuotedStringMySQL(s_, buf_); });
}
@ -905,11 +905,4 @@ inline String toString(const T & x)
writeText(x, buf);
return buf.str();
}
/// Quote the identifier with backquotes, if required.
String backQuoteIfNeed(const String & x);
/// Quote the identifier with backquotes.
String backQuote(const String & x);
}

View File

@ -7,6 +7,7 @@
#include <Parsers/ASTDropQuery.h>
#include <Storages/IStorage.h>
#include <Common/escapeForFileName.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>

View File

@ -8,7 +8,7 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/formatAST.h>
#include <Parsers/ASTSubquery.h>
#include <IO/WriteHelpers.h>
#include <Common/quoteString.h>
namespace DB
{

View File

@ -10,7 +10,7 @@
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <Common/quoteString.h>
namespace DB
{

View File

@ -1,5 +1,6 @@
#include <boost/algorithm/string/replace.hpp>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
#include <Columns/IColumn.h>
#include <Core/Field.h>
#include <DataTypes/IDataType.h>

View File

@ -1,22 +1,10 @@
#include <string.h>
#include <Poco/RegularExpression.h>
#include <Poco/Net/IPAddress.h>
#include <Poco/Net/SocketAddress.h>
#include <Poco/Net/DNS.h>
#include <Poco/Util/Application.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Poco/String.h>
#include <Common/Exception.h>
#include <IO/ReadHelpers.h>
#include <IO/HexWriteBuffer.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <common/SimpleCache.h>
#include <Common/StringUtils/StringUtils.h>
#include <Interpreters/Users.h>
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <Common/config.h>
namespace DB
@ -24,255 +12,12 @@ namespace DB
namespace ErrorCodes
{
extern const int DNS_ERROR;
extern const int UNKNOWN_ADDRESS_PATTERN_TYPE;
extern const int UNKNOWN_USER;
extern const int REQUIRED_PASSWORD;
extern const int WRONG_PASSWORD;
extern const int IP_ADDRESS_NOT_ALLOWED;
extern const int BAD_ARGUMENTS;
}
static Poco::Net::IPAddress toIPv6(const Poco::Net::IPAddress addr)
{
if (addr.family() == Poco::Net::IPAddress::IPv6)
return addr;
return Poco::Net::IPAddress("::FFFF:" + addr.toString());
}
/// IP-address or subnet mask. Example: 213.180.204.3 or 10.0.0.1/8 or 312.234.1.1/255.255.255.0
/// 2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/FFFF:FFFF:FFFF:FFFF::
class IPAddressPattern : public IAddressPattern
{
private:
/// Address of mask. Always transformed to IPv6.
Poco::Net::IPAddress mask_address;
/// Mask of net (ip form). Always transformed to IPv6.
Poco::Net::IPAddress subnet_mask;
public:
explicit IPAddressPattern(const String & str)
{
const char * pos = strchr(str.c_str(), '/');
if (nullptr == pos)
{
construct(Poco::Net::IPAddress(str));
}
else
{
String addr(str, 0, pos - str.c_str());
auto real_address = Poco::Net::IPAddress(addr);
String str_mask(str, addr.length() + 1, str.length() - addr.length() - 1);
if (isDigits(str_mask))
{
UInt8 prefix_bits = parse<UInt8>(pos + 1);
construct(prefix_bits, real_address.family() == Poco::Net::AddressFamily::IPv4);
}
else
{
subnet_mask = netmaskToIPv6(Poco::Net::IPAddress(str_mask));
}
mask_address = toIPv6(real_address);
}
}
bool contains(const Poco::Net::IPAddress & addr) const override
{
return prefixBitsEquals(addr, mask_address, subnet_mask);
}
private:
void construct(const Poco::Net::IPAddress & mask_address_)
{
mask_address = toIPv6(mask_address_);
subnet_mask = Poco::Net::IPAddress(128, Poco::Net::IPAddress::IPv6);
}
void construct(UInt8 prefix_bits, bool is_ipv4)
{
prefix_bits = is_ipv4 ? prefix_bits + 96 : prefix_bits;
subnet_mask = Poco::Net::IPAddress(prefix_bits, Poco::Net::IPAddress::IPv6);
}
static bool prefixBitsEquals(const Poco::Net::IPAddress & ip_address, const Poco::Net::IPAddress & net_address, const Poco::Net::IPAddress & mask)
{
return ((toIPv6(ip_address) & mask) == (toIPv6(net_address) & mask));
}
static bool isDigits(const std::string & str)
{
return std::all_of(str.begin(), str.end(), isNumericASCII);
}
static Poco::Net::IPAddress netmaskToIPv6(Poco::Net::IPAddress mask)
{
if (mask.family() == Poco::Net::IPAddress::IPv6)
return mask;
return Poco::Net::IPAddress(96, Poco::Net::IPAddress::IPv6) | toIPv6(mask);
}
};
/// Check that address equals to one of hostname addresses.
class HostExactPattern : public IAddressPattern
{
private:
String host;
static bool containsImpl(const String & host, const Poco::Net::IPAddress & addr)
{
Poco::Net::IPAddress addr_v6 = toIPv6(addr);
/// Resolve by hand, because Poco don't use AI_ALL flag but we need it.
addrinfo * ai = nullptr;
addrinfo hints;
memset(&hints, 0, sizeof(hints));
hints.ai_family = AF_UNSPEC;
hints.ai_flags |= AI_V4MAPPED | AI_ALL;
int ret = getaddrinfo(host.c_str(), nullptr, &hints, &ai);
if (0 != ret)
throw Exception("Cannot getaddrinfo: " + std::string(gai_strerror(ret)), ErrorCodes::DNS_ERROR);
SCOPE_EXIT(
{
freeaddrinfo(ai);
});
for (; ai != nullptr; ai = ai->ai_next)
{
if (ai->ai_addrlen && ai->ai_addr)
{
if (ai->ai_family == AF_INET6)
{
if (addr_v6 == Poco::Net::IPAddress(
&reinterpret_cast<sockaddr_in6*>(ai->ai_addr)->sin6_addr, sizeof(in6_addr),
reinterpret_cast<sockaddr_in6*>(ai->ai_addr)->sin6_scope_id))
{
return true;
}
}
else if (ai->ai_family == AF_INET)
{
if (addr_v6 == toIPv6(Poco::Net::IPAddress(
&reinterpret_cast<sockaddr_in*>(ai->ai_addr)->sin_addr, sizeof(in_addr))))
{
return true;
}
}
}
}
return false;
}
public:
explicit HostExactPattern(const String & host_) : host(host_) {}
bool contains(const Poco::Net::IPAddress & addr) const override
{
static SimpleCache<decltype(containsImpl), &containsImpl> cache;
return cache(host, addr);
}
};
/// Check that PTR record for address match the regexp (and in addition, check that PTR record is resolved back to client address).
class HostRegexpPattern : public IAddressPattern
{
private:
Poco::RegularExpression host_regexp;
static String getDomain(const Poco::Net::IPAddress & addr)
{
Poco::Net::SocketAddress sock_addr(addr, 0);
/// Resolve by hand, because Poco library doesn't have such functionality.
char domain[1024];
int gai_errno = getnameinfo(sock_addr.addr(), sock_addr.length(), domain, sizeof(domain), nullptr, 0, NI_NAMEREQD);
if (0 != gai_errno)
throw Exception("Cannot getnameinfo: " + std::string(gai_strerror(gai_errno)), ErrorCodes::DNS_ERROR);
return domain;
}
public:
explicit HostRegexpPattern(const String & host_regexp_) : host_regexp(host_regexp_) {}
bool contains(const Poco::Net::IPAddress & addr) const override
{
static SimpleCache<decltype(getDomain), &getDomain> cache;
String domain = cache(addr);
Poco::RegularExpression::Match match;
if (host_regexp.match(domain, match) && HostExactPattern(domain).contains(addr))
return true;
return false;
}
};
bool AddressPatterns::contains(const Poco::Net::IPAddress & addr) const
{
for (size_t i = 0, size = patterns.size(); i < size; ++i)
{
/// If host cannot be resolved, skip it and try next.
try
{
if (patterns[i]->contains(addr))
return true;
}
catch (const DB::Exception & e)
{
LOG_WARNING(&Logger::get("AddressPatterns"),
"Failed to check if pattern contains address " << addr.toString() << ". " << e.displayText() << ", code = " << e.code());
if (e.code() == ErrorCodes::DNS_ERROR)
{
continue;
}
else
throw;
}
}
return false;
}
void AddressPatterns::addFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config)
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_elem, config_keys);
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
{
Container::value_type pattern;
String value = config.getString(config_elem + "." + *it);
if (startsWith(*it, "ip"))
pattern = std::make_unique<IPAddressPattern>(value);
else if (startsWith(*it, "host_regexp"))
pattern = std::make_unique<HostRegexpPattern>(value);
else if (startsWith(*it, "host"))
pattern = std::make_unique<HostExactPattern>(value);
else
throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE);
patterns.emplace_back(std::move(pattern));
}
}
User::User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config)
: name(name_)
{
@ -288,28 +33,43 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A
throw Exception("Either 'password' or 'password_sha256_hex' or 'password_double_sha1_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS);
if (has_password)
password = config.getString(config_elem + ".password");
if (has_password_sha256_hex)
{
password_sha256_hex = Poco::toLower(config.getString(config_elem + ".password_sha256_hex"));
if (password_sha256_hex.size() != 64)
throw Exception("password_sha256_hex for user " + name + " has length " + toString(password_sha256_hex.size()) + " but must be exactly 64 symbols.", ErrorCodes::BAD_ARGUMENTS);
authentication = Authentication{Authentication::PLAINTEXT_PASSWORD};
authentication.setPassword(config.getString(config_elem + ".password"));
}
if (has_password_double_sha1_hex)
else if (has_password_sha256_hex)
{
password_double_sha1_hex = Poco::toLower(config.getString(config_elem + ".password_double_sha1_hex"));
if (password_double_sha1_hex.size() != 40)
throw Exception("password_double_sha1_hex for user " + name + " has length " + toString(password_double_sha1_hex.size()) + " but must be exactly 40 symbols.", ErrorCodes::BAD_ARGUMENTS);
authentication = Authentication{Authentication::SHA256_PASSWORD};
authentication.setPasswordHashHex(config.getString(config_elem + ".password_sha256_hex"));
}
else if (has_password_double_sha1_hex)
{
authentication = Authentication{Authentication::DOUBLE_SHA1_PASSWORD};
authentication.setPasswordHashHex(config.getString(config_elem + ".password_double_sha1_hex"));
}
profile = config.getString(config_elem + ".profile");
quota = config.getString(config_elem + ".quota");
addresses.addFromConfig(config_elem + ".networks", config);
/// Fill list of allowed hosts.
const auto config_networks = config_elem + ".networks";
if (config.has(config_networks))
{
Poco::Util::AbstractConfiguration::Keys config_keys;
config.keys(config_networks, config_keys);
for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it)
{
String value = config.getString(config_networks + "." + *it);
if (startsWith(*it, "ip"))
allowed_client_hosts.addSubnet(value);
else if (startsWith(*it, "host_regexp"))
allowed_client_hosts.addHostRegexp(value);
else if (startsWith(*it, "host"))
allowed_client_hosts.addHostName(value);
else
throw Exception("Unknown address pattern type: " + *it, ErrorCodes::UNKNOWN_ADDRESS_PATTERN_TYPE);
}
}
/// Fill list of allowed databases.
const auto config_sub_elem = config_elem + ".allow_databases";

View File

@ -1,20 +1,16 @@
#pragma once
#include <Core/Types.h>
#include <Access/Authentication.h>
#include <Access/AllowedClientHosts.h>
#include <memory>
#include <unordered_map>
#include <unordered_set>
#include <vector>
namespace Poco
{
namespace Net
{
class IPAddress;
}
namespace Util
{
class AbstractConfiguration;
@ -24,44 +20,19 @@ namespace Poco
namespace DB
{
/// Allow to check that address matches a pattern.
class IAddressPattern
{
public:
virtual bool contains(const Poco::Net::IPAddress & addr) const = 0;
virtual ~IAddressPattern() {}
};
class AddressPatterns
{
private:
using Container = std::vector<std::shared_ptr<IAddressPattern>>;
Container patterns;
public:
bool contains(const Poco::Net::IPAddress & addr) const;
void addFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config);
};
/** User and ACL.
*/
struct User
{
String name;
/// Required password. Could be stored in plaintext or in SHA256.
String password;
String password_sha256_hex;
String password_double_sha1_hex;
/// Required password.
Authentication authentication;
String profile;
String quota;
AddressPatterns addresses;
AllowedClientHosts allowed_client_hosts;
/// List of allowed databases.
using DatabaseSet = std::unordered_set<std::string>;

View File

@ -1,18 +1,7 @@
#include <Interpreters/UsersManager.h>
#include "config_core.h"
#include <Common/Exception.h>
#include <common/logger_useful.h>
#include <IO/HexWriteBuffer.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Poco/Net/IPAddress.h>
#include <Poco/SHA1Engine.h>
#include <Poco/String.h>
#include <Poco/Util/AbstractConfiguration.h>
#if USE_SSL
# include <openssl/sha.h>
#endif
namespace DB
@ -20,14 +9,7 @@ namespace DB
namespace ErrorCodes
{
extern const int DNS_ERROR;
extern const int UNKNOWN_ADDRESS_PATTERN_TYPE;
extern const int UNKNOWN_USER;
extern const int REQUIRED_PASSWORD;
extern const int WRONG_PASSWORD;
extern const int IP_ADDRESS_NOT_ALLOWED;
extern const int BAD_ARGUMENTS;
extern const int SUPPORT_IS_DISABLED;
}
using UserPtr = UsersManager::UserPtr;
@ -58,62 +40,8 @@ UserPtr UsersManager::authorizeAndGetUser(
if (users.end() == it)
throw Exception("Unknown user " + user_name, ErrorCodes::UNKNOWN_USER);
if (!it->second->addresses.contains(address))
throw Exception("User " + user_name + " is not allowed to connect from address " + address.toString(), ErrorCodes::IP_ADDRESS_NOT_ALLOWED);
auto on_wrong_password = [&]()
{
if (password.empty())
throw Exception("Password required for user " + user_name, ErrorCodes::REQUIRED_PASSWORD);
else
throw Exception("Wrong password for user " + user_name, ErrorCodes::WRONG_PASSWORD);
};
if (!it->second->password_sha256_hex.empty())
{
#if USE_SSL
unsigned char hash[32];
SHA256_CTX ctx;
SHA256_Init(&ctx);
SHA256_Update(&ctx, reinterpret_cast<const unsigned char *>(password.data()), password.size());
SHA256_Final(hash, &ctx);
String hash_hex;
{
WriteBufferFromString buf(hash_hex);
HexWriteBuffer hex_buf(buf);
hex_buf.write(reinterpret_cast<const char *>(hash), sizeof(hash));
}
Poco::toLowerInPlace(hash_hex);
if (hash_hex != it->second->password_sha256_hex)
on_wrong_password();
#else
throw DB::Exception("SHA256 passwords support is disabled, because ClickHouse was built without SSL library", DB::ErrorCodes::SUPPORT_IS_DISABLED);
#endif
}
else if (!it->second->password_double_sha1_hex.empty())
{
Poco::SHA1Engine engine;
engine.update(password);
const auto & first_sha1 = engine.digest();
/// If it was MySQL compatibility server, then first_sha1 already contains double SHA1.
if (Poco::SHA1Engine::digestToHex(first_sha1) == it->second->password_double_sha1_hex)
return it->second;
engine.update(first_sha1.data(), first_sha1.size());
if (Poco::SHA1Engine::digestToHex(engine.digest()) != it->second->password_double_sha1_hex)
on_wrong_password();
}
else if (password != it->second->password)
{
on_wrong_password();
}
it->second->allowed_client_hosts.checkContains(address, user_name);
it->second->authentication.checkPassword(password, user_name);
return it->second;
}

View File

@ -1,6 +1,6 @@
#include <Parsers/ASTAlterQuery.h>
#include <iomanip>
#include <IO/WriteHelpers.h>
#include <Common/quoteString.h>
namespace DB
@ -183,9 +183,7 @@ void ASTAlterCommand::formatImpl(
settings.ostr << "VOLUME ";
break;
}
WriteBufferFromOwnString move_destination_name_buf;
writeQuoted(move_destination_name, move_destination_name_buf);
settings.ostr << move_destination_name_buf.str();
settings.ostr << quoteString(move_destination_name);
}
else if (type == ASTAlterCommand::REPLACE_PARTITION)
{

View File

@ -2,6 +2,7 @@
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Parsers/ASTPartition.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTColumnDeclaration.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,9 +1,6 @@
#include "ASTColumnsMatcher.h"
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <Common/quoteString.h>
#include <re2/re2.h>
@ -22,10 +19,8 @@ void ASTColumnsMatcher::appendColumnName(WriteBuffer & ostr) const { writeString
void ASTColumnsMatcher::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const
{
WriteBufferFromOwnString pattern_quoted;
writeQuotedString(original_pattern, pattern_quoted);
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "(" << pattern_quoted.str() << ")";
settings.ostr << (settings.hilite ? hilite_keyword : "") << "COLUMNS" << (settings.hilite ? hilite_none : "") << "("
<< quoteString(original_pattern) << ")";
}
void ASTColumnsMatcher::setPattern(String pattern)

View File

@ -1,4 +1,6 @@
#include <Parsers/ASTConstraintDeclaration.h>
#include <Common/quoteString.h>
namespace DB
{

View File

@ -3,6 +3,7 @@
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,4 +1,6 @@
#include <Parsers/ASTDictionaryAttributeDeclaration.h>
#include <Common/quoteString.h>
namespace DB
{

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTDropQuery.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -2,7 +2,8 @@
#include <Core/Field.h>
#include <Core/Types.h>
#include <IO/WriteHelpers.h>
#include <Common/FieldVisitors.h>
#include <Common/quoteString.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/IAST.h>
@ -52,7 +53,7 @@ public:
s.ostr << (s.hilite ? hilite_keyword : "") << " TYPE " << (s.hilite ? hilite_none : "");
type->formatImpl(s, state, frame);
s.ostr << (s.hilite ? hilite_keyword : "") << " GRANULARITY " << (s.hilite ? hilite_none : "");
s.ostr << toString(granularity);
s.ostr << granularity;
}
};

View File

@ -1,6 +1,7 @@
#include <iomanip>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,6 +1,7 @@
#pragma once
#include <Parsers/IAST.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTOptimizeQuery.h>
#include <Common/quoteString.h>
namespace DB
{

View File

@ -1,5 +1,6 @@
#include <Parsers/ASTQueryParameter.h>
#include <IO/WriteHelpers.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -4,6 +4,7 @@
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
#include <Interpreters/evaluateConstantExpression.h>

View File

@ -1,4 +1,5 @@
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -3,11 +3,12 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTQueryWithOutput.h>
#include <Parsers/ASTQueryWithOnCluster.h>
#include <Common/quoteString.h>
namespace DB
{
/** RENAME query
*/
class ASTRenameQuery : public ASTQueryWithOutput, public ASTQueryWithOnCluster

View File

@ -1,5 +1,6 @@
#include <iomanip>
#include <Parsers/ASTShowTablesQuery.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,5 +1,6 @@
#include <Parsers/IAST.h>
#include <Parsers/ASTSystemQuery.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -1,6 +1,7 @@
#pragma once
#include <Parsers/IAST.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -12,6 +12,7 @@ limitations under the License. */
#pragma once
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -5,7 +5,6 @@
#include <Parsers/IdentifierQuotingStyle.h>
#include <Common/Exception.h>
#include <Common/TypePromotion.h>
#include <IO/WriteHelpers.h> /// backQuote, backQuoteIfNeed
#include <algorithm>
#include <ostream>
@ -223,5 +222,4 @@ private:
size_t checkDepthImpl(size_t max_depth, size_t level) const;
};
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Parsers/ASTQueryWithTableAndOutput.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -4,6 +4,7 @@
#include <Columns/ColumnConst.h>
#include <Parsers/IAST.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
namespace DB
{

View File

@ -3,6 +3,7 @@
#include <Storages/AlterCommands.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Common/quoteString.h>
#include <Processors/Executors/TreeExecutor.h>

View File

@ -7,6 +7,7 @@
#include <Parsers/ASTAssignment.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Common/quoteString.h>
namespace DB

View File

@ -19,6 +19,7 @@
#include <Common/CurrentMetrics.h>
#include <Common/MemoryTracker.h>
#include <Common/FieldVisitors.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <Common/ProfileEvents.h>
#include <common/logger_useful.h>

View File

@ -11,6 +11,7 @@
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/quoteString.h>
#include <TableFunctions/TableFunctionFactory.h>

View File

@ -18,9 +18,9 @@
#include <Core/Defines.h>
#include <Common/Exception.h>
#include <Common/parseAddress.h>
#include <Common/quoteString.h>
#include <Common/typeid_cast.h>
#include <DataTypes/convertMySQLDataType.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <mysqlxx/Pool.h>