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:
Alexander Tokmakov 2022-07-21 16:43:05 +03:00 committed by GitHub
commit 9e9969cea7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 467 additions and 55 deletions

3
.gitmodules vendored
View File

@ -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

View File

@ -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

@ -0,0 +1 @@
Subproject commit afee6748b0b99acf4509d42fa37ac8422262f91b

View 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)

View File

@ -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,

View File

@ -135,6 +135,7 @@ function clone_submodules
contrib/replxx
contrib/wyhash
contrib/hashidsxx
contrib/c-ares
)
git submodule sync

View File

@ -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

View File

@ -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)
{

View File

@ -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)

View 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);
}
}
}

View 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;
};
}

View 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;
};
}

View 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;
}
}

View 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();
};
}

View File

@ -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);

View File

@ -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();

View File

@ -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}"

View File

@ -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>

View File

@ -0,0 +1,5 @@
<yandex>
<listen_host>::</listen_host>
<listen_host>0.0.0.0</listen_host>
<listen_try>1</listen_try>
</yandex>

View File

@ -0,0 +1,8 @@
. {
hosts /example.com {
reload "200ms"
fallthrough
}
forward . 127.0.0.11
log
}

View File

@ -0,0 +1 @@
filled in runtime, but needs to exist in order to be volume mapped in docker

View File

@ -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}"]))