Add tests

This commit is contained in:
Aleksei Filatov 2023-07-14 12:08:40 +03:00 committed by Aleksei Filatov
parent 1b9bcae68e
commit 801d0955ec
6 changed files with 152 additions and 18 deletions

View File

@ -3522,6 +3522,24 @@ class ClickHouseInstance:
return error
def append_hosts(self, name, ip):
self.exec_in_container(
(["bash", "-c", "echo '{}' {} >> /etc/hosts".format(ip, name)]),
privileged=True,
user="root",
)
def set_hosts(self, hosts):
entries = ["127.0.0.1 localhost", "::1 localhost"]
for host in hosts:
entries.append(f"{host[0]} {host[1]}")
self.exec_in_container(
["bash", "-c", 'echo -e "{}" > /etc/hosts'.format("\\n".join(entries))],
privileged=True,
user="root",
)
# Connects to the instance via HTTP interface, sends a query and returns both the answer and the error message
# as a tuple (output, error).
def http_query_and_get_answer_with_error(

View File

@ -68,5 +68,9 @@
"test_server_reload/test.py::test_remove_postgresql_port",
"test_server_reload/test.py::test_remove_tcp_port",
"test_keeper_map/test.py::test_keeper_map_without_zk"
"test_keeper_map/test.py::test_keeper_map_without_zk",
"test_http_failover/test.py::test_url_destination_host_with_multiple_addrs",
"test_http_failover/test.py::test_url_invalid_hostname",
"test_http_failover/test.py::test_url_ip_change"
]

View File

