Merge remote-tracking branch 'origin/master' into improve-integration-tests-5

This commit is contained in:
Yatsishin Ilya 2024-08-19 16:32:08 +00:00
commit 718957886d
10 changed files with 176 additions and 26 deletions

View File

@ -482,7 +482,7 @@ jobs:
if: ${{ !failure() }}
run: |
# update overall ci report
python3 finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }}
python3 ./tests/ci/finish_check.py --wf-status ${{ contains(needs.*.result, 'failure') && 'failure' || 'success' }}
- name: Check Workflow results
if: ${{ !cancelled() }}
run: |
@ -490,5 +490,4 @@ jobs:
cat > "$WORKFLOW_RESULT_FILE" << 'EOF'
${{ toJson(needs) }}
EOF
python3 ./tests/ci/ci_buddy.py --check-wf-status

View File

@ -1769,6 +1769,8 @@ try
new_server_settings.http_connections_store_limit,
});
DNSResolver::instance().setFilterSettings(new_server_settings.dns_allow_resolve_names_to_ipv4, new_server_settings.dns_allow_resolve_names_to_ipv6);
if (global_context->isServerCompletelyStarted())
CannotAllocateThreadFaultInjector::setFaultProbability(new_server_settings.cannot_allocate_thread_fault_injection_probability);

View File

