mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
use DNSResolver to check if user is allowed to connect
This commit is contained in:
parent
a31115d4ad
commit
0cf8015f4a
@ -8,6 +8,7 @@
|
||||
#include <ext/scope_guard.h>
|
||||
#include <boost/algorithm/string/replace.hpp>
|
||||
#include <ifaddrs.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -44,66 +45,22 @@ namespace
|
||||
return IPSubnet(toIPv6(subnet.getPrefix()), subnet.getMask());
|
||||
}
|
||||
|
||||
|
||||
/// Helper function for isAddressOfHost().
|
||||
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_begin = nullptr;
|
||||
SCOPE_EXIT(
|
||||
{
|
||||
if (ai_begin)
|
||||
freeaddrinfo(ai_begin);
|
||||
});
|
||||
|
||||
addrinfo hints;
|
||||
memset(&hints, 0, sizeof(hints));
|
||||
hints.ai_family = AF_UNSPEC;
|
||||
hints.ai_flags |= AI_V4MAPPED | AI_ALL;
|
||||
|
||||
int err = getaddrinfo(host.c_str(), nullptr, &hints, &ai_begin);
|
||||
if (err)
|
||||
throw Exception("Cannot getaddrinfo(" + host + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR);
|
||||
|
||||
for (const addrinfo * ai = ai_begin; ai; ai = ai->ai_next)
|
||||
{
|
||||
if (ai->ai_addrlen && ai->ai_addr)
|
||||
{
|
||||
if (ai->ai_family == AF_INET)
|
||||
{
|
||||
const auto & sin = *reinterpret_cast<const sockaddr_in *>(ai->ai_addr);
|
||||
if (addr_v6 == toIPv6(IPAddress(&sin.sin_addr, sizeof(sin.sin_addr))))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
else if (ai->ai_family == AF_INET6)
|
||||
{
|
||||
const auto & sin = *reinterpret_cast<const sockaddr_in6*>(ai->ai_addr);
|
||||
if (addr_v6 == IPAddress(&sin.sin6_addr, sizeof(sin.sin6_addr), sin.sin6_scope_id))
|
||||
{
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
auto & getIsAddressOfHostCache()
|
||||
{
|
||||
static SimpleCache<decltype(isAddressOfHostImpl), isAddressOfHostImpl> cache;
|
||||
return cache;
|
||||
}
|
||||
|
||||
/// Whether a specified address is one of the addresses of a specified host.
|
||||
bool isAddressOfHost(const IPAddress & address, const String & host)
|
||||
{
|
||||
/// We need to cache DNS requests.
|
||||
return getIsAddressOfHostCache()(address, host);
|
||||
IPAddress addr_v6 = toIPv6(address);
|
||||
|
||||
auto host_addresses = DNSResolver::instance().resolveHostAll(host);
|
||||
|
||||
for (const auto & addr : host_addresses)
|
||||
{
|
||||
if (addr.family() == IPAddress::Family::IPv4 && addr_v6 == toIPv6(addr))
|
||||
return true;
|
||||
else if (addr.family() == IPAddress::Family::IPv6 && addr_v6 == addr)
|
||||
return true;
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Helper function for isAddressOfLocalhost().
|
||||
@ -147,16 +104,10 @@ namespace
|
||||
return boost::range::find(local_addresses, toIPv6(address)) != local_addresses.end();
|
||||
}
|
||||
|
||||
/// Helper function for getHostByAddress().
|
||||
String getHostByAddressImpl(const IPAddress & address)
|
||||
/// Returns the host name by its address.
|
||||
String getHostByAddress(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 err = getnameinfo(sock_addr.addr(), sock_addr.length(), host, sizeof(host), nullptr, 0, NI_NAMEREQD);
|
||||
if (err)
|
||||
throw Exception("Cannot getnameinfo(" + address.toString() + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR);
|
||||
String host = DNSResolver::instance().reverseResolve(address);
|
||||
|
||||
/// Check that PTR record is resolved back to client address
|
||||
if (!isAddressOfHost(address, host))
|
||||
@ -165,19 +116,6 @@ namespace
|
||||
return host;
|
||||
}
|
||||
|
||||
auto & getHostByAddressCache()
|
||||
{
|
||||
static SimpleCache<decltype(getHostByAddressImpl), &getHostByAddressImpl> cache;
|
||||
return cache;
|
||||
}
|
||||
|
||||
/// Returns the host name by its address.
|
||||
String getHostByAddress(const IPAddress & address)
|
||||
{
|
||||
/// We need to cache DNS requests.
|
||||
return getHostByAddressCache()(address);
|
||||
}
|
||||
|
||||
|
||||
void parseLikePatternIfIPSubnet(const String & pattern, IPSubnet & subnet, IPAddress::Family address_family)
|
||||
{
|
||||
@ -376,10 +314,4 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
|
||||
return false;
|
||||
}
|
||||
|
||||
void AllowedClientHosts::dropDNSCaches()
|
||||
{
|
||||
getIsAddressOfHostCache().drop();
|
||||
getHostByAddressCache().drop();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -114,8 +114,6 @@ public:
|
||||
friend bool operator ==(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs);
|
||||
friend bool operator !=(const AllowedClientHosts & lhs, const AllowedClientHosts & rhs) { return !(lhs == rhs); }
|
||||
|
||||
static void dropDNSCaches();
|
||||
|
||||
private:
|
||||
std::vector<IPAddress> addresses;
|
||||
std::vector<IPSubnet> subnets;
|
||||
|
@ -4,20 +4,32 @@
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <Poco/Net/DNS.h>
|
||||
#include <Poco/Net/NetException.h>
|
||||
#include <Poco/NumberParser.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <arpa/inet.h>
|
||||
#include <atomic>
|
||||
#include <optional>
|
||||
#include <string_view>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern Event DNSError;
|
||||
}
|
||||
|
||||
namespace std
|
||||
{
|
||||
template<> struct hash<Poco::Net::IPAddress>
|
||||
{
|
||||
size_t operator()(const Poco::Net::IPAddress & address) const noexcept
|
||||
{
|
||||
std::string_view addr(static_cast<const char *>(address.addr()), address.length());
|
||||
std::hash<std::string_view> hash_impl;
|
||||
return hash_impl(addr);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -25,6 +37,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int DNS_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -76,16 +89,44 @@ static void splitHostAndPort(const std::string & host_and_port, std::string & ou
|
||||
}
|
||||
}
|
||||
|
||||
static Poco::Net::IPAddress resolveIPAddressImpl(const std::string & host)
|
||||
static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host)
|
||||
{
|
||||
Poco::Net::IPAddress ip;
|
||||
|
||||
/// NOTE: Poco::Net::DNS::resolveOne(host) doesn't work for IP addresses like 127.0.0.2
|
||||
/// Therefore we use SocketAddress constructor with dummy port to resolve IP
|
||||
return Poco::Net::SocketAddress(host, 0U).host();
|
||||
if (Poco::Net::IPAddress::tryParse(host, ip))
|
||||
return DNSResolver::IPAddresses(1, ip);
|
||||
|
||||
/// Family: AF_UNSPEC
|
||||
/// AI_ALL is required for checking if client is allowed to connect from an address
|
||||
auto flags = Poco::Net::DNS::DNS_HINT_AI_V4MAPPED | Poco::Net::DNS::DNS_HINT_AI_ALL;
|
||||
/// Do not resolve IPv6 (or IPv4) if no local IPv6 (or IPv4) addresses are configured.
|
||||
/// It should not affect client address checking, since client cannot connect from IPv6 address
|
||||
/// if server has no IPv6 addresses.
|
||||
flags |= Poco::Net::DNS::DNS_HINT_AI_ADDRCONFIG;
|
||||
auto addresses = Poco::Net::DNS::hostByName(host, flags).addresses();
|
||||
if (addresses.empty())
|
||||
throw Exception("Not found address of host: " + host, ErrorCodes::DNS_ERROR);
|
||||
|
||||
return addresses;
|
||||
}
|
||||
|
||||
static String reverseResolveImpl(const Poco::Net::IPAddress & address)
|
||||
{
|
||||
Poco::Net::SocketAddress sock_addr(address, 0);
|
||||
|
||||
/// Resolve by hand, because Poco::Net::DNS::hostByAddress(...) does getaddrinfo(...) after getnameinfo(...)
|
||||
char host[1024];
|
||||
int err = getnameinfo(sock_addr.addr(), sock_addr.length(), host, sizeof(host), nullptr, 0, NI_NAMEREQD);
|
||||
if (err)
|
||||
throw Exception("Cannot getnameinfo(" + address.toString() + "): " + gai_strerror(err), ErrorCodes::DNS_ERROR);
|
||||
return host;
|
||||
}
|
||||
|
||||
struct DNSResolver::Impl
|
||||
{
|
||||
SimpleCache<decltype(resolveIPAddressImpl), &resolveIPAddressImpl> cache_host;
|
||||
SimpleCache<decltype(reverseResolveImpl), &reverseResolveImpl> cache_address;
|
||||
|
||||
std::mutex drop_mutex;
|
||||
std::mutex update_mutex;
|
||||
@ -95,18 +136,25 @@ struct DNSResolver::Impl
|
||||
|
||||
/// Store hosts, which was asked to resolve from last update of DNS cache.
|
||||
NameSet new_hosts;
|
||||
std::unordered_set<Poco::Net::IPAddress> new_addresses;
|
||||
|
||||
/// Store all hosts, which was whenever asked to resolve
|
||||
NameSet known_hosts;
|
||||
std::unordered_set<Poco::Net::IPAddress> known_addresses;
|
||||
|
||||
/// If disabled, will not make cache lookups, will resolve addresses manually on each call
|
||||
std::atomic<bool> disable_cache{false};
|
||||
};
|
||||
|
||||
|
||||
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()) {}
|
||||
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(&Poco::Logger::get("DNSResolver")) {}
|
||||
|
||||
Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host)
|
||||
{
|
||||
return resolveHostAll(host).front();
|
||||
}
|
||||
|
||||
DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host)
|
||||
{
|
||||
if (impl->disable_cache)
|
||||
return resolveIPAddressImpl(host);
|
||||
@ -125,7 +173,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_an
|
||||
splitHostAndPort(host_and_port, host, port);
|
||||
|
||||
addToNewHosts(host);
|
||||
return Poco::Net::SocketAddress(impl->cache_host(host), port);
|
||||
return Poco::Net::SocketAddress(impl->cache_host(host).front(), port);
|
||||
}
|
||||
|
||||
Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, UInt16 port)
|
||||
@ -134,17 +182,29 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U
|
||||
return Poco::Net::SocketAddress(host, port);
|
||||
|
||||
addToNewHosts(host);
|
||||
return Poco::Net::SocketAddress(impl->cache_host(host), port);
|
||||
return Poco::Net::SocketAddress(impl->cache_host(host).front(), port);
|
||||
}
|
||||
|
||||
String DNSResolver::reverseResolve(const Poco::Net::IPAddress & address)
|
||||
{
|
||||
if (impl->disable_cache)
|
||||
return reverseResolveImpl(address);
|
||||
|
||||
addToNewAddresses(address);
|
||||
return impl->cache_address(address);
|
||||
}
|
||||
|
||||
void DNSResolver::dropCache()
|
||||
{
|
||||
impl->cache_host.drop();
|
||||
impl->cache_address.drop();
|
||||
|
||||
std::scoped_lock lock(impl->update_mutex, impl->drop_mutex);
|
||||
|
||||
impl->known_hosts.clear();
|
||||
impl->known_addresses.clear();
|
||||
impl->new_hosts.clear();
|
||||
impl->new_addresses.clear();
|
||||
impl->host_name.reset();
|
||||
}
|
||||
|
||||
@ -166,34 +226,27 @@ String DNSResolver::getHostName()
|
||||
return *impl->host_name;
|
||||
}
|
||||
|
||||
bool DNSResolver::updateCache()
|
||||
static const String & cacheElemToString(const String & str) { return str; }
|
||||
static String cacheElemToString(const Poco::Net::IPAddress & addr) { return addr.toString(); }
|
||||
|
||||
template<typename UpdateF, typename ElemsT>
|
||||
bool DNSResolver::updateCacheImpl(UpdateF update_func, const ElemsT & elems, const String & log_msg)
|
||||
{
|
||||
{
|
||||
std::lock_guard lock(impl->drop_mutex);
|
||||
for (const auto & host : impl->new_hosts)
|
||||
impl->known_hosts.insert(host);
|
||||
impl->new_hosts.clear();
|
||||
|
||||
impl->host_name.emplace(Poco::Net::DNS::hostName());
|
||||
}
|
||||
|
||||
std::lock_guard lock(impl->update_mutex);
|
||||
|
||||
bool updated = false;
|
||||
String lost_hosts;
|
||||
for (const auto & host : impl->known_hosts)
|
||||
String lost_elems;
|
||||
for (const auto & elem : elems)
|
||||
{
|
||||
try
|
||||
{
|
||||
updated |= updateHost(host);
|
||||
updated |= (this->*update_func)(elem);
|
||||
}
|
||||
catch (const Poco::Net::NetException &)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::DNSError);
|
||||
|
||||
if (!lost_hosts.empty())
|
||||
lost_hosts += ", ";
|
||||
lost_hosts += host;
|
||||
if (!lost_elems.empty())
|
||||
lost_elems += ", ";
|
||||
lost_elems += cacheElemToString(elem);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
@ -201,12 +254,41 @@ bool DNSResolver::updateCache()
|
||||
}
|
||||
}
|
||||
|
||||
if (!lost_hosts.empty())
|
||||
LOG_INFO(&Poco::Logger::get("DNSResolver"), "Cached hosts not found: {}", lost_hosts);
|
||||
if (!lost_elems.empty())
|
||||
LOG_INFO(log, log_msg, lost_elems);
|
||||
|
||||
return updated;
|
||||
}
|
||||
|
||||
bool DNSResolver::updateCache()
|
||||
{
|
||||
LOG_DEBUG(log, "Updating DNS cache");
|
||||
|
||||
{
|
||||
std::lock_guard lock(impl->drop_mutex);
|
||||
|
||||
for (auto & host : impl->new_hosts)
|
||||
impl->known_hosts.insert(std::move(host));
|
||||
impl->new_hosts.clear();
|
||||
|
||||
for (auto & address : impl->new_addresses)
|
||||
impl->known_addresses.insert(std::move(address));
|
||||
impl->new_addresses.clear();
|
||||
|
||||
impl->host_name.emplace(Poco::Net::DNS::hostName());
|
||||
}
|
||||
|
||||
/// FIXME Updating may take a long time becouse we cannot manage timeouts of getaddrinfo(...) and getnameinfo(...).
|
||||
/// DROP DNS CACHE will wait on update_mutex (possibly while holding drop_mutex)
|
||||
std::lock_guard lock(impl->update_mutex);
|
||||
|
||||
bool hosts_updated = updateCacheImpl(&DNSResolver::updateHost, impl->known_hosts, "Cached hosts not found: {}");
|
||||
updateCacheImpl(&DNSResolver::updateAddress, impl->known_addresses, "Cached addresses not found: {}");
|
||||
|
||||
LOG_DEBUG(log, "Updated DNS cache");
|
||||
return hosts_updated;
|
||||
}
|
||||
|
||||
bool DNSResolver::updateHost(const String & host)
|
||||
{
|
||||
/// Usage of updateHost implies that host is already in cache and there is no extra computations
|
||||
@ -215,12 +297,25 @@ bool DNSResolver::updateHost(const String & host)
|
||||
return old_value != impl->cache_host(host);
|
||||
}
|
||||
|
||||
bool DNSResolver::updateAddress(const Poco::Net::IPAddress & address)
|
||||
{
|
||||
auto old_value = impl->cache_address(address);
|
||||
impl->cache_address.update(address);
|
||||
return old_value == impl->cache_address(address);
|
||||
}
|
||||
|
||||
void DNSResolver::addToNewHosts(const String & host)
|
||||
{
|
||||
std::lock_guard lock(impl->drop_mutex);
|
||||
impl->new_hosts.insert(host);
|
||||
}
|
||||
|
||||
void DNSResolver::addToNewAddresses(const Poco::Net::IPAddress & address)
|
||||
{
|
||||
std::lock_guard lock(impl->drop_mutex);
|
||||
impl->new_addresses.insert(address);
|
||||
}
|
||||
|
||||
DNSResolver::~DNSResolver() = default;
|
||||
|
||||
DNSResolver & DNSResolver::instance()
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Core/Names.h>
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -16,18 +17,26 @@ namespace DB
|
||||
class DNSResolver : private boost::noncopyable
|
||||
{
|
||||
public:
|
||||
typedef std::vector<Poco::Net::IPAddress> IPAddresses;
|
||||
|
||||
static DNSResolver & instance();
|
||||
|
||||
DNSResolver(const DNSResolver &) = delete;
|
||||
|
||||
/// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolve its IP
|
||||
/// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolves its IP
|
||||
Poco::Net::IPAddress resolveHost(const std::string & host);
|
||||
|
||||
/// Accepts host names like 'example.com:port' or '127.0.0.1:port' or '[::1]:port' and resolve its IP and port
|
||||
/// Accepts host names like 'example.com' or '127.0.0.1' or '::1' and resolves all its IPs
|
||||
IPAddresses resolveHostAll(const std::string & host);
|
||||
|
||||
/// Accepts host names like 'example.com:port' or '127.0.0.1:port' or '[::1]:port' and resolves its IP and port
|
||||
Poco::Net::SocketAddress resolveAddress(const std::string & host_and_port);
|
||||
|
||||
Poco::Net::SocketAddress resolveAddress(const std::string & host, UInt16 port);
|
||||
|
||||
/// Accepts host IP and resolves its host name
|
||||
String reverseResolve(const Poco::Net::IPAddress & address);
|
||||
|
||||
/// Get this server host name
|
||||
String getHostName();
|
||||
|
||||
@ -44,16 +53,21 @@ public:
|
||||
~DNSResolver();
|
||||
|
||||
private:
|
||||
template<typename UpdateF, typename ElemsT>
|
||||
bool updateCacheImpl(UpdateF update_func, const ElemsT & elems, const String & log_msg);
|
||||
|
||||
DNSResolver();
|
||||
|
||||
struct Impl;
|
||||
std::unique_ptr<Impl> impl;
|
||||
Poco::Logger * log;
|
||||
|
||||
/// Returns true if IP of host has been changed.
|
||||
/// Updates cached value and returns true it has been changed.
|
||||
bool updateHost(const String & host);
|
||||
bool updateAddress(const Poco::Net::IPAddress & address);
|
||||
|
||||
void addToNewHosts(const String & host);
|
||||
void addToNewAddresses(const Poco::Net::IPAddress & address);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -204,7 +204,6 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
case Type::DROP_DNS_CACHE:
|
||||
context.checkAccess(AccessType::SYSTEM_DROP_DNS_CACHE);
|
||||
DNSResolver::instance().dropCache();
|
||||
AllowedClientHosts::dropDNSCaches();
|
||||
/// Reinitialize clusters to update their resolved_addresses
|
||||
system_context.reloadClusterConfig();
|
||||
break;
|
||||
|
@ -76,8 +76,6 @@ node3 = cluster.add_instance('node3', main_configs=['configs/listen_host.xml'],
|
||||
with_zookeeper=True, ipv6_address='2001:3984:3989::1:1113')
|
||||
node4 = cluster.add_instance('node4', main_configs=['configs/remote_servers.xml', 'configs/listen_host.xml', 'configs/dns_update_short.xml'],
|
||||
with_zookeeper=True, ipv6_address='2001:3984:3989::1:1114')
|
||||
node5 = cluster.add_instance('node5', main_configs=['configs/listen_host.xml', 'configs/dns_update_short.xml'],
|
||||
user_configs=['configs/users_with_hostname.xml'], ipv6_address='2001:3984:3989::1:1115')
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
def cluster_with_dns_cache_update():
|
||||
@ -142,24 +140,39 @@ def test_dns_cache_update(cluster_with_dns_cache_update):
|
||||
assert TSV(node4.query("SELECT DISTINCT host_name, host_address FROM system.clusters WHERE cluster='lost_host_cluster'")) == TSV("lost_host\t127.0.0.1\n")
|
||||
assert TSV(node4.query("SELECT hostName()")) == TSV("node4")
|
||||
|
||||
def test_user_access_ip_change(cluster_with_dns_cache_update):
|
||||
assert node3.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n"
|
||||
assert node4.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n"
|
||||
# Check SYSTEM DROP DNS CACHE on node5 and background cache update on node6
|
||||
node5 = cluster.add_instance('node5', main_configs=['configs/listen_host.xml', 'configs/dns_update_long.xml'],
|
||||
user_configs=['configs/users_with_hostname.xml'], ipv6_address='2001:3984:3989::1:1115')
|
||||
node6 = cluster.add_instance('node6', main_configs=['configs/listen_host.xml', 'configs/dns_update_short.xml'],
|
||||
user_configs=['configs/users_with_hostname.xml'], ipv6_address='2001:3984:3989::1:1116')
|
||||
|
||||
set_hosts(node5, ['127.255.255.255 node3', '2001:3984:3989::1:8884 unknown_host'])
|
||||
@pytest.mark.parametrize("node", [node5, node6])
|
||||
def test_user_access_ip_change(cluster_with_dns_cache_update, node):
|
||||
node_name = node.name
|
||||
node_num = node.name[-1]
|
||||
# getaddrinfo(...) may hang for a log time without this options
|
||||
node.exec_in_container(['bash', '-c', 'echo -e "options timeout:1\noptions attempts:2" >> /etc/resolv.conf'], privileged=True, user='root')
|
||||
|
||||
cluster.restart_instance_with_ip_change(node3, "2001:3984:3989::1:8883")
|
||||
cluster.restart_instance_with_ip_change(node4, "2001:3984:3989::1:8884")
|
||||
assert node3.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) == "0\n"
|
||||
assert node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name)) == "0\n"
|
||||
|
||||
set_hosts(node, ['127.255.255.255 node3', '2001:3984:3989::1:88{}4 unknown_host'.format(node_num)])
|
||||
|
||||
cluster.restart_instance_with_ip_change(node3, "2001:3984:3989::1:88{}3".format(node_num))
|
||||
cluster.restart_instance_with_ip_change(node4, "2001:3984:3989::1:88{}4".format(node_num))
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node3.query("SELECT * FROM remote('node5', 'system', 'one')")
|
||||
node3.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name))
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node4.query("SELECT * FROM remote('node5', 'system', 'one')")
|
||||
node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name))
|
||||
# now wrong addresses are cached
|
||||
|
||||
set_hosts(node5, [])
|
||||
# client is not allowed to connect, so execute it directly in container to send query from localhost
|
||||
node5.exec_in_container(['bash', '-c', 'clickhouse client -q "SYSTEM DROP DNS CACHE"'], privileged=True, user='root')
|
||||
set_hosts(node, [])
|
||||
retry_count = 60
|
||||
if node_name == 'node5':
|
||||
# client is not allowed to connect, so execute it directly in container to send query from localhost
|
||||
node.exec_in_container(['bash', '-c', 'clickhouse client -q "SYSTEM DROP DNS CACHE"'], privileged=True, user='root')
|
||||
retry_count = 1
|
||||
|
||||
assert node3.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n"
|
||||
assert node4.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n"
|
||||
assert_eq_with_retry(node3, "SELECT * FROM remote('{}', 'system', 'one')".format(node_name), "0", retry_count=retry_count, sleep_time=1)
|
||||
assert_eq_with_retry(node4, "SELECT * FROM remote('{}', 'system', 'one')".format(node_name), "0", retry_count=retry_count, sleep_time=1)
|
||||
|
Loading…
Reference in New Issue
Block a user