diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index dd66e92d098..dd409429665 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -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, - 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=[], + 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=[], ) -> "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, - instance_env_variables=instance_env_variables, + exclusive_env_variables=exclusive_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, - 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, + 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.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.instance_env_variables = instance_env_variables + self.exclusive_env_variables = exclusive_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.instance_env_variables is True: + if self.exclusive_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.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: + # 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: self.env_file = p.abspath(p.join(self.path, ".env")) _create_env_file(self.env_file, self.env_variables) diff --git a/tests/integration/test_s3_table_function_with_http_proxy/test.py b/tests/integration/test_s3_table_function_with_http_proxy/test.py index 42579c43a9e..169de81759d 100644 --- a/tests/integration/test_s3_table_function_with_http_proxy/test.py +++ b/tests/integration/test_s3_table_function_with_http_proxy/test.py @@ -32,7 +32,7 @@ def cluster(): env_variables={ "http_proxy": "http://proxy1", }, - instance_env_variables=True, + exclusive_env_variables=True, ) logging.info("Starting cluster...")