@ -55,6 +55,13 @@ def cluster_without_dns_cache_update():
# node1 is a source, node2 downloads data
# node2 has long dns_cache_update_period, so dns cache update wouldn't work
def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update):
# In this case we should manually set up the static DNS entries on the source host
# to exclude resplving addresses automatically added by docker.
# We use ipv6 for hosts, but resolved DNS entries may contain an unexpected ipv4 address.
node2.set_hosts([("2001:3984:3989::1:1111", "node1")])
# drop DNS cache
node2.query("SYSTEM DROP DNS CACHE")
# First we check, that normal replication works
node1.query(
"INSERT INTO test_table_drop VALUES ('2018-10-01', 1), ('2018-10-02', 2), ('2018-10-03', 3)"
@ -64,6 +71,7 @@ def test_ip_change_drop_dns_cache(cluster_without_dns_cache_update):
# We change source node ip
cluster.restart_instance_with_ip_change(node1, "2001:3984:3989::1:7777")
node2.set_hosts([("2001:3984:3989::1:7777", "node1")])
# Put some data to source node1
node1.query(
@ -163,17 +171,8 @@ def test_ip_change_update_dns_cache(cluster_with_dns_cache_update):
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):
set_hosts(node4, ["127.255.255.255 lost_host"])
node4.set_hosts([("127.255.255.255", "lost_host")])
with pytest.raises(QueryRuntimeException):
node4.query("SELECT * FROM remote('lost_host', 'system', 'one')")
@ -184,7 +183,7 @@ def test_dns_cache_update(cluster_with_dns_cache_update):
with pytest.raises(QueryRuntimeException):
node4.query("SELECT * FROM distributed_lost_host")
set_hosts(node4, ["127.0.0.1 lost_host"])
node4.set_hosts([("127.0.0.1", "lost_host")])
# Wait a bit until dns cache will be updated
assert_eq_with_retry(
@ -239,11 +238,10 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node):
== "0\n"
)
set_hosts(
node,
node.set_hosts(
[
"127.255.255.255 node3",
"2001:3984:3989::1:88{}4 unknown_host".format(node_num),
("127.255.255.255", "node3"),
(f"2001:3984:3989::1:88{node_num}4", "unknown_host"),
],
)
@ -260,7 +258,7 @@ def test_user_access_ip_change(cluster_with_dns_cache_update, node):
node4.query("SELECT * FROM remote('{}', 'system', 'one')".format(node_name))
# now wrong addresses are cached
set_hosts(node, [])
node.set_hosts([])
retry_count = 60
if node_name == "node5":
# client is not allowed to connect, so execute it directly in container to send query from localhost
@ -298,7 +296,7 @@ def test_host_is_drop_from_cache_after_consecutive_failures(
# Note that the list of hosts in variable since lost_host will be there too (and it's dropped and added back)
# dns_update_short -> dns_max_consecutive_failures set to 6
assert node4.wait_for_log_line(
"Cannot resolve host \\(InvalidHostThatDoesNotExist\\), error 0: Host not found."
"Code: 198. DB::Exception: Not found address of host: InvalidHostThatDoesNotExist."
)
assert node4.wait_for_log_line(
"Cached hosts not found:.*InvalidHostThatDoesNotExist**",

View File

@ -0,0 +1 @@
<clickhouse><listen_host>::</listen_host></clickhouse>

View File

@ -0,0 +1,113 @@
import pytest
from contextlib import nullcontext as does_not_raise
from helpers.cluster import ClickHouseCluster
from helpers.client import QueryRuntimeException
from helpers.test_tools import exec_query_with_retry
from helpers.test_tools import assert_eq_with_retry
ACCESSIBLE_IPV4 = "10.5.172.10"
OTHER_ACCESSIBLE_IPV4 = "10.5.172.20"
NOT_ACCESSIBLE_IPV4 = "10.5.172.11"
ACCESSIBLE_IPV6 = "2001:3984:3989::1:1000"
NOT_ACCESSIBLE_IPV6 = "2001:3984:3989::1:1001"
DST_NODE_IPV4 = ACCESSIBLE_IPV4
DST_NODE_IPV6 = ACCESSIBLE_IPV6
SRC_NODE_IPV6 = "2001:3984:3989::1:2000"
cluster = ClickHouseCluster(__file__)
# Destination node
dst_node = cluster.add_instance(
"dst_node",
with_zookeeper=True,
ipv4_address=DST_NODE_IPV4,
ipv6_address=DST_NODE_IPV6,
main_configs=["configs/listen.xml"],
)
# Source node
src_node = cluster.add_instance(
"src_node",
with_zookeeper=True,
ipv6_address=SRC_NODE_IPV6,
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
except Exception as ex:
print(ex)
finally:
cluster.shutdown()
pass
@pytest.fixture
def dst_node_addrs(started_cluster, request):
src_node.set_hosts([(ip, "dst_node") for ip in request.param])
src_node.query("SYSTEM DROP DNS CACHE")
yield
# Clear static DNS entries
src_node.set_hosts([])
src_node.query("SYSTEM DROP DNS CACHE")
@pytest.mark.parametrize(
"dst_node_addrs, expectation",
[
((ACCESSIBLE_IPV4, ACCESSIBLE_IPV6), does_not_raise()),
((NOT_ACCESSIBLE_IPV4, ACCESSIBLE_IPV6), does_not_raise()),
((ACCESSIBLE_IPV4, NOT_ACCESSIBLE_IPV6), does_not_raise()),
(
(NOT_ACCESSIBLE_IPV4, NOT_ACCESSIBLE_IPV6),
pytest.raises(QueryRuntimeException),
),
],
indirect=["dst_node_addrs"],
)
def test_url_destination_host_with_multiple_addrs(dst_node_addrs, expectation):
with expectation:
result = src_node.query(
"SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')"
)
assert result == "42\n"
def test_url_invalid_hostname(started_cluster):
with pytest.raises(QueryRuntimeException):
src_node.query(
"SELECT count(*) FROM url('http://notvalidhost:8123/?query=SELECT+1', TSV, 'column1 UInt32');"
)
def test_url_ip_change(started_cluster):
assert (
src_node.query(
"SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')"
)
== "42\n"
)
started_cluster.restart_instance_with_ip_change(dst_node, OTHER_ACCESSIBLE_IPV4)
# Ensure that only new IPV4 address is accessible
src_node.set_hosts(
[(OTHER_ACCESSIBLE_IPV4, "dst_node"), (NOT_ACCESSIBLE_IPV6, "dst_node")]
)
src_node.query("SYSTEM DROP DNS CACHE")
assert (
src_node.query(
"SELECT * FROM url('http://dst_node:8123/?query=SELECT+42', TSV, 'column1 UInt32')"
)
== "42\n"
)