mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Merge pull request #19431 from ClickHouse/integration_tests_forbid_check_call
Integration tests: print stderr of failed subprocess
This commit is contained in:
commit
be2d1deac2
@ -45,10 +45,19 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
|
||||
f.write("=".join([var, value]) + "\n")
|
||||
return full_path
|
||||
|
||||
def run_and_check(args, env=None, shell=False):
|
||||
res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell)
|
||||
if res.returncode != 0:
|
||||
# check_call(...) from subprocess does not print stderr, so we do it manually
|
||||
print('Stderr:\n{}\n'.format(res.stderr))
|
||||
print('Stdout:\n{}\n'.format(res.stdout))
|
||||
raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr))
|
||||
|
||||
|
||||
def subprocess_check_call(args):
|
||||
# Uncomment for debugging
|
||||
# print('run:', ' ' . join(args))
|
||||
subprocess.check_call(args)
|
||||
run_and_check(args)
|
||||
|
||||
|
||||
def subprocess_call(args):
|
||||
@ -56,7 +65,6 @@ def subprocess_call(args):
|
||||
# print('run:', ' ' . join(args))
|
||||
subprocess.call(args)
|
||||
|
||||
|
||||
def get_odbc_bridge_path():
|
||||
path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH')
|
||||
if path is None:
|
||||
@ -390,9 +398,9 @@ class ClickHouseCluster:
|
||||
raise Exception("You should specity ipv4_address in add_node method")
|
||||
self._replace(node.docker_compose_path, node.ipv4_address, new_ip)
|
||||
node.ipv4_address = new_ip
|
||||
subprocess.check_call(self.base_cmd + ["stop", node.name])
|
||||
subprocess.check_call(self.base_cmd + ["rm", "--force", "--stop", node.name])
|
||||
subprocess.check_call(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name])
|
||||
run_and_check(self.base_cmd + ["stop", node.name])
|
||||
run_and_check(self.base_cmd + ["rm", "--force", "--stop", node.name])
|
||||
run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name])
|
||||
node.ip_address = self.get_instance_ip(node.name)
|
||||
node.client = Client(node.ip_address, command=self.client_bin_path)
|
||||
start_deadline = time.time() + 20.0 # seconds
|
||||
@ -637,7 +645,7 @@ class ClickHouseCluster:
|
||||
os.mkdir(zk_log_data_path)
|
||||
env['ZK_DATA' + str(i)] = zk_data_path
|
||||
env['ZK_DATA_LOG' + str(i)] = zk_log_data_path
|
||||
subprocess.check_call(self.base_zookeeper_cmd + common_opts, env=env)
|
||||
run_and_check(self.base_zookeeper_cmd + common_opts, env=env)
|
||||
for command in self.pre_zookeeper_commands:
|
||||
self.run_kazoo_commands_with_retries(command, repeats=5)
|
||||
self.wait_zookeeper_to_start(120)
|
||||
@ -662,7 +670,7 @@ class ClickHouseCluster:
|
||||
print('Setup kerberized kafka')
|
||||
env = os.environ.copy()
|
||||
env['KERBERIZED_KAFKA_DIR'] = instance.path + '/'
|
||||
subprocess.check_call(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env)
|
||||
run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env)
|
||||
self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1')
|
||||
if self.with_rabbitmq and self.base_rabbitmq_cmd:
|
||||
subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes'])
|
||||
@ -678,13 +686,13 @@ class ClickHouseCluster:
|
||||
print('Setup kerberized HDFS')
|
||||
env = os.environ.copy()
|
||||
env['KERBERIZED_HDFS_DIR'] = instance.path + '/'
|
||||
subprocess.check_call(self.base_kerberized_hdfs_cmd + common_opts, env=env)
|
||||
run_and_check(self.base_kerberized_hdfs_cmd + common_opts, env=env)
|
||||
self.make_hdfs_api(kerberized=True)
|
||||
self.wait_hdfs_to_start(timeout=300)
|
||||
|
||||
if self.with_mongo and self.base_mongo_cmd:
|
||||
print('Setup Mongo')
|
||||
subprocess_check_call(self.base_mongo_cmd + common_opts)
|
||||
run_and_check(self.base_mongo_cmd + common_opts)
|
||||
self.wait_mongo_to_start(30)
|
||||
|
||||
if self.with_redis and self.base_redis_cmd:
|
||||
@ -709,7 +717,7 @@ class ClickHouseCluster:
|
||||
minio_start_cmd = self.base_minio_cmd + common_opts
|
||||
|
||||
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
|
||||
subprocess.check_call(minio_start_cmd, env=env)
|
||||
run_and_check(minio_start_cmd, env=env)
|
||||
|
||||
try:
|
||||
logging.info("Trying to connect to Minio...")
|
||||
@ -754,7 +762,7 @@ class ClickHouseCluster:
|
||||
sanitizer_assert_instance = None
|
||||
with open(self.docker_logs_path, "w+") as f:
|
||||
try:
|
||||
subprocess.check_call(self.base_cmd + ['logs'], stdout=f)
|
||||
subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
|
||||
except Exception as e:
|
||||
print("Unable to get logs from docker.")
|
||||
f.seek(0)
|
||||
|
@ -191,7 +191,7 @@ class _NetworkManager:
|
||||
# before running
|
||||
for i in range(5):
|
||||
try:
|
||||
subprocess.check_call("docker pull yandex/clickhouse-integration-helper", shell=True)
|
||||
subprocess.check_call("docker pull yandex/clickhouse-integration-helper", shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL
|
||||
break
|
||||
except:
|
||||
time.sleep(i)
|
||||
|
@ -1,10 +1,9 @@
|
||||
import os
|
||||
import pytest
|
||||
import subprocess
|
||||
import sys
|
||||
import time
|
||||
import grpc
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, run_and_check
|
||||
from threading import Thread
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
@ -15,7 +14,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
proto_dir = os.path.join(SCRIPT_DIR, './protos')
|
||||
gen_dir = os.path.join(SCRIPT_DIR, './_gen')
|
||||
os.makedirs(gen_dir, exist_ok=True)
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
'python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \
|
||||
{proto_dir}/clickhouse_grpc.proto'.format(proto_dir=proto_dir, gen_dir=gen_dir), shell=True)
|
||||
|
||||
|
@ -1,9 +1,8 @@
|
||||
import os
|
||||
import pytest
|
||||
import subprocess
|
||||
import sys
|
||||
import grpc
|
||||
from helpers.cluster import ClickHouseCluster
|
||||
from helpers.cluster import ClickHouseCluster, run_and_check
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
|
||||
@ -13,7 +12,7 @@ SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
proto_dir = os.path.join(SCRIPT_DIR, './protos')
|
||||
gen_dir = os.path.join(SCRIPT_DIR, './_gen')
|
||||
os.makedirs(gen_dir, exist_ok=True)
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
'python3 -m grpc_tools.protoc -I{proto_dir} --python_out={gen_dir} --grpc_python_out={gen_dir} \
|
||||
{proto_dir}/clickhouse_grpc.proto'.format(proto_dir=proto_dir, gen_dir=gen_dir), shell=True)
|
||||
|
||||
|
@ -1,13 +1,11 @@
|
||||
import time
|
||||
|
||||
import pymysql.cursors
|
||||
import subprocess
|
||||
import pytest
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.network import PartitionManager
|
||||
import pytest
|
||||
from helpers.client import QueryRuntimeException
|
||||
from helpers.cluster import get_docker_compose_path
|
||||
from helpers.cluster import get_docker_compose_path, run_and_check
|
||||
import random
|
||||
|
||||
import threading
|
||||
@ -684,7 +682,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name):
|
||||
t = threading.Thread(target=insert, args=(10000,))
|
||||
t.start()
|
||||
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'stop'])
|
||||
finally:
|
||||
with pytest.raises(QueryRuntimeException) as execption:
|
||||
@ -692,7 +690,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name):
|
||||
clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test")
|
||||
assert "Master maybe lost." in str(execption.value)
|
||||
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'start'])
|
||||
mysql_node.wait_mysql_to_start(120)
|
||||
|
||||
|
@ -1,12 +1,11 @@
|
||||
import os
|
||||
import os.path as p
|
||||
import subprocess
|
||||
import time
|
||||
import pwd
|
||||
import re
|
||||
import pymysql.cursors
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
|
||||
import docker
|
||||
|
||||
from . import materialize_with_ddl
|
||||
@ -87,7 +86,7 @@ class MySQLNodeInstance:
|
||||
print("Can't connect to MySQL " + str(ex))
|
||||
time.sleep(0.5)
|
||||
|
||||
subprocess.check_call(['docker-compose', 'ps', '--services', 'all'])
|
||||
run_and_check(['docker-compose', 'ps', '--services', 'all'])
|
||||
raise Exception("Cannot wait MySQL container")
|
||||
|
||||
@pytest.fixture(scope="module")
|
||||
@ -96,13 +95,13 @@ def started_mysql_5_7():
|
||||
mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose)
|
||||
|
||||
try:
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d'])
|
||||
mysql_node.wait_mysql_to_start(120)
|
||||
yield mysql_node
|
||||
finally:
|
||||
mysql_node.close()
|
||||
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
|
||||
run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
|
||||
'--remove-orphans'])
|
||||
|
||||
|
||||
@ -112,13 +111,13 @@ def started_mysql_8_0():
|
||||
mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose)
|
||||
|
||||
try:
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d'])
|
||||
mysql_node.wait_mysql_to_start(120)
|
||||
yield mysql_node
|
||||
finally:
|
||||
mysql_node.close()
|
||||
subprocess.check_call(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
|
||||
run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes',
|
||||
'--remove-orphans'])
|
||||
|
||||
|
||||
|
@ -3,14 +3,13 @@
|
||||
import datetime
|
||||
import math
|
||||
import os
|
||||
import subprocess
|
||||
import time
|
||||
|
||||
import docker
|
||||
import pymysql.connections
|
||||
import pytest
|
||||
from docker.models.containers import Container
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
|
||||
|
||||
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
|
||||
DOCKER_COMPOSE_PATH = get_docker_compose_path()
|
||||
@ -35,7 +34,7 @@ def server_address():
|
||||
@pytest.fixture(scope='module')
|
||||
def mysql_client():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1')
|
||||
|
||||
@ -62,7 +61,7 @@ def mysql_server(mysql_client):
|
||||
@pytest.fixture(scope='module')
|
||||
def golang_container():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_golang1_1')
|
||||
|
||||
@ -70,7 +69,7 @@ def golang_container():
|
||||
@pytest.fixture(scope='module')
|
||||
def php_container():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_php1_1')
|
||||
|
||||
@ -78,7 +77,7 @@ def php_container():
|
||||
@pytest.fixture(scope='module')
|
||||
def nodejs_container():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1')
|
||||
|
||||
@ -86,7 +85,7 @@ def nodejs_container():
|
||||
@pytest.fixture(scope='module')
|
||||
def java_container():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_java1_1')
|
||||
|
||||
|
@ -1,11 +1,8 @@
|
||||
# -*- coding: utf-8 -*-
|
||||
|
||||
|
||||
|
||||
import datetime
|
||||
import decimal
|
||||
import os
|
||||
import subprocess
|
||||
import sys
|
||||
import time
|
||||
import uuid
|
||||
@ -14,7 +11,7 @@ import docker
|
||||
import psycopg2 as py_psql
|
||||
import psycopg2.extras
|
||||
import pytest
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path
|
||||
from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check
|
||||
|
||||
psycopg2.extras.register_uuid()
|
||||
|
||||
@ -42,7 +39,7 @@ def server_address():
|
||||
@pytest.fixture(scope='module')
|
||||
def psql_client():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_psql_1')
|
||||
|
||||
@ -66,7 +63,7 @@ def psql_server(psql_client):
|
||||
@pytest.fixture(scope='module')
|
||||
def java_container():
|
||||
docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml')
|
||||
subprocess.check_call(
|
||||
run_and_check(
|
||||
['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build'])
|
||||
yield docker.from_env().containers.get(cluster.project_name + '_java_1')
|
||||
|
||||
|
@ -126,3 +126,7 @@ find $ROOT_PATH/{src,programs,utils} -name '*.h' -or -name '*.cpp' |
|
||||
# Conflict markers
|
||||
find $ROOT_PATH/{src,base,programs,utils,tests,docs,website,cmake} -name '*.md' -or -name '*.cpp' -or -name '*.h' |
|
||||
xargs grep -P '^(<<<<<<<|=======|>>>>>>>)$' | grep -P '.' && echo "Conflict markers are found in files"
|
||||
|
||||
# Forbid subprocess.check_call(...) in integration tests because it does not provide enough information on errors
|
||||
find $ROOT_PATH'/tests/integration' -name '*.py' |
|
||||
xargs grep -F 'subprocess.check_call' | grep -v "STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL" && echo "Use helpers.cluster.run_and_check or subprocess.run instead of subprocess.check_call to print detailed info on error"
|
||||
|
Loading…
Reference in New Issue
Block a user