@ -12,6 +12,7 @@
#include <atomic>
#include <optional>
#include <string_view>
#include "Common/MultiVersion.h"
#include <unordered_set>
#include "DNSPTRResolverProvider.h"
@ -139,12 +140,6 @@ DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host)
return addresses;
}
DNSResolver::IPAddresses resolveIPAddressWithCache(CacheBase<std::string, DNSResolver::CacheEntry> & cache, const std::string & host)
{
auto [result, _ ] = cache.getOrSet(host, [&host]() {return std::make_shared<DNSResolver::CacheEntry>(resolveIPAddressImpl(host), std::chrono::system_clock::now());});
return result->addresses;
}
std::unordered_set<String> reverseResolveImpl(const Poco::Net::IPAddress & address)
{
auto ptr_resolver = DB::DNSPTRResolverProvider::get();
@ -198,21 +193,89 @@ struct DNSResolver::Impl
std::atomic<bool> disable_cache{false};
};
struct DNSResolver::AddressFilter
{
struct DNSFilterSettings
{
bool dns_allow_resolve_names_to_ipv4{true};
bool dns_allow_resolve_names_to_ipv6{true};
};
DNSResolver::DNSResolver() : impl(std::make_unique<DNSResolver::Impl>()), log(getLogger("DNSResolver")) {}
AddressFilter() : settings(std::make_unique<DNSFilterSettings>()) {}
void performAddressFiltering(DNSResolver::IPAddresses & addresses) const
{
const auto current_settings = settings.get();
bool dns_resolve_ipv4 = current_settings->dns_allow_resolve_names_to_ipv4;
bool dns_resolve_ipv6 = current_settings->dns_allow_resolve_names_to_ipv6;
if (dns_resolve_ipv4 && dns_resolve_ipv6)
{
return;
}
if (!dns_resolve_ipv4 && !dns_resolve_ipv6)
{
throw Exception(ErrorCodes::BAD_ARGUMENTS, "DNS can't resolve any address, because dns_resolve_ipv6_interfaces and dns_resolve_ipv4_interfaces both are disabled");
}
std::erase_if(addresses, [dns_resolve_ipv6, dns_resolve_ipv4](const Poco::Net::IPAddress& address)
{
return (address.family() == Poco::Net::IPAddress::IPv6 && !dns_resolve_ipv6)
|| (address.family() == Poco::Net::IPAddress::IPv4 && !dns_resolve_ipv4);
});
}
void setSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6)
{
settings.set(std::make_unique<DNSFilterSettings>(dns_allow_resolve_names_to_ipv4, dns_allow_resolve_names_to_ipv6));
}
MultiVersion<DNSFilterSettings> settings;
};
DNSResolver::DNSResolver()
: impl(std::make_unique<DNSResolver::Impl>())
, addressFilter(std::make_unique<DNSResolver::AddressFilter>())
, log(getLogger("DNSResolver")) {}
DNSResolver::IPAddresses DNSResolver::getResolvedIPAdressessWithFiltering(const std::string & host)
{
auto addresses = resolveIPAddressImpl(host);
addressFilter->performAddressFiltering(addresses);
if (addresses.empty())
{
ProfileEvents::increment(ProfileEvents::DNSError);
throw DB::NetException(ErrorCodes::DNS_ERROR, "After filtering there are no resolved address for host({}).", host);
}
return addresses;
}
DNSResolver::IPAddresses DNSResolver::resolveIPAddressWithCache(const std::string & host)
{
auto [result, _ ] = impl->cache_host.getOrSet(host, [&host, this]() {return std::make_shared<DNSResolver::CacheEntry>(getResolvedIPAdressessWithFiltering(host), std::chrono::system_clock::now());});
return result->addresses;
}
Poco::Net::IPAddress DNSResolver::resolveHost(const std::string & host)
{
return pickAddress(resolveHostAll(host)); // random order -> random pick
}
void DNSResolver::setFilterSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6)
{
addressFilter->setSettings(dns_allow_resolve_names_to_ipv4, dns_allow_resolve_names_to_ipv6);
}
DNSResolver::IPAddresses DNSResolver::resolveHostAllInOriginOrder(const std::string & host)
{
if (impl->disable_cache)
return resolveIPAddressImpl(host);
return getResolvedIPAdressessWithFiltering(host);
addToNewHosts(host);
return resolveIPAddressWithCache(impl->cache_host, host);
return resolveIPAddressWithCache(host);
}
DNSResolver::IPAddresses DNSResolver::resolveHostAll(const std::string & host)
@ -232,7 +295,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host_an
splitHostAndPort(host_and_port, host, port);
addToNewHosts(host);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(impl->cache_host, host)), port);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(host)), port);
}
Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, UInt16 port)
@ -241,7 +304,7 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U
return Poco::Net::SocketAddress(host, port);
addToNewHosts(host);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(impl->cache_host, host)), port);
return Poco::Net::SocketAddress(pickAddress(resolveIPAddressWithCache(host)), port);
}
std::vector<Poco::Net::SocketAddress> DNSResolver::resolveAddressList(const std::string & host, UInt16 port)
@ -254,7 +317,7 @@ std::vector<Poco::Net::SocketAddress> DNSResolver::resolveAddressList(const std:
if (!impl->disable_cache)
addToNewHosts(host);
std::vector<Poco::Net::IPAddress> ips = impl->disable_cache ? hostByName(host) : resolveIPAddressWithCache(impl->cache_host, host);
std::vector<Poco::Net::IPAddress> ips = impl->disable_cache ? hostByName(host) : resolveIPAddressWithCache(host);
auto ips_end = std::unique(ips.begin(), ips.end());
addresses.reserve(ips_end - ips.begin());
@ -419,8 +482,8 @@ bool DNSResolver::updateCache(UInt32 max_consecutive_failures)
bool DNSResolver::updateHost(const String & host)
{
const auto old_value = resolveIPAddressWithCache(impl->cache_host, host);
auto new_value = resolveIPAddressImpl(host);
const auto old_value = resolveIPAddressWithCache(host);
auto new_value = getResolvedIPAdressessWithFiltering(host);
const bool result = old_value != new_value;
impl->cache_host.set(host, std::make_shared<DNSResolver::CacheEntry>(std::move(new_value), std::chrono::system_clock::now()));
return result;

View File

@ -68,6 +68,8 @@ public:
/// Returns true if IP of any host has been changed or an element was dropped (too many failures)
bool updateCache(UInt32 max_consecutive_failures);
void setFilterSettings(bool dns_allow_resolve_names_to_ipv4, bool dns_allow_resolve_names_to_ipv6);
/// Returns a copy of cache entries
std::vector<std::pair<std::string, CacheEntry>> cacheEntries() const;
@ -86,6 +88,10 @@ private:
struct Impl;
std::unique_ptr<Impl> impl;
struct AddressFilter;
std::unique_ptr<AddressFilter> addressFilter;
LoggerPtr log;
/// Updates cached value and returns true it has been changed.
@ -94,6 +100,9 @@ private:
void addToNewHosts(const String & host);
void addToNewAddresses(const Poco::Net::IPAddress & address);
IPAddresses resolveIPAddressWithCache(const std::string & host);
IPAddresses getResolvedIPAdressessWithFiltering(const std::string & host);
};
}

View File

@ -106,6 +106,8 @@ namespace DB
M(UInt64, dns_cache_max_entries, 10000, "Internal DNS cache max entries.", 0) \
M(Int32, dns_cache_update_period, 15, "Internal DNS cache update period in seconds.", 0) \
M(UInt32, dns_max_consecutive_failures, 10, "Max DNS resolve failures of a hostname before dropping the hostname from ClickHouse DNS cache.", 0) \
M(Bool, dns_allow_resolve_names_to_ipv4, true, "Allows resolve names to ipv4 addresses.", 0) \
M(Bool, dns_allow_resolve_names_to_ipv6, true, "Allows resolve names to ipv6 addresses.", 0) \
\
M(UInt64, max_table_size_to_drop, 50000000000lu, "If size of a table is greater than this value (in bytes) than table could not be dropped with any DROP query.", 0) \
M(UInt64, max_partition_size_to_drop, 50000000000lu, "Same as max_table_size_to_drop, but for the partitions.", 0) \

