mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
parent
0b97cd49e8
commit
9695bf5db9
@ -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)
|
||||
|
||||
|
@ -32,7 +32,7 @@ def cluster():
|
||||
env_variables={
|
||||
"http_proxy": "http://proxy1",
|
||||
},
|
||||
instance_env_variables=True,
|
||||
exclusive_env_variables=True,
|
||||
)
|
||||
|
||||
logging.info("Starting cluster...")
|
||||
|
Loading…
Reference in New Issue
Block a user