tests: make test_config_substitutions idempotent

Now the following passed:

    $ bin=$(which clickhouse); g=$PWD; docker run --privileged -it --rm --volume=$bin:/clickhouse --volume=$bin:/usr/share/clickhouse_fresh --volume=/dev/null:/usr/bin/clickhouse-odbc-bridge --volume=/dev/null:/usr/share/clickhouse-odbc-bridge_fresh --volume=$g/programs/server:/clickhouse-config --volume=$g:/ClickHouse --volume=$g/docker/test/integration/runner/compose:/compose:ro --volume=clickhouse-integration-tests:/var/lib/docker -e PYTEST_ADDOPTS='-vvv -s --pdb --count 2 test_config_substitutions/test.py' --name ch clickhouse/integration-tests-runner

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
This commit is contained in:
Azat Khuzhin 2024-12-01 17:06:43 +01:00
parent f1e909e309
commit 47397a45c5
2 changed files with 66 additions and 42 deletions

View File

@ -17,6 +17,7 @@ import subprocess
import time import time
import traceback import traceback
import urllib.parse import urllib.parse
from contextlib import contextmanager
from functools import cache from functools import cache
from pathlib import Path from pathlib import Path
from typing import Any, List, Sequence, Tuple, Union from typing import Any, List, Sequence, Tuple, Union
@ -4542,6 +4543,20 @@ class ClickHouseInstance:
if key != "DSN": if key != "DSN":
f.write(key + "=" + value + "\n") f.write(key + "=" + value + "\n")
@contextmanager
def with_replace_config(self, path, replacement):
"""Create a copy of existing config (if exists) and revert on leaving the context"""
self.exec_in_container(
["bash", "-c", f"test ! -f {path} || mv --no-clobber {path} {path}.bak"]
)
self.exec_in_container(
["bash", "-c", "echo '{}' > {}".format(replacement, path)]
)
yield
self.exec_in_container(
["bash", "-c", f"test ! -f {path}.bak || mv {path}.bak {path}"]
)
def replace_config(self, path_to_config, replacement): def replace_config(self, path_to_config, replacement):
self.exec_in_container( self.exec_in_container(
["bash", "-c", "echo '{}' > {}".format(replacement, path_to_config)] ["bash", "-c", "echo '{}' > {}".format(replacement, path_to_config)]

View File

@ -21,6 +21,7 @@ node3 = cluster.add_instance(
"configs/config_zk_include_test.xml", "configs/config_zk_include_test.xml",
], ],
with_zookeeper=True, with_zookeeper=True,
stay_alive=True,
) )
node4 = cluster.add_instance( node4 = cluster.add_instance(
"node4", "node4",
@ -133,7 +134,7 @@ def test_config(start_cluster):
def test_config_from_env_overrides(start_cluster): def test_config_from_env_overrides(start_cluster):
node7.replace_config( with node7.with_replace_config(
"/etc/clickhouse-server/users.d/000-users_with_env_subst.xml", "/etc/clickhouse-server/users.d/000-users_with_env_subst.xml",
""" """
<clickhouse> <clickhouse>
@ -155,13 +156,14 @@ def test_config_from_env_overrides(start_cluster):
</users> </users>
</clickhouse> </clickhouse>
""", """,
) ):
with pytest.raises( with pytest.raises(
QueryRuntimeException, QueryRuntimeException,
match="Failed to preprocess config '/etc/clickhouse-server/users.xml': Exception: Element <max_threads> has value and does not have 'replace' attribute, can't process from_env substitution", match="Failed to preprocess config '/etc/clickhouse-server/users.xml': Exception: Element <max_threads> has value and does not have 'replace' attribute, can't process from_env substitution",
): ):
node7.query("SYSTEM RELOAD CONFIG") node7.query("SYSTEM RELOAD CONFIG")
node7.replace_config(
with node7.with_replace_config(
"/etc/clickhouse-server/users.d/000-users_with_env_subst.xml", "/etc/clickhouse-server/users.d/000-users_with_env_subst.xml",
""" """
<clickhouse> <clickhouse>
@ -183,25 +185,26 @@ def test_config_from_env_overrides(start_cluster):
</users> </users>
</clickhouse> </clickhouse>
""", """,
) ):
node7.query("SYSTEM RELOAD CONFIG") node7.query("SYSTEM RELOAD CONFIG")
def test_config_merge_from_env_overrides(start_cluster): def test_config_merge_from_env_overrides(start_cluster):
node7.query("SYSTEM RELOAD CONFIG")
assert ( assert (
node7.query( node7.query(
"SELECT value FROM system.server_settings WHERE name='max_thread_pool_size'" "SELECT value FROM system.server_settings WHERE name='max_thread_pool_size'"
) )
== "10000\n" == "10000\n"
) )
node7.replace_config( with node7.with_replace_config(
"/etc/clickhouse-server/config.d/010-server_with_env_subst.xml", "/etc/clickhouse-server/config.d/010-server_with_env_subst.xml",
""" """
<clickhouse> <clickhouse>
<max_thread_pool_size from_env="CH_THREADS" replace="1">9000</max_thread_pool_size> <max_thread_pool_size from_env="CH_THREADS" replace="1">9000</max_thread_pool_size>
</clickhouse> </clickhouse>
""", """,
) ):
node7.query("SYSTEM RELOAD CONFIG") node7.query("SYSTEM RELOAD CONFIG")
@ -223,6 +226,7 @@ def test_include_config(start_cluster):
def test_allow_databases(start_cluster): def test_allow_databases(start_cluster):
node5.query("DROP DATABASE IF EXISTS db1 SYNC")
node5.query("CREATE DATABASE db1") node5.query("CREATE DATABASE db1")
node5.query( node5.query(
"CREATE TABLE db1.test_table(date Date, k1 String, v1 Int32) ENGINE = MergeTree(date, (k1, date), 8192)" "CREATE TABLE db1.test_table(date Date, k1 String, v1 Int32) ENGINE = MergeTree(date, (k1, date), 8192)"
@ -293,6 +297,9 @@ def test_allow_databases(start_cluster):
def test_config_multiple_zk_substitutions(start_cluster): def test_config_multiple_zk_substitutions(start_cluster):
# NOTE: background_pool_size cannot be decreased, so let's restart ClickHouse to make the test idempotent (i.e. runned multiple times)
node3.restart_clickhouse()
node3.query("SYSTEM RELOAD CONFIG")
assert ( assert (
node3.query( node3.query(
"SELECT value FROM system.merge_tree_settings WHERE name='min_bytes_for_wide_part'" "SELECT value FROM system.merge_tree_settings WHERE name='min_bytes_for_wide_part'"
@ -319,13 +326,14 @@ def test_config_multiple_zk_substitutions(start_cluster):
) )
zk = cluster.get_kazoo_client("zoo1") zk = cluster.get_kazoo_client("zoo1")
try:
zk.create( zk.create(
path="/background_pool_size", path="/background_pool_size",
value=b"<background_pool_size>72</background_pool_size>", value=b"<background_pool_size>72</background_pool_size>",
makepath=True, makepath=True,
) )
node3.replace_config( with node3.with_replace_config(
"/etc/clickhouse-server/config.d/config_zk_include_test.xml", "/etc/clickhouse-server/config.d/config_zk_include_test.xml",
""" """
<clickhouse> <clickhouse>
@ -340,8 +348,7 @@ def test_config_multiple_zk_substitutions(start_cluster):
<include from_zk="/min_bytes_for_wide_part" merge="true"/> <include from_zk="/min_bytes_for_wide_part" merge="true"/>
</clickhouse> </clickhouse>
""", """,
) ):
node3.query("SYSTEM RELOAD CONFIG") node3.query("SYSTEM RELOAD CONFIG")
assert ( assert (
@ -350,3 +357,5 @@ def test_config_multiple_zk_substitutions(start_cluster):
) )
== "72\n" == "72\n"
) )
finally:
zk.delete(path="/background_pool_size")