rename setting

This commit is contained in:
Arthur Passos 2023-10-02 18:21:20 -03:00
parent 3813e59b49
commit 0b97cd49e8
2 changed files with 304 additions and 304 deletions

View File

@ -83,14 +83,14 @@ def _create_env_file(path, variables):
def run_and_check(
args,
env=None,
shell=False,
stdout=subprocess.PIPE,
stderr=subprocess.PIPE,
timeout=300,
nothrow=False,
detach=False,
args,
env=None,
shell=False,
stdout=subprocess.PIPE,
stderr=subprocess.PIPE,
timeout=300,
nothrow=False,
detach=False,
):
if detach:
subprocess.Popen(
@ -325,19 +325,19 @@ class ClickHouseCluster:
"""
def __init__(
self,
base_path,
name=None,
base_config_dir=None,
server_bin_path=None,
client_bin_path=None,
odbc_bridge_bin_path=None,
library_bridge_bin_path=None,
zookeeper_config_path=None,
custom_dockerd_host=None,
zookeeper_keyfile=None,
zookeeper_certfile=None,
with_spark=False,
self,
base_path,
name=None,
base_config_dir=None,
server_bin_path=None,
client_bin_path=None,
odbc_bridge_bin_path=None,
library_bridge_bin_path=None,
zookeeper_config_path=None,
custom_dockerd_host=None,
zookeeper_keyfile=None,
zookeeper_certfile=None,
with_spark=False,
):
for param in list(os.environ.keys()):
logging.debug("ENV %40s %s" % (param, os.environ[param]))
@ -371,7 +371,7 @@ class ClickHouseCluster:
)
project_name = (
pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name
pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name
)
# docker-compose removes everything non-alphanumeric from project names so we do it too.
self.project_name = re.sub(r"[^a-z0-9]", "", project_name.lower())
@ -739,9 +739,9 @@ class ClickHouseCluster:
self.print_all_docker_pieces()
if (
os.environ
and "DISABLE_CLEANUP" in os.environ
and os.environ["DISABLE_CLEANUP"] == "1"
os.environ
and "DISABLE_CLEANUP" in os.environ
and os.environ["DISABLE_CLEANUP"] == "1"
):
logging.warning("Cleanup is disabled")
return
@ -837,7 +837,7 @@ class ClickHouseCluster:
return containers
def copy_file_from_container_to_container(
self, src_node, src_path, dst_node, dst_path
self, src_node, src_path, dst_node, dst_path
):
fname = os.path.basename(src_path)
run_and_check(
@ -850,7 +850,7 @@ class ClickHouseCluster:
)
def setup_zookeeper_secure_cmd(
self, instance, env_variables, docker_compose_yml_dir
self, instance, env_variables, docker_compose_yml_dir
):
logging.debug("Setup ZooKeeper Secure")
zookeeper_docker_compose_path = p.join(
@ -926,7 +926,7 @@ class ClickHouseCluster:
# always prefer clickhouse-keeper standalone binary
if os.path.exists(
os.path.join(binary_dir, "clickhouse-keeper")
os.path.join(binary_dir, "clickhouse-keeper")
) and not os.path.islink(os.path.join(binary_dir, "clickhouse-keeper")):
binary_path = os.path.join(binary_dir, "clickhouse-keeper")
keeper_cmd_prefix = "clickhouse-keeper"
@ -1073,7 +1073,7 @@ class ClickHouseCluster:
return self.base_postgres_cmd
def setup_postgres_cluster_cmd(
self, instance, env_variables, docker_compose_yml_dir
self, instance, env_variables, docker_compose_yml_dir
):
self.with_postgres_cluster = True
env_variables["POSTGRES_PORT"] = str(self.postgres_port)
@ -1098,7 +1098,7 @@ class ClickHouseCluster:
]
def setup_postgresql_java_client_cmd(
self, instance, env_variables, docker_compose_yml_dir
self, instance, env_variables, docker_compose_yml_dir
):
self.with_postgresql_java_client = True
self.base_cmd.extend(
@ -1142,7 +1142,7 @@ class ClickHouseCluster:
return self.base_hdfs_cmd
def setup_kerberized_hdfs_cmd(
self, instance, env_variables, docker_compose_yml_dir
self, instance, env_variables, docker_compose_yml_dir
):
self.with_kerberized_hdfs = True
env_variables["KERBERIZED_HDFS_HOST"] = self.hdfs_kerberized_host
@ -1192,7 +1192,7 @@ class ClickHouseCluster:
return self.base_kafka_cmd
def setup_kerberized_kafka_cmd(
self, instance, env_variables, docker_compose_yml_dir
self, instance, env_variables, docker_compose_yml_dir
):
self.with_kerberized_kafka = True
env_variables["KERBERIZED_KAFKA_DIR"] = instance.path + "/"
@ -1494,66 +1494,66 @@ class ClickHouseCluster:
return self.base_hive_cmd
def add_instance(
self,
name,
base_config_dir=None,
main_configs=None,
user_configs=None,
dictionaries=None,
macros=None,
with_zookeeper=False,
with_zookeeper_secure=False,
with_mysql_client=False,
with_mysql=False,
with_mysql8=False,
with_mysql_cluster=False,
with_kafka=False,
with_kerberized_kafka=False,
with_kerberos_kdc=False,
with_secrets=False,
with_rabbitmq=False,
with_nats=False,
clickhouse_path_dir=None,
with_odbc_drivers=False,
with_postgres=False,
with_postgres_cluster=False,
with_postgresql_java_client=False,
clickhouse_log_file=CLICKHOUSE_LOG_FILE,
clickhouse_error_log_file=CLICKHOUSE_ERROR_LOG_FILE,
with_hdfs=False,
with_kerberized_hdfs=False,
with_mongo=False,
with_mongo_secure=False,
with_meili=False,
with_nginx=False,
with_redis=False,
with_minio=False,
with_azurite=False,
with_cassandra=False,
with_jdbc_bridge=False,
with_hive=False,
with_coredns=False,
allow_analyzer=True,
hostname=None,
env_variables=None,
exclusive_env_variables=False,
image="clickhouse/integration-test",
tag=None,
stay_alive=False,
ipv4_address=None,
ipv6_address=None,
with_installed_binary=False,
external_dirs=None,
tmpfs=None,
zookeeper_docker_compose_path=None,
minio_certs_dir=None,
minio_data_dir=None,
use_keeper=True,
main_config_name="config.xml",
users_config_name="users.xml",
copy_common_configs=True,
config_root_name="clickhouse",
extra_configs=[],
self,
name,
base_config_dir=None,
main_configs=None,
user_configs=None,
dictionaries=None,
macros=None,
with_zookeeper=False,
with_zookeeper_secure=False,
with_mysql_client=False,
with_mysql=False,
with_mysql8=False,
with_mysql_cluster=False,
with_kafka=False,
with_kerberized_kafka=False,
with_kerberos_kdc=False,
with_secrets=False,
with_rabbitmq=False,
with_nats=False,
clickhouse_path_dir=None,
with_odbc_drivers=False,
with_postgres=False,
with_postgres_cluster=False,
with_postgresql_java_client=False,
clickhouse_log_file=CLICKHOUSE_LOG_FILE,
clickhouse_error_log_file=CLICKHOUSE_ERROR_LOG_FILE,
with_hdfs=False,
with_kerberized_hdfs=False,
with_mongo=False,
with_mongo_secure=False,
with_meili=False,
with_nginx=False,
with_redis=False,
with_minio=False,
with_azurite=False,
with_cassandra=False,
with_jdbc_bridge=False,
with_hive=False,
with_coredns=False,
allow_analyzer=True,
hostname=None,
env_variables=None,
instance_env_variables=False,
image="clickhouse/integration-test",
tag=None,
stay_alive=False,
ipv4_address=None,
ipv6_address=None,
with_installed_binary=False,
external_dirs=None,
tmpfs=None,
zookeeper_docker_compose_path=None,
minio_certs_dir=None,
minio_data_dir=None,
use_keeper=True,
main_config_name="config.xml",
users_config_name="users.xml",
copy_common_configs=True,
config_root_name="clickhouse",
extra_configs=[],
) -> "ClickHouseInstance":
"""Add an instance to the cluster.
@ -1619,9 +1619,9 @@ class ClickHouseCluster:
with_nginx=with_nginx,
with_kerberized_hdfs=with_kerberized_hdfs,
with_secrets=with_secrets
or with_kerberized_hdfs
or with_kerberos_kdc
or with_kerberized_kafka,
or with_kerberized_hdfs
or with_kerberos_kdc
or with_kerberized_kafka,
with_mongo=with_mongo or with_mongo_secure,
with_meili=with_meili,
with_redis=with_redis,
@ -1646,7 +1646,7 @@ class ClickHouseCluster:
copy_common_configs=copy_common_configs,
hostname=hostname,
env_variables=env_variables,
exclusive_env_variables=exclusive_env_variables,
instance_env_variables=instance_env_variables,
image=image,
tag=tag,
stay_alive=stay_alive,
@ -1796,7 +1796,7 @@ class ClickHouseCluster:
)
if (with_mongo or with_mongo_secure) and not (
self.with_mongo or self.with_mongo_secure
self.with_mongo or self.with_mongo_secure
):
if with_mongo_secure:
cmds.append(
@ -1959,7 +1959,7 @@ class ClickHouseCluster:
return self.docker_client.api.logs(container_id).decode()
def exec_in_container(
self, container_id, cmd, detach=False, nothrow=False, use_cli=True, **kwargs
self, container_id, cmd, detach=False, nothrow=False, use_cli=True, **kwargs
):
if use_cli:
logging.debug(
@ -2017,7 +2017,7 @@ class ClickHouseCluster:
)
def wait_for_url(
self, url="http://localhost:8123/ping", conn_timeout=2, interval=2, timeout=60
self, url="http://localhost:8123/ping", conn_timeout=2, interval=2, timeout=60
):
if not url.startswith("http"):
url = "http://" + url
@ -2233,7 +2233,7 @@ class ClickHouseCluster:
while time.time() - start < timeout:
try:
if check_postgresql_java_client_is_available(
self.postgresql_java_client_docker_id
self.postgresql_java_client_docker_id
):
logging.debug("PostgreSQL Java Client is available")
return True
@ -2267,7 +2267,7 @@ class ClickHouseCluster:
retries = 0
while True:
if asyncio.run(
check_nats_is_available(self.nats_port, ssl_ctx=self.nats_ssl_context)
check_nats_is_available(self.nats_port, ssl_ctx=self.nats_ssl_context)
):
break
else:
@ -2722,8 +2722,8 @@ class ClickHouseCluster:
self.wait_postgres_cluster_to_start()
if (
self.with_postgresql_java_client
and self.base_postgresql_java_client_cmd
self.with_postgresql_java_client
and self.base_postgresql_java_client_cmd
):
logging.debug("Setup Postgres Java Client")
subprocess_check_call(
@ -2973,7 +2973,7 @@ class ClickHouseCluster:
# NOTE: we cannot do this via docker since in case of Fatal message container may already die.
for name, instance in self.instances.items():
if instance.contains_in_log(
SANITIZER_SIGN, from_host=True, filename="stderr.log"
SANITIZER_SIGN, from_host=True, filename="stderr.log"
):
sanitizer_assert_instance = instance.grep_in_log(
SANITIZER_SIGN,
@ -2988,7 +2988,7 @@ class ClickHouseCluster:
)
if not ignore_fatal and instance.contains_in_log(
"Fatal", from_host=True
"Fatal", from_host=True
):
fatal_log = instance.grep_in_log("Fatal", from_host=True)
if "Child process was terminated by signal 9 (KILL)" in fatal_log:
@ -3061,7 +3061,7 @@ class ClickHouseCluster:
return zk
def run_kazoo_commands_with_retries(
self, kazoo_callback, zoo_instance_name="zoo1", repeats=1, sleep_for=1
self, kazoo_callback, zoo_instance_name="zoo1", repeats=1, sleep_for=1
):
zk = self.get_kazoo_client(zoo_instance_name)
logging.debug(
@ -3140,64 +3140,64 @@ services:
class ClickHouseInstance:
def __init__(
self,
cluster,
base_path,
name,
base_config_dir,
custom_main_configs,
custom_user_configs,
custom_dictionaries,
macros,
with_zookeeper,
zookeeper_config_path,
with_mysql_client,
with_mysql,
with_mysql8,
with_mysql_cluster,
with_kafka,
with_kerberized_kafka,
with_kerberos_kdc,
with_rabbitmq,
with_nats,
with_nginx,
with_kerberized_hdfs,
with_secrets,
with_mongo,
with_meili,
with_redis,
with_minio,
with_azurite,
with_jdbc_bridge,
with_hive,
with_coredns,
with_cassandra,
allow_analyzer,
server_bin_path,
odbc_bridge_bin_path,
library_bridge_bin_path,
clickhouse_path_dir,
with_odbc_drivers,
with_postgres,
with_postgres_cluster,
with_postgresql_java_client,
clickhouse_start_command=CLICKHOUSE_START_COMMAND,
main_config_name="config.xml",
users_config_name="users.xml",
copy_common_configs=True,
hostname=None,
env_variables=None,
exclusive_env_variables=False,
image="clickhouse/integration-test",
tag="latest",
stay_alive=False,
ipv4_address=None,
ipv6_address=None,
with_installed_binary=False,
external_dirs=None,
tmpfs=None,
config_root_name="clickhouse",
extra_configs=[],
self,
cluster,
base_path,
name,
base_config_dir,
custom_main_configs,
custom_user_configs,
custom_dictionaries,
macros,
with_zookeeper,
zookeeper_config_path,
with_mysql_client,
with_mysql,
with_mysql8,
with_mysql_cluster,
with_kafka,
with_kerberized_kafka,
with_kerberos_kdc,
with_rabbitmq,
with_nats,
with_nginx,
with_kerberized_hdfs,
with_secrets,
with_mongo,
with_meili,
with_redis,
with_minio,
with_azurite,
with_jdbc_bridge,
with_hive,
with_coredns,
with_cassandra,
allow_analyzer,
server_bin_path,
odbc_bridge_bin_path,
library_bridge_bin_path,
clickhouse_path_dir,
with_odbc_drivers,
with_postgres,
with_postgres_cluster,
with_postgresql_java_client,
clickhouse_start_command=CLICKHOUSE_START_COMMAND,
main_config_name="config.xml",
users_config_name="users.xml",
copy_common_configs=True,
hostname=None,
env_variables=None,
instance_env_variables=False,
image="clickhouse/integration-test",
tag="latest",
stay_alive=False,
ipv4_address=None,
ipv6_address=None,
with_installed_binary=False,
external_dirs=None,
tmpfs=None,
config_root_name="clickhouse",
extra_configs=[],
):
self.name = name
self.base_cmd = cluster.base_cmd
@ -3277,7 +3277,7 @@ class ClickHouseInstance:
self.path = p.join(self.cluster.instances_dir, name)
self.docker_compose_path = p.join(self.path, "docker-compose.yml")
self.env_variables = env_variables or {}
self.exclusive_env_variables = exclusive_env_variables
self.instance_env_variables = instance_env_variables
self.env_file = self.cluster.env_file
if with_odbc_drivers:
self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini"
@ -3292,7 +3292,7 @@ class ClickHouseInstance:
base_secrets_dir = os.path.dirname(self.docker_compose_path)
self.keytab_path = "- " + base_secrets_dir + "/secrets:/tmp/keytab"
self.krb5_conf = (
"- " + base_secrets_dir + "/secrets/krb.conf:/etc/krb5.conf:ro"
"- " + base_secrets_dir + "/secrets/krb.conf:/etc/krb5.conf:ro"
)
else:
self.keytab_path = ""
@ -3333,17 +3333,17 @@ class ClickHouseInstance:
# Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer
def query(
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
host=None,
ignore_error=False,
query_id=None,
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
host=None,
ignore_error=False,
query_id=None,
):
sql_for_log = ""
if len(sql) > 1000:
@ -3365,19 +3365,19 @@ class ClickHouseInstance:
)
def query_with_retry(
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
host=None,
ignore_error=False,
retry_count=20,
sleep_time=0.5,
check_callback=lambda x: True,
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
host=None,
ignore_error=False,
retry_count=20,
sleep_time=0.5,
check_callback=lambda x: True,
):
# logging.debug(f"Executing query {sql} on {self.name}")
result = None
@ -3412,15 +3412,15 @@ class ClickHouseInstance:
# Connects to the instance via clickhouse-client, sends a query (1st argument), expects an error and return its code
def query_and_get_error(
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
query_id=None,
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
query_id=None,
):
logging.debug(f"Executing query {sql} on {self.name}")
return self.client.query_and_get_error(
@ -3435,16 +3435,16 @@ class ClickHouseInstance:
)
def query_and_get_error_with_retry(
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
retry_count=20,
sleep_time=0.5,
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
retry_count=20,
sleep_time=0.5,
):
logging.debug(f"Executing query {sql} on {self.name}")
result = None
@ -3471,15 +3471,15 @@ class ClickHouseInstance:
# The same as query_and_get_error but ignores successful query.
def query_and_get_answer_with_error(
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
query_id=None,
self,
sql,
stdin=None,
timeout=None,
settings=None,
user=None,
password=None,
database=None,
query_id=None,
):
logging.debug(f"Executing query {sql} on {self.name}")
return self.client.query_and_get_answer_with_error(
@ -3495,17 +3495,17 @@ class ClickHouseInstance:
# Connects to the instance via HTTP interface, sends a query and returns the answer
def http_query(
self,
sql,
data=None,
method=None,
params=None,
user=None,
password=None,
port=8123,
timeout=None,
retry_strategy=None,
content=False,
self,
sql,
data=None,
method=None,
params=None,
user=None,
password=None,
port=8123,
timeout=None,
retry_strategy=None,
content=False,
):
output, error = self.http_query_and_get_answer_with_error(
sql,
@ -3527,16 +3527,16 @@ class ClickHouseInstance:
# Connects to the instance via HTTP interface, sends a query, expects an error and return the error message
def http_query_and_get_error(
self,
sql,
data=None,
method=None,
params=None,
user=None,
password=None,
port=8123,
timeout=None,
retry_strategy=None,
self,
sql,
data=None,
method=None,
params=None,
user=None,
password=None,
port=8123,
timeout=None,
retry_strategy=None,
):
output, error = self.http_query_and_get_answer_with_error(
sql,
@ -3578,17 +3578,17 @@ class ClickHouseInstance:
# 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(
self,
sql,
data=None,
method=None,
params=None,
user=None,
password=None,
port=8123,
timeout=None,
retry_strategy=None,
content=False,
self,
sql,
data=None,
method=None,
params=None,
user=None,
password=None,
port=8123,
timeout=None,
retry_strategy=None,
content=False,
):
logging.debug(f"Executing query {sql} on {self.name} via HTTP interface")
if params is None:
@ -3776,7 +3776,7 @@ class ClickHouseInstance:
)
def contains_in_log(
self, substring, from_host=False, filename="clickhouse-server.log"
self, substring, from_host=False, filename="clickhouse-server.log"
):
if from_host:
# We check fist file exists but want to look for all rotated logs as well
@ -3798,7 +3798,7 @@ class ClickHouseInstance:
return len(result) > 0
def grep_in_log(
self, substring, from_host=False, filename="clickhouse-server.log", after=None
self, substring, from_host=False, filename="clickhouse-server.log", after=None
):
logging.debug(f"grep in log called %s", substring)
if after is not None:
@ -3838,12 +3838,12 @@ class ClickHouseInstance:
return result
def wait_for_log_line(
self,
regexp,
filename="/var/log/clickhouse-server/clickhouse-server.log",
timeout=30,
repetitions=1,
look_behind_lines=100,
self,
regexp,
filename="/var/log/clickhouse-server/clickhouse-server.log",
timeout=30,
repetitions=1,
look_behind_lines=100,
):
start_time = time.time()
result = self.exec_in_container(
@ -3885,16 +3885,16 @@ class ClickHouseInstance:
def path_exists(self, path):
return (
self.exec_in_container(
[
"bash",
"-c",
"echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(
path
),
]
)
== "yes\n"
self.exec_in_container(
[
"bash",
"-c",
"echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(
path
),
]
)
== "yes\n"
)
def copy_file_to_container(self, local_path, dest_path):
@ -3921,7 +3921,7 @@ class ClickHouseInstance:
return None
def restart_with_original_version(
self, stop_start_wait_sec=300, callback_onstop=None, signal=15
self, stop_start_wait_sec=300, callback_onstop=None, signal=15
):
begin_time = time.time()
if not self.stay_alive:
@ -3977,11 +3977,11 @@ class ClickHouseInstance:
self.wait_start(time_left)
def restart_with_latest_version(
self,
stop_start_wait_sec=300,
callback_onstop=None,
signal=15,
fix_metadata=False,
self,
stop_start_wait_sec=300,
callback_onstop=None,
signal=15,
fix_metadata=False,
):
begin_time = time.time()
if not self.stay_alive:
@ -4127,9 +4127,9 @@ class ClickHouseInstance:
continue
except socket.error as e:
if (
e.errno == errno.ECONNREFUSED
or e.errno == errno.EHOSTUNREACH
or e.errno == errno.ENETUNREACH
e.errno == errno.ECONNREFUSED
or e.errno == errno.EHOSTUNREACH
or e.errno == errno.ENETUNREACH
):
time.sleep(0.1)
else:
@ -4255,8 +4255,8 @@ class ClickHouseInstance:
write_embedded_config("0_common_instance_users.xml", users_d_dir)
if (
os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None
and self.allow_analyzer
os.environ.get("CLICKHOUSE_USE_NEW_ANALYZER") is not None
and self.allow_analyzer
):
write_embedded_config("0_common_enable_analyzer.xml", users_d_dir)
@ -4272,9 +4272,9 @@ class ClickHouseInstance:
# for tags that don't specify a version we assume it has a version of ClickHouse
# that supports async replication if a test for it is present
if (
version == None
or version["major"] > 23
or (version["major"] == 23 and version["minor"] >= 9)
version == None
or version["major"] > 23
or (version["major"] == 23 and version["minor"] >= 9)
):
write_embedded_config(
"0_common_enable_keeper_async_replication.xml", self.config_d_dir
@ -4396,7 +4396,7 @@ class ClickHouseInstance:
# In case the environment variables are exclusive, we don't want it to be in the cluster's env file.
# Instead, a separate env file will be created for the instance and needs to be filled with cluster's env variables.
if self.exclusive_env_variables is True:
if self.instance_env_variables is True:
self.env_variables.update(self.cluster.env_variables)
else:
self.cluster.env_variables.update(self.env_variables)
@ -4414,18 +4414,18 @@ class ClickHouseInstance:
)
else:
entrypoint_cmd = (
"["
+ ", ".join(map(lambda x: '"' + x + '"', entrypoint_cmd.split()))
+ "]"
"["
+ ", ".join(map(lambda x: '"' + x + '"', entrypoint_cmd.split()))
+ "]"
)
logging.debug("Entrypoint cmd: {}".format(entrypoint_cmd))
networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = ""
if (
self.ipv4_address is not None
or self.ipv6_address is not None
or self.hostname != self.name
self.ipv4_address is not None
or self.ipv6_address is not None
or self.hostname != self.name
):
networks = "networks:"
app_net = "default:"
@ -4440,24 +4440,24 @@ class ClickHouseInstance:
if not self.with_installed_binary:
binary_volume = "- " + self.server_bin_path + ":/usr/bin/clickhouse"
odbc_bridge_volume = (
"- " + self.odbc_bridge_bin_path + ":/usr/bin/clickhouse-odbc-bridge"
"- " + self.odbc_bridge_bin_path + ":/usr/bin/clickhouse-odbc-bridge"
)
library_bridge_volume = (
"- "
+ self.library_bridge_bin_path
+ ":/usr/bin/clickhouse-library-bridge"
"- "
+ self.library_bridge_bin_path
+ ":/usr/bin/clickhouse-library-bridge"
)
else:
binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh"
odbc_bridge_volume = (
"- "
+ self.odbc_bridge_bin_path
+ ":/usr/share/clickhouse-odbc-bridge_fresh"
"- "
+ self.odbc_bridge_bin_path
+ ":/usr/share/clickhouse-odbc-bridge_fresh"
)
library_bridge_volume = (
"- "
+ self.library_bridge_bin_path
+ ":/usr/share/clickhouse-library-bridge_fresh"
"- "
+ self.library_bridge_bin_path
+ ":/usr/share/clickhouse-library-bridge_fresh"
)
external_dirs_volumes = ""
@ -4469,14 +4469,14 @@ class ClickHouseInstance:
logging.info(f"external_dir_abs_path={external_dir_abs_path}")
os.makedirs(external_dir_abs_path, exist_ok=True)
external_dirs_volumes += (
"- " + external_dir_abs_path + ":" + external_dir + "\n"
"- " + external_dir_abs_path + ":" + external_dir + "\n"
)
# The current implementation of `self.env_variables` is not exclusive. Meaning the variables
# are shared with all nodes within the same cluster, even if it is specified for a single node.
# In order not to break the existing tests, the `self.exclusive_env_variables` option was added as a workaround.
# IMHO, it would be better to make `self.env_variables` exclusive by defaultand remove the `self.exclusive_env_variables` option.
if self.exclusive_env_variables:
# In order not to break the existing tests, the `self.instance_env_variables` option was added as a workaround.
# IMHO, it would be better to make `self.env_variables` exclusive by defaultand remove the `self.instance_env_variables` option.
if self.instance_env_variables:
self.env_file = p.abspath(p.join(self.path, ".env"))
_create_env_file(self.env_file, self.env_variables)

View File

@ -32,7 +32,7 @@ def cluster():
env_variables={
"http_proxy": "http://proxy1",
},
exclusive_env_variables=True,
instance_env_variables=True,
)
logging.info("Starting cluster...")