View File

@ -18,13 +18,15 @@
#include <Core/Settings.h>
#include <IO/Operators.h>
#include "config.h"
#if USE_SIMDJSON
#include <Common/JSONParsers/SimdJSONParser.h>
# include <Common/JSONParsers/SimdJSONParser.h>
#elif USE_RAPIDJSON
# include <Common/JSONParsers/RapidJSONParser.h>
#else
# include <Common/JSONParsers/DummyJSONParser.h>
#endif
#if USE_RAPIDJSON
#include <Common/JSONParsers/RapidJSONParser.h>
#endif
#include <Common/JSONParsers/DummyJSONParser.h>
namespace DB
{
@ -105,7 +107,7 @@ SerializationPtr DataTypeObject::doGetDefaultSerialization() const
switch (schema_format)
{
case SchemaFormat::JSON:
#ifdef USE_SIMDJSON
#if USE_SIMDJSON
return std::make_shared<SerializationJSON<SimdJSONParser>>(
std::move(typed_path_serializations),
paths_to_skip,

View File

@ -163,6 +163,7 @@ class CI:
tidy=True,
comment="clang-tidy is used for static analysis",
),
timeout=10800,
),
BuildNames.BINARY_DARWIN: CommonJobConfigs.BUILD.with_properties(
build_config=BuildConfig(

View File

@ -2572,12 +2572,12 @@ def do_run_tests(jobs, test_suite: TestSuite):
try:
clickhouse_execute(
args,
query="SELECT 1 /*hang up check*/",
max_http_retries=5,
timeout=20,
query="SELECT 1 /*hung check*/",
max_http_retries=20,
timeout=10,
)
except Exception:
print("Hang up check failed")
print("Hung check failed")
server_died.set()
if server_died.is_set():

View File

@ -1,6 +1,7 @@
[
"test_dns_cache/test.py::test_dns_cache_update",
"test_dns_cache/test.py::test_ip_change_drop_dns_cache",
"test_dns_cache/test.py::test_dns_resolver_filter",
"test_dns_cache/test.py::test_ip_change_update_dns_cache",
"test_dns_cache/test.py::test_user_access_ip_change[node0]",
"test_dns_cache/test.py::test_user_access_ip_change[node1]",

View File

@ -317,3 +317,74 @@ def test_host_is_drop_from_cache_after_consecutive_failures(
assert node4.wait_for_log_line(
"Cached hosts dropped:.*InvalidHostThatDoesNotExist.*"
)
node7 = cluster.add_instance(
"node7",
main_configs=["configs/listen_host.xml", "configs/dns_update_long.xml"],
with_zookeeper=True,
ipv6_address="2001:3984:3989::1:1117",
ipv4_address="10.5.95.17",
)
def _render_filter_config(allow_ipv4, allow_ipv6):
config = f"""
<clickhouse>
<dns_allow_resolve_names_to_ipv4>{int(allow_ipv4)}</dns_allow_resolve_names_to_ipv4>
<dns_allow_resolve_names_to_ipv6>{int(allow_ipv6)}</dns_allow_resolve_names_to_ipv6>
</clickhouse>
"""
return config
@pytest.mark.parametrize(
"allow_ipv4, allow_ipv6",
[
(True, False),
(False, True),
(False, False),
],
)
def test_dns_resolver_filter(cluster_without_dns_cache_update, allow_ipv4, allow_ipv6):
node = node7
host_ipv6 = node.ipv6_address
host_ipv4 = node.ipv4_address
node.set_hosts(
[
(host_ipv6, "test_host"),
(host_ipv4, "test_host"),
]
)
node.replace_config(
"/etc/clickhouse-server/config.d/dns_filter.xml",
_render_filter_config(allow_ipv4, allow_ipv6),
)
node.query("SYSTEM RELOAD CONFIG")
node.query("SYSTEM DROP DNS CACHE")
node.query("SYSTEM DROP CONNECTIONS CACHE")
if not allow_ipv4 and not allow_ipv6:
with pytest.raises(QueryRuntimeException):
node.query("SELECT * FROM remote('lost_host', 'system', 'one')")
else:
node.query("SELECT * FROM remote('test_host', system, one)")
assert (
node.query(
"SELECT ip_address FROM system.dns_cache WHERE hostname='test_host'"
)
== f"{host_ipv4 if allow_ipv4 else host_ipv6}\n"
)
node.exec_in_container(
[
"bash",
"-c",
"rm /etc/clickhouse-server/config.d/dns_filter.xml",
],
privileged=True,
user="root",
)
node.query("SYSTEM RELOAD CONFIG")