mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
drop access cache on DROP DNS CACHE
This commit is contained in:
parent
d952e5aba2
commit
9ca144cab9
@ -93,12 +93,17 @@ namespace
|
||||
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.
|
||||
static SimpleCache<decltype(isAddressOfHostImpl), isAddressOfHostImpl> cache;
|
||||
return cache(address, host);
|
||||
return getIsAddressOfHostCache()(address, host);
|
||||
}
|
||||
|
||||
/// Helper function for isAddressOfLocalhost().
|
||||
@ -160,12 +165,17 @@ 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.
|
||||
static SimpleCache<decltype(getHostByAddressImpl), &getHostByAddressImpl> cache;
|
||||
return cache(address);
|
||||
return getHostByAddressCache()(address);
|
||||
}
|
||||
|
||||
|
||||
@ -366,4 +376,10 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const
|
||||
return false;
|
||||
}
|
||||
|
||||
void AllowedClientHosts::dropDNSCaches()
|
||||
{
|
||||
getIsAddressOfHostCache().drop();
|
||||
getHostByAddressCache().drop();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -114,6 +114,8 @@ 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;
|
||||
|
@ -21,6 +21,7 @@
|
||||
#include <Interpreters/TextLog.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Access/ContextAccess.h>
|
||||
#include <Access/AllowedClientHosts.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
@ -198,6 +199,7 @@ 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;
|
||||
|
@ -0,0 +1,13 @@
|
||||
<yandex>
|
||||
<users>
|
||||
<default>
|
||||
<networks>
|
||||
<host>node3</host>
|
||||
<host_regexp>[a-z]+4</host_regexp>
|
||||
<ip>::1</ip>
|
||||
<ip>127.0.0.1</ip>
|
||||
</networks>
|
||||
<profile>default</profile>
|
||||
</default>
|
||||
</users>
|
||||
</yandex>
|
@ -76,6 +76,8 @@ 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():
|
||||
@ -117,10 +119,12 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update):
|
||||
assert node3.query("SELECT count(*) from test_table_update") == "7\n"
|
||||
assert_eq_with_retry(node4, "SELECT count(*) from test_table_update", "7")
|
||||
|
||||
def set_hosts(node, hosts):
|
||||
new_content = '\\n'.join(['127.0.0.1 localhost', '::1 localhost'] + hosts)
|
||||
node.exec_in_container(['bash', '-c', 'echo -e "{}" > /etc/hosts'.format(new_content)], privileged=True, user='root')
|
||||
|
||||
def test_dns_cache_update(cluster_with_dns_cache_update):
|
||||
node4.exec_in_container(['bash', '-c', 'echo 127.0.0.1 localhost > /etc/hosts'], privileged=True, user='root')
|
||||
node4.exec_in_container(['bash', '-c', 'echo ::1 localhost >> /etc/hosts'], privileged=True, user='root')
|
||||
node4.exec_in_container(['bash', '-c', 'echo 127.255.255.255 lost_host >> /etc/hosts'], privileged=True, user='root')
|
||||
set_hosts(node4, ['127.255.255.255 lost_host'])
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node4.query("SELECT * FROM remote('lost_host', 'system', 'one')")
|
||||
@ -129,9 +133,7 @@ def test_dns_cache_update(cluster_with_dns_cache_update):
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node4.query("SELECT * FROM distributed_lost_host")
|
||||
|
||||
node4.exec_in_container(['bash', '-c', 'echo 127.0.0.1 localhost > /etc/hosts'], privileged=True, user='root')
|
||||
node4.exec_in_container(['bash', '-c', 'echo ::1 localhost >> /etc/hosts'], privileged=True, user='root')
|
||||
node4.exec_in_container(['bash', '-c', 'echo 127.0.0.1 lost_host >> /etc/hosts'], privileged=True, user='root')
|
||||
set_hosts(node4, ['127.0.0.1 lost_host'])
|
||||
|
||||
# Wait a bit until dns cache will be updated
|
||||
assert_eq_with_retry(node4, "SELECT * FROM remote('lost_host', 'system', 'one')", "0")
|
||||
@ -139,3 +141,25 @@ 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"
|
||||
|
||||
set_hosts(node5, ['127.255.255.255 node3', '2001:3984:3989::1:8884 unknown_host'])
|
||||
|
||||
cluster.restart_instance_with_ip_change(node3, "2001:3984:3989::1:8883")
|
||||
cluster.restart_instance_with_ip_change(node4, "2001:3984:3989::1:8884")
|
||||
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node3.query("SELECT * FROM remote('node5', 'system', 'one')")
|
||||
with pytest.raises(QueryRuntimeException):
|
||||
node4.query("SELECT * FROM remote('node5', 'system', 'one')")
|
||||
# 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')
|
||||
|
||||
assert node3.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n"
|
||||
assert node4.query("SELECT * FROM remote('node5', 'system', 'one')") == "0\n"
|
||||
|
Loading…
Reference in New Issue
Block a user