mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Merge pull request #37827 from arthurpassos/host_regexp_multiple_domains
Test host_regexp against all PTR records instead of only one
This commit is contained in:
commit
9e9969cea7
3
.gitmodules
vendored
3
.gitmodules
vendored
@ -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
|
||||
|
1
contrib/CMakeLists.txt
vendored
1
contrib/CMakeLists.txt
vendored
@ -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/<immediate-subdir>" 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
|
||||
|
1
contrib/c-ares
vendored
Submodule
1
contrib/c-ares
vendored
Submodule
@ -0,0 +1 @@
|
||||
Subproject commit afee6748b0b99acf4509d42fa37ac8422262f91b
|
35
contrib/c-ares-cmake/CMakeLists.txt
Normal file
35
contrib/c-ares-cmake/CMakeLists.txt
Normal file
@ -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)
|
@ -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,
|
||||
|
@ -135,6 +135,7 @@ function clone_submodules
|
||||
contrib/replxx
|
||||
contrib/wyhash
|
||||
contrib/hashidsxx
|
||||
contrib/c-ares
|
||||
)
|
||||
|
||||
git submodule sync
|
||||
|
@ -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
|
@ -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<String> resolved_host;
|
||||
std::optional<Strings> 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)
|
||||
{
|
||||
|
@ -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)
|
||||
|
109
src/Common/CaresPTRResolver.cpp
Normal file
109
src/Common/CaresPTRResolver.cpp
Normal file
@ -0,0 +1,109 @@
|
||||
#include "CaresPTRResolver.h"
|
||||
#include <arpa/inet.h>
|
||||
#include <sys/select.h>
|
||||
#include <Common/Exception.h>
|
||||
#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<std::vector<std::string>*>(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<std::string> CaresPTRResolver::resolve(const std::string & ip)
|
||||
{
|
||||
std::vector<std::string> ptr_records;
|
||||
|
||||
resolve(ip, ptr_records);
|
||||
wait();
|
||||
|
||||
return ptr_records;
|
||||
}
|
||||
|
||||
std::vector<std::string> CaresPTRResolver::resolve_v6(const std::string & ip)
|
||||
{
|
||||
std::vector<std::string> ptr_records;
|
||||
|
||||
resolve_v6(ip, ptr_records);
|
||||
wait();
|
||||
|
||||
return ptr_records;
|
||||
}
|
||||
|
||||
void CaresPTRResolver::resolve(const std::string & ip, std::vector<std::string> & response)
|
||||
{
|
||||
in_addr addr;
|
||||
|
||||
inet_pton(AF_INET, ip.c_str(), &addr);
|
||||
|
||||
ares_gethostbyaddr(channel, reinterpret_cast<const void*>(&addr), sizeof(addr), AF_INET, callback, &response);
|
||||
}
|
||||
|
||||
void CaresPTRResolver::resolve_v6(const std::string & ip, std::vector<std::string> & response)
|
||||
{
|
||||
in6_addr addr;
|
||||
inet_pton(AF_INET6, ip.c_str(), &addr);
|
||||
|
||||
ares_gethostbyaddr(channel, reinterpret_cast<const void*>(&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);
|
||||
}
|
||||
}
|
||||
}
|
42
src/Common/CaresPTRResolver.h
Normal file
42
src/Common/CaresPTRResolver.h
Normal file
@ -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<std::string> resolve(const std::string & ip) override;
|
||||
|
||||
std::vector<std::string> resolve_v6(const std::string & ip) override;
|
||||
|
||||
private:
|
||||
void wait();
|
||||
|
||||
void resolve(const std::string & ip, std::vector<std::string> & response);
|
||||
|
||||
void resolve_v6(const std::string & ip, std::vector<std::string> & response);
|
||||
|
||||
ares_channel channel;
|
||||
};
|
||||
}
|
||||
|
18
src/Common/DNSPTRResolver.h
Normal file
18
src/Common/DNSPTRResolver.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
#include <vector>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
struct DNSPTRResolver
|
||||
{
|
||||
|
||||
virtual ~DNSPTRResolver() = default;
|
||||
|
||||
virtual std::vector<std::string> resolve(const std::string & ip) = 0;
|
||||
|
||||
virtual std::vector<std::string> resolve_v6(const std::string & ip) = 0;
|
||||
|
||||
};
|
||||
}
|
13
src/Common/DNSPTRResolverProvider.cpp
Normal file
13
src/Common/DNSPTRResolverProvider.cpp
Normal file
@ -0,0 +1,13 @@
|
||||
#include "DNSPTRResolverProvider.h"
|
||||
#include "CaresPTRResolver.h"
|
||||
|
||||
namespace DB
|
||||
{
|
||||
std::shared_ptr<DNSPTRResolver> DNSPTRResolverProvider::get()
|
||||
{
|
||||
static auto cares_resolver = std::make_shared<CaresPTRResolver>(
|
||||
CaresPTRResolver::provider_token {}
|
||||
);
|
||||
return cares_resolver;
|
||||
}
|
||||
}
|
18
src/Common/DNSPTRResolverProvider.h
Normal file
18
src/Common/DNSPTRResolverProvider.h
Normal file
@ -0,0 +1,18 @@
|
||||
#pragma once
|
||||
|
||||
#include <memory>
|
||||
#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<DNSPTRResolver> get();
|
||||
};
|
||||
}
|
@ -12,6 +12,7 @@
|
||||
#include <atomic>
|
||||
#include <optional>
|
||||
#include <string_view>
|
||||
#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<Poco::Net::SocketAddress> 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);
|
||||
|
@ -36,8 +36,8 @@ public:
|
||||
|
||||
std::vector<Poco::Net::SocketAddress> 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();
|
||||
|
@ -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}"
|
||||
|
@ -0,0 +1,11 @@
|
||||
<yandex>
|
||||
<users>
|
||||
<test_dns>
|
||||
<password/>
|
||||
<networks>
|
||||
<host_regexp>test1\.example\.com$</host_regexp>
|
||||
</networks>
|
||||
<profile>default</profile>
|
||||
</test_dns>
|
||||
</users>
|
||||
</yandex>
|
@ -0,0 +1,5 @@
|
||||
<yandex>
|
||||
<listen_host>::</listen_host>
|
||||
<listen_host>0.0.0.0</listen_host>
|
||||
<listen_try>1</listen_try>
|
||||
</yandex>
|
@ -0,0 +1,8 @@
|
||||
. {
|
||||
hosts /example.com {
|
||||
reload "200ms"
|
||||
fallthrough
|
||||
}
|
||||
forward . 127.0.0.11
|
||||
log
|
||||
}
|
@ -0,0 +1 @@
|
||||
filled in runtime, but needs to exist in order to be volume mapped in docker
|
@ -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}"]))
|
Loading…
Reference in New Issue
Block a user