diff --git a/.gitmodules b/.gitmodules index 979f347e6ad..48c9747c158 100644 --- a/.gitmodules +++ b/.gitmodules @@ -277,3 +277,6 @@ [submodule "contrib/base-x"] path = contrib/base-x url = https://github.com/ClickHouse/base-x.git +[submodule "contrib/c-ares"] + path = contrib/c-ares + url = https://github.com/ClickHouse/c-ares diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index d4a3f164214..7c617004dbd 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -157,6 +157,7 @@ endif() add_contrib (sqlite-cmake sqlite-amalgamation) add_contrib (s2geometry-cmake s2geometry) add_contrib (base-x-cmake base-x) +add_contrib(c-ares-cmake c-ares) # Put all targets defined here and in subdirectories under "contrib/" folders in GUI-based IDEs. # Some of third-party projects may override CMAKE_FOLDER or FOLDER property of their targets, so they would not appear diff --git a/contrib/c-ares b/contrib/c-ares new file mode 160000 index 00000000000..afee6748b0b --- /dev/null +++ b/contrib/c-ares @@ -0,0 +1 @@ +Subproject commit afee6748b0b99acf4509d42fa37ac8422262f91b diff --git a/contrib/c-ares-cmake/CMakeLists.txt b/contrib/c-ares-cmake/CMakeLists.txt new file mode 100644 index 00000000000..603c1f8b65c --- /dev/null +++ b/contrib/c-ares-cmake/CMakeLists.txt @@ -0,0 +1,35 @@ +# Choose to build static or shared library for c-ares. +if (USE_STATIC_LIBRARIES) + set(CARES_STATIC ON CACHE BOOL "" FORCE) + set(CARES_SHARED OFF CACHE BOOL "" FORCE) +else () + set(CARES_STATIC OFF CACHE BOOL "" FORCE) + set(CARES_SHARED ON CACHE BOOL "" FORCE) +endif () + +# Disable looking for libnsl on a platforms that has gethostbyname in glibc +# +# c-ares searching for gethostbyname in the libnsl library, however in the +# version that shipped with gRPC it doing it wrong [1], since it uses +# CHECK_LIBRARY_EXISTS(), which will return TRUE even if the function exists in +# another dependent library. The upstream already contains correct macro [2], +# but it is not included in gRPC (even upstream gRPC, not the one that is +# shipped with clickhousee). +# +# [1]: https://github.com/c-ares/c-ares/blob/e982924acee7f7313b4baa4ee5ec000c5e373c30/CMakeLists.txt#L125 +# [2]: https://github.com/c-ares/c-ares/blob/44fbc813685a1fa8aa3f27fcd7544faf612d376a/CMakeLists.txt#L146 +# +# And because if you by some reason have libnsl [3] installed, clickhouse will +# reject to start w/o it. While this is completelly different library. +# +# [3]: https://packages.debian.org/bullseye/libnsl2 +if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") + set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) +endif() + +# Force use of c-ares inet_net_pton instead of libresolv one +set(HAVE_INET_NET_PTON OFF CACHE BOOL "" FORCE) + +add_subdirectory("../c-ares/" "../c-ares/") + +add_library(ch_contrib::c-ares ALIAS c-ares) \ No newline at end of file diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 520e04d198e..b1ed7e464b6 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -45,38 +45,11 @@ set(_gRPC_SSL_LIBRARIES OpenSSL::Crypto OpenSSL::SSL) # Use abseil-cpp from ClickHouse contrib, not from gRPC third_party. set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) -# Choose to build static or shared library for c-ares. -if (USE_STATIC_LIBRARIES) - set(CARES_STATIC ON CACHE BOOL "" FORCE) - set(CARES_SHARED OFF CACHE BOOL "" FORCE) -else () - set(CARES_STATIC OFF CACHE BOOL "" FORCE) - set(CARES_SHARED ON CACHE BOOL "" FORCE) -endif () - -# Disable looking for libnsl on a platforms that has gethostbyname in glibc -# -# c-ares searching for gethostbyname in the libnsl library, however in the -# version that shipped with gRPC it doing it wrong [1], since it uses -# CHECK_LIBRARY_EXISTS(), which will return TRUE even if the function exists in -# another dependent library. The upstream already contains correct macro [2], -# but it is not included in gRPC (even upstream gRPC, not the one that is -# shipped with clickhousee). -# -# [1]: https://github.com/c-ares/c-ares/blob/e982924acee7f7313b4baa4ee5ec000c5e373c30/CMakeLists.txt#L125 -# [2]: https://github.com/c-ares/c-ares/blob/44fbc813685a1fa8aa3f27fcd7544faf612d376a/CMakeLists.txt#L146 -# -# And because if you by some reason have libnsl [3] installed, clickhouse will -# reject to start w/o it. While this is completelly different library. -# -# [3]: https://packages.debian.org/bullseye/libnsl2 -if (NOT CMAKE_SYSTEM_NAME STREQUAL "SunOS") - set(HAVE_LIBNSL OFF CACHE BOOL "" FORCE) -endif() - # We don't want to build C# extensions. set(gRPC_BUILD_CSHARP_EXT OFF) +set(_gRPC_CARES_LIBRARIES ch_contrib::c-ares) +set(gRPC_CARES_PROVIDER "clickhouse" CACHE STRING "" FORCE) add_subdirectory("${_gRPC_SOURCE_DIR}" "${_gRPC_BINARY_DIR}") # The contrib/grpc/CMakeLists.txt redefined the PROTOBUF_GENERATE_GRPC_CPP() function for its own purposes, diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 2bbdd978e5e..6b8109a15b2 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -135,6 +135,7 @@ function clone_submodules contrib/replxx contrib/wyhash contrib/hashidsxx + contrib/c-ares ) git submodule sync diff --git a/docker/test/integration/runner/compose/docker_compose_coredns.yml b/docker/test/integration/runner/compose/docker_compose_coredns.yml new file mode 100644 index 00000000000..b329d4e0a46 --- /dev/null +++ b/docker/test/integration/runner/compose/docker_compose_coredns.yml @@ -0,0 +1,9 @@ +version: "2.3" + +services: + coredns: + image: coredns/coredns:latest + restart: always + volumes: + - ${COREDNS_CONFIG_DIR}/example.com:/example.com + - ${COREDNS_CONFIG_DIR}/Corefile:/Corefile diff --git a/src/Access/Common/AllowedClientHosts.cpp b/src/Access/Common/AllowedClientHosts.cpp index 85d7065d823..efbdf3924e8 100644 --- a/src/Access/Common/AllowedClientHosts.cpp +++ b/src/Access/Common/AllowedClientHosts.cpp @@ -110,18 +110,24 @@ namespace } /// Returns the host name by its address. - String getHostByAddress(const IPAddress & address) + Strings getHostsByAddress(const IPAddress & address) { - String host = DNSResolver::instance().reverseResolve(address); + auto hosts = DNSResolver::instance().reverseResolve(address); - /// 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); + if (hosts.empty()) + throw Exception(ErrorCodes::DNS_ERROR, "{} could not be resolved", address.toString()); - return host; + + for (const auto & host : hosts) + { + /// Check that PTR record is resolved back to client address + if (!isAddressOfHost(address, host)) + throw Exception(ErrorCodes::DNS_ERROR, "Host {} isn't resolved back to {}", host, address.toString()); + } + + return hosts; } - void parseLikePatternIfIPSubnet(const String & pattern, IPSubnet & subnet, IPAddress::Family address_family) { size_t slash = pattern.find('/'); @@ -520,20 +526,29 @@ bool AllowedClientHosts::contains(const IPAddress & client_address) const return true; /// Check `name_regexps`. - std::optional resolved_host; + std::optional resolved_hosts; auto check_name_regexp = [&](const String & name_regexp_) { try { if (boost::iequals(name_regexp_, "localhost")) return is_client_local(); - if (!resolved_host) - resolved_host = getHostByAddress(client_v6); - if (resolved_host->empty()) - return false; - Poco::RegularExpression re(name_regexp_); - Poco::RegularExpression::Match match; - return re.match(*resolved_host, match) != 0; + if (!resolved_hosts) + { + resolved_hosts = getHostsByAddress(client_address); + } + + for (const auto & host : resolved_hosts.value()) + { + Poco::RegularExpression re(name_regexp_); + Poco::RegularExpression::Match match; + if (re.match(host, match) != 0) + { + return true; + } + } + + return false; } catch (const Exception & e) { diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index cf8ae28df08..5c02905172c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -447,6 +447,9 @@ if (TARGET ch_contrib::avrocpp) dbms_target_link_libraries(PRIVATE ch_contrib::avrocpp) endif () +set_source_files_properties(Common/CaresPTRResolver.cpp PROPERTIES COMPILE_FLAGS -Wno-reserved-identifier) +target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::c-ares) + if (TARGET OpenSSL::Crypto) dbms_target_link_libraries (PRIVATE OpenSSL::Crypto) target_link_libraries (clickhouse_common_io PRIVATE OpenSSL::Crypto) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp new file mode 100644 index 00000000000..f6228e97c02 --- /dev/null +++ b/src/Common/CaresPTRResolver.cpp @@ -0,0 +1,109 @@ +#include "CaresPTRResolver.h" +#include +#include +#include +#include "ares.h" +#include "netdb.h" + +namespace DB +{ + + namespace ErrorCodes + { + extern const int DNS_ERROR; + } + + static void callback(void * arg, int status, int, struct hostent * host) + { + auto * ptr_records = reinterpret_cast*>(arg); + if (status == ARES_SUCCESS && host->h_aliases) + { + int i = 0; + while (auto * ptr_record = host->h_aliases[i]) + { + ptr_records->emplace_back(ptr_record); + i++; + } + } + } + + CaresPTRResolver::CaresPTRResolver(CaresPTRResolver::provider_token) : channel(nullptr) + { + /* + * ares_library_init is not thread safe. Currently, the only other usage of c-ares seems to be in grpc. + * In grpc, ares_library_init seems to be called only in Windows. + * See https://github.com/grpc/grpc/blob/master/src/core/ext/filters/client_channel/resolver/dns/c_ares/grpc_ares_wrapper.cc#L1187 + * That means it's safe to init it here, but we should be cautious when introducing new code that depends on c-ares and even updates + * to grpc. As discussed in https://github.com/ClickHouse/ClickHouse/pull/37827#discussion_r919189085, c-ares should be adapted to be atomic + * */ + if (ares_library_init(ARES_LIB_INIT_ALL) != ARES_SUCCESS || ares_init(&channel) != ARES_SUCCESS) + { + throw DB::Exception("Failed to initialize c-ares", DB::ErrorCodes::DNS_ERROR); + } + } + + CaresPTRResolver::~CaresPTRResolver() + { + ares_destroy(channel); + ares_library_cleanup(); + } + + std::vector CaresPTRResolver::resolve(const std::string & ip) + { + std::vector ptr_records; + + resolve(ip, ptr_records); + wait(); + + return ptr_records; + } + + std::vector CaresPTRResolver::resolve_v6(const std::string & ip) + { + std::vector ptr_records; + + resolve_v6(ip, ptr_records); + wait(); + + return ptr_records; + } + + void CaresPTRResolver::resolve(const std::string & ip, std::vector & response) + { + in_addr addr; + + inet_pton(AF_INET, ip.c_str(), &addr); + + ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET, callback, &response); + } + + void CaresPTRResolver::resolve_v6(const std::string & ip, std::vector & response) + { + in6_addr addr; + inet_pton(AF_INET6, ip.c_str(), &addr); + + ares_gethostbyaddr(channel, reinterpret_cast(&addr), sizeof(addr), AF_INET6, callback, &response); + } + + void CaresPTRResolver::wait() + { + timeval * tvp, tv; + fd_set read_fds; + fd_set write_fds; + int nfds; + + for (;;) + { + FD_ZERO(&read_fds); + FD_ZERO(&write_fds); + nfds = ares_fds(channel, &read_fds,&write_fds); + if (nfds == 0) + { + break; + } + tvp = ares_timeout(channel, nullptr, &tv); + select(nfds, &read_fds, &write_fds, nullptr, tvp); + ares_process(channel, &read_fds, &write_fds); + } + } +} diff --git a/src/Common/CaresPTRResolver.h b/src/Common/CaresPTRResolver.h new file mode 100644 index 00000000000..fd6a1cf7bc5 --- /dev/null +++ b/src/Common/CaresPTRResolver.h @@ -0,0 +1,42 @@ +#pragma once + +#include "DNSPTRResolver.h" + +using ares_channel = struct ares_channeldata *; + +namespace DB +{ + + /* + * Implements reverse DNS resolution using c-ares lib. System reverse DNS resolution via + * gethostbyaddr or getnameinfo does not work reliably because in some systems + * it returns all PTR records for a given IP and in others it returns only one. + * */ + class CaresPTRResolver : public DNSPTRResolver + { + friend class DNSPTRResolverProvider; + + /* + * Allow only DNSPTRProvider to instantiate this class + * */ + struct provider_token {}; + + public: + explicit CaresPTRResolver(provider_token); + ~CaresPTRResolver() override; + + std::vector resolve(const std::string & ip) override; + + std::vector resolve_v6(const std::string & ip) override; + + private: + void wait(); + + void resolve(const std::string & ip, std::vector & response); + + void resolve_v6(const std::string & ip, std::vector & response); + + ares_channel channel; + }; +} + diff --git a/src/Common/DNSPTRResolver.h b/src/Common/DNSPTRResolver.h new file mode 100644 index 00000000000..e6cce83f79d --- /dev/null +++ b/src/Common/DNSPTRResolver.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include + +namespace DB +{ + struct DNSPTRResolver + { + + virtual ~DNSPTRResolver() = default; + + virtual std::vector resolve(const std::string & ip) = 0; + + virtual std::vector resolve_v6(const std::string & ip) = 0; + + }; +} diff --git a/src/Common/DNSPTRResolverProvider.cpp b/src/Common/DNSPTRResolverProvider.cpp new file mode 100644 index 00000000000..41c73f4f36f --- /dev/null +++ b/src/Common/DNSPTRResolverProvider.cpp @@ -0,0 +1,13 @@ +#include "DNSPTRResolverProvider.h" +#include "CaresPTRResolver.h" + +namespace DB +{ + std::shared_ptr DNSPTRResolverProvider::get() + { + static auto cares_resolver = std::make_shared( + CaresPTRResolver::provider_token {} + ); + return cares_resolver; + } +} diff --git a/src/Common/DNSPTRResolverProvider.h b/src/Common/DNSPTRResolverProvider.h new file mode 100644 index 00000000000..a7f534749e3 --- /dev/null +++ b/src/Common/DNSPTRResolverProvider.h @@ -0,0 +1,18 @@ +#pragma once + +#include +#include "DNSPTRResolver.h" + +namespace DB +{ + /* + * Provides a ready-to-use DNSPTRResolver instance. + * It hides 3rd party lib dependencies, handles initialization and lifetime. + * Since `get` function is static, it can be called from any context. Including cached static functions. + * */ + class DNSPTRResolverProvider + { + public: + static std::shared_ptr get(); + }; +} diff --git a/src/Common/DNSResolver.cpp b/src/Common/DNSResolver.cpp index 0616e324b73..10797b7a809 100644 --- a/src/Common/DNSResolver.cpp +++ b/src/Common/DNSResolver.cpp @@ -12,6 +12,7 @@ #include #include #include +#include "DNSPTRResolverProvider.h" namespace ProfileEvents { @@ -138,16 +139,17 @@ static DNSResolver::IPAddresses resolveIPAddressImpl(const std::string & host) return addresses; } -static String reverseResolveImpl(const Poco::Net::IPAddress & address) +static Strings reverseResolveImpl(const Poco::Net::IPAddress & address) { - Poco::Net::SocketAddress sock_addr(address, 0); + auto ptr_resolver = DB::DNSPTRResolverProvider::get(); - /// 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; + if (address.family() == Poco::Net::IPAddress::Family::IPv4) + { + return ptr_resolver->resolve(address.toString()); + } else + { + return ptr_resolver->resolve_v6(address.toString()); + } } struct DNSResolver::Impl @@ -235,7 +237,7 @@ std::vector DNSResolver::resolveAddressList(const std: return addresses; } -String DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) +Strings DNSResolver::reverseResolve(const Poco::Net::IPAddress & address) { if (impl->disable_cache) return reverseResolveImpl(address); diff --git a/src/Common/DNSResolver.h b/src/Common/DNSResolver.h index fdd9799f96f..84c88586636 100644 --- a/src/Common/DNSResolver.h +++ b/src/Common/DNSResolver.h @@ -36,8 +36,8 @@ public: std::vector resolveAddressList(const std::string & host, UInt16 port); - /// Accepts host IP and resolves its host name - String reverseResolve(const Poco::Net::IPAddress & address); + /// Accepts host IP and resolves its host names + Strings reverseResolve(const Poco::Net::IPAddress & address); /// Get this server host name String getHostName(); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5f1d1a32588..7700fc2dffd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -385,6 +385,7 @@ class ClickHouseCluster: self.with_jdbc_bridge = False self.with_nginx = False self.with_hive = False + self.with_coredns = False self.with_minio = False self.minio_dir = os.path.join(self.instances_dir, "minio") @@ -428,6 +429,8 @@ class ClickHouseCluster: self.schema_registry_port = get_free_port() self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.coredns_host = "coredns" + # available when with_kerberozed_kafka == True self.kerberized_kafka_host = "kerberized_kafka1" self.kerberized_kafka_port = get_free_port() @@ -1102,6 +1105,25 @@ class ClickHouseCluster: ] return self.base_mongo_cmd + def setup_coredns_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_coredns = True + env_variables["COREDNS_CONFIG_DIR"] = instance.path + "/" + "coredns_config" + self.base_cmd.extend( + ["--file", p.join(docker_compose_yml_dir, "docker_compose_coredns.yml")] + ) + + self.base_coredns_cmd = [ + "docker-compose", + "--env-file", + instance.env_file, + "--project-name", + self.project_name, + "--file", + p.join(docker_compose_yml_dir, "docker_compose_coredns.yml"), + ] + + return self.base_coredns_cmd + def setup_meili_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_meili = True env_variables["MEILI_HOST"] = self.meili_host @@ -1265,6 +1287,7 @@ class ClickHouseCluster: with_cassandra=False, with_jdbc_bridge=False, with_hive=False, + with_coredns=False, hostname=None, env_variables=None, image="clickhouse/integration-test", @@ -1349,6 +1372,7 @@ class ClickHouseCluster: with_cassandra=with_cassandra, with_jdbc_bridge=with_jdbc_bridge, with_hive=with_hive, + with_coredns=with_coredns, server_bin_path=self.server_bin_path, odbc_bridge_bin_path=self.odbc_bridge_bin_path, library_bridge_bin_path=self.library_bridge_bin_path, @@ -1513,6 +1537,11 @@ class ClickHouseCluster: ) ) + if with_coredns and not self.with_coredns: + cmds.append( + self.setup_coredns_cmd(instance, env_variables, docker_compose_yml_dir) + ) + if with_meili and not self.with_meili: cmds.append( self.setup_meili_cmd(instance, env_variables, docker_compose_yml_dir) @@ -1629,6 +1658,16 @@ class ClickHouseCluster: "IPAddress" ] + def get_instance_global_ipv6(self, instance_name): + logging.debug("get_instance_ip instance_name={}".format(instance_name)) + docker_id = self.get_instance_docker_id(instance_name) + # for cont in self.docker_client.containers.list(): + # logging.debug("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) + handle = self.docker_client.containers.get(docker_id) + return list(handle.attrs["NetworkSettings"]["Networks"].values())[0][ + "GlobalIPv6Address" + ] + def get_container_id(self, instance_name): return self.get_instance_docker_id(instance_name) # docker_id = self.get_instance_docker_id(instance_name) @@ -2453,6 +2492,12 @@ class ClickHouseCluster: self.up_called = True self.wait_mongo_to_start(30, secure=self.with_mongo_secure) + if self.with_coredns and self.base_coredns_cmd: + logging.debug("Setup coredns") + run_and_check(self.base_coredns_cmd + common_opts) + self.up_called = True + time.sleep(10) + if self.with_meili and self.base_meili_cmd: logging.debug("Setup MeiliSearch") run_and_check(self.base_meili_cmd + common_opts) @@ -2791,6 +2836,7 @@ class ClickHouseInstance: with_azurite, with_jdbc_bridge, with_hive, + with_coredns, with_cassandra, server_bin_path, odbc_bridge_bin_path, @@ -2874,6 +2920,8 @@ class ClickHouseInstance: self.with_cassandra = with_cassandra self.with_jdbc_bridge = with_jdbc_bridge self.with_hive = with_hive + self.with_coredns = with_coredns + self.coredns_config_dir = p.abspath(p.join(base_path, "coredns_config")) self.main_config_name = main_config_name self.users_config_name = users_config_name @@ -3783,6 +3831,11 @@ class ClickHouseInstance: self.kerberos_secrets_dir, p.abspath(p.join(self.path, "secrets")) ) + if self.with_coredns: + shutil.copytree( + self.coredns_config_dir, p.abspath(p.join(self.path, "coredns_config")) + ) + # Copy config.d configs logging.debug( f"Copy custom test config files {self.custom_main_config_paths} to {self.config_d_dir}" diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py b/tests/integration/test_host_regexp_multiple_ptr_records/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml new file mode 100644 index 00000000000..7a2141e6c7e --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/configs/host_regexp.xml @@ -0,0 +1,11 @@ + + + + + + test1\.example\.com$ + + default + + + \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml new file mode 100644 index 00000000000..58ef55cd3f3 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/configs/listen_host.xml @@ -0,0 +1,5 @@ + + :: + 0.0.0.0 + 1 + diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile new file mode 100644 index 00000000000..0dd198441dc --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/Corefile @@ -0,0 +1,8 @@ +. { + hosts /example.com { + reload "200ms" + fallthrough + } + forward . 127.0.0.11 + log +} diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com new file mode 100644 index 00000000000..9beb415c290 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/coredns_config/example.com @@ -0,0 +1 @@ +filled in runtime, but needs to exist in order to be volume mapped in docker \ No newline at end of file diff --git a/tests/integration/test_host_regexp_multiple_ptr_records/test.py b/tests/integration/test_host_regexp_multiple_ptr_records/test.py new file mode 100644 index 00000000000..fa2917411e4 --- /dev/null +++ b/tests/integration/test_host_regexp_multiple_ptr_records/test.py @@ -0,0 +1,91 @@ +import pytest +from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +import os + +DOCKER_COMPOSE_PATH = get_docker_compose_path() +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = ClickHouseCluster(__file__) + +ch_server = cluster.add_instance( + "clickhouse-server", + with_coredns=True, + main_configs=["configs/listen_host.xml"], + user_configs=["configs/host_regexp.xml"], + ipv6_address="2001:3984:3989::1:1111", +) + +client = cluster.add_instance( + "clickhouse-client", + ipv6_address="2001:3984:3989::1:1112", +) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def setup_dns_server(ip): + domains_string = "test3.example.com test2.example.com test1.example.com" + example_file_path = f'{ch_server.env_variables["COREDNS_CONFIG_DIR"]}/example.com' + run_and_check(f"echo '{ip} {domains_string}' > {example_file_path}", shell=True) + + +def setup_ch_server(dns_server_ip): + ch_server.exec_in_container( + (["bash", "-c", f"echo 'nameserver {dns_server_ip}' > /etc/resolv.conf"]) + ) + ch_server.exec_in_container( + (["bash", "-c", "echo 'options ndots:0' >> /etc/resolv.conf"]) + ) + ch_server.query("SYSTEM DROP DNS CACHE") + + +def build_endpoint_v4(ip): + return f"'http://{ip}:8123/?query=SELECT+1&user=test_dns'" + + +def build_endpoint_v6(ip): + return build_endpoint_v4(f"[{ip}]") + + +def test_host_regexp_multiple_ptr_v4_fails_with_wrong_resolution(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + random_ip = "9.9.9.9" + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(random_ip) + setup_ch_server(dns_server_ip) + + endpoint = build_endpoint_v4(server_ip) + + assert "1\n" != client.exec_in_container((["bash", "-c", f"curl {endpoint}"])) + + +def test_host_regexp_multiple_ptr_v4(started_cluster): + server_ip = cluster.get_instance_ip("clickhouse-server") + client_ip = cluster.get_instance_ip("clickhouse-client") + dns_server_ip = cluster.get_instance_ip(cluster.coredns_host) + + setup_dns_server(client_ip) + setup_ch_server(dns_server_ip) + + endpoint = build_endpoint_v4(server_ip) + + assert "1\n" == client.exec_in_container((["bash", "-c", f"curl {endpoint}"])) + + +def test_host_regexp_multiple_ptr_v6(started_cluster): + setup_dns_server(client.ipv6_address) + setup_ch_server(cluster.get_instance_global_ipv6(cluster.coredns_host)) + + endpoint = build_endpoint_v6(ch_server.ipv6_address) + + assert "1\n" == client.exec_in_container((["bash", "-c", f"curl -6 {endpoint}"]))