Merge pull request #51514 from azat/tests/fix-integration-tests

Fix running integration tests with spaces in it's names
This commit is contained in:
Alexey Milovidov 2023-07-22 03:38:51 +03:00 committed by GitHub
commit e5797b7fea
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 206 additions and 166 deletions

1
.gitignore vendored
View File

@ -161,6 +161,7 @@ tests/queries/0_stateless/test_*
tests/queries/0_stateless/*.binary
tests/queries/0_stateless/*.generated-expect
tests/queries/0_stateless/*.expect.history
tests/integration/**/_gen
# rust
/rust/**/target

View File

@ -135,4 +135,5 @@ ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'
EXPOSE 2375
ENTRYPOINT ["dockerd-entrypoint.sh"]
CMD ["sh", "-c", "pytest $PYTEST_OPTS"]
# To pass additional arguments (i.e. list of tests) use PYTEST_ADDOPTS
CMD ["sh", "-c", "pytest"]

View File

@ -196,3 +196,8 @@ test_quota/test.py::test_tracking_quota
test_quota/test.py::test_users_xml_is_readonly
test_replicating_constants/test.py::test_different_versions
test_merge_tree_s3/test.py::test_heavy_insert_select_check_memory[node]
test_drop_is_lock_free/test.py::test_query_is_lock_free[detach table]
test_backward_compatibility/test_data_skipping_indices.py::test_index
test_backward_compatibility/test_convert_ordinary.py::test_convert_ordinary_to_atomic
test_backward_compatibility/test_memory_bound_aggregation.py::test_backward_compatability
test_odbc_interaction/test.py::test_postgres_insert

View File

@ -7,9 +7,11 @@ import json
import logging
import os
import random
import re
import shutil
import subprocess
import time
import shlex
import zlib # for crc32
@ -110,16 +112,36 @@ def get_counters(fname):
if not (".py::" in line and " " in line):
continue
line_arr = line.strip().split(" ")
line = line.strip()
# [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client
# ^^^^^^^^^^^^^
if line.strip().startswith("["):
line = re.sub("^\[[^\[\]]*\] \[[^\[\]]*\] ", "", line)
line_arr = line.split(" ")
if len(line_arr) < 2:
logging.debug("Strange line %s", line)
continue
# Lines like:
# [gw0] [ 7%] ERROR test_mysql_protocol/test.py::test_golang_client
# [gw3] [ 40%] PASSED test_replicated_users/test.py::test_rename_replicated[QUOTA]
state = line_arr[-2]
test_name = line_arr[-1]
#
# ERROR test_mysql_protocol/test.py::test_golang_client
# PASSED test_replicated_users/test.py::test_rename_replicated[QUOTA]
# PASSED test_drop_is_lock_free/test.py::test_query_is_lock_free[detach part]
#
state = line_arr.pop(0)
test_name = " ".join(line_arr)
# Normalize test names for lines like this:
#
# FAILED test_storage_s3/test.py::test_url_reconnect_in_the_middle - Exception
# FAILED test_distributed_ddl/test.py::test_default_database[configs] - AssertionError: assert ...
#
test_name = re.sub(
r"^(?P<test_name>[^\[\] ]+)(?P<test_param>\[[^\[\]]*\]|)(?P<test_error> - .*|)$",
r"\g<test_name>\g<test_param>",
test_name,
)
if state in counters:
counters[state].add(test_name)
@ -411,7 +433,7 @@ class ClickhouseIntegrationTestsRunner:
out_file_full = os.path.join(self.result_path, "runner_get_all_tests.log")
cmd = (
"cd {repo_path}/tests/integration && "
"timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' "
"timeout -s 9 1h ./runner {runner_opts} {image_cmd} -- --setup-plan "
"| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' "
"| grep -v 'SKIPPED' | sort -u > {out_file}".format(
repo_path=repo_path,
@ -646,7 +668,7 @@ class ClickhouseIntegrationTestsRunner:
info_basename = test_group_str + "_" + str(i) + ".nfo"
info_path = os.path.join(repo_path, "tests/integration", info_basename)
test_cmd = " ".join([test for test in sorted(test_names)])
test_cmd = " ".join([shlex.quote(test) for test in sorted(test_names)])
parallel_cmd = (
" --parallel {} ".format(num_workers) if num_workers > 0 else ""
)
@ -655,7 +677,7 @@ class ClickhouseIntegrationTestsRunner:
# -E -- (E)rror
# -p -- (p)assed
# -s -- (s)kipped
cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} '-rfEps --run-id={} --color=no --durations=0 {}' | tee {}".format(
cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} -- -rfEps --run-id={} --color=no --durations=0 {} | tee {}".format(
repo_path,
self._get_runner_opts(),
image_cmd,
@ -766,6 +788,7 @@ class ClickhouseIntegrationTestsRunner:
and test not in counters["ERROR"]
and test not in counters["SKIPPED"]
and test not in counters["FAILED"]
and test not in counters["BROKEN"]
and "::" in test
):
counters["ERROR"].append(test)
@ -999,16 +1022,6 @@ class ClickhouseIntegrationTestsRunner:
if "(memory)" in self.params["context_name"]:
result_state = "success"
for res in test_result:
# It's not easy to parse output of pytest
# Especially when test names may contain spaces
# Do not allow it to avoid obscure failures
if " " not in res[0]:
continue
logging.warning("Found invalid test name with space: %s", res[0])
status_text = "Found test with invalid name, see main log"
result_state = "failure"
return result_state, status_text, test_result, []

View File

@ -11,6 +11,7 @@ import subprocess
import sys
import string
import random
import shlex
def random_str(length=6):
@ -135,9 +136,7 @@ def check_args_and_update_paths(args):
def docker_kill_handler_handler(signum, frame):
subprocess.check_call(
'docker ps --all --quiet --filter name={name} --format="{{{{.ID}}}}"'.format(
name=CONTAINER_NAME
),
"docker ps --all --quiet --filter name={name}".format(name=CONTAINER_NAME),
shell=True,
)
raise KeyboardInterrupt("Killed by Ctrl+C")
@ -407,8 +406,14 @@ if __name__ == "__main__":
if args.analyzer:
use_analyzer = "-e CLICKHOUSE_USE_NEW_ANALYZER=1"
pytest_opts = " ".join(args.pytest_args).replace("'", "\\'")
tests_list = " ".join(args.tests_list)
# NOTE: since pytest options is in the argument value already we need to additionally escape '"'
pytest_opts = " ".join(
map(lambda x: shlex.quote(x).replace('"', '\\"'), args.pytest_args)
)
tests_list = " ".join(
map(lambda x: shlex.quote(x).replace('"', '\\"'), args.tests_list)
)
cmd_base = (
f"docker run {net} {tty} --rm --name {CONTAINER_NAME} "
"--privileged --dns-search='.' " # since recent dns search leaks from host
@ -420,7 +425,7 @@ if __name__ == "__main__":
f"--volume={args.src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos "
f"--volume=/run:/run/host:ro {dockerd_internal_volume} {env_tags} {env_cleanup} "
f"-e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 {use_analyzer} -e PYTHONUNBUFFERED=1 "
f"-e PYTEST_OPTS='{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv'"
f'-e PYTEST_ADDOPTS="{parallel_args} {pytest_opts} {tests_list} {rand_args} -vvv"'
f" {DIND_INTEGRATION_TESTS_IMAGE_NAME}:{args.docker_image_version}"
)
@ -431,7 +436,7 @@ if __name__ == "__main__":
)
containers = subprocess.check_output(
f"docker ps --all --quiet --filter name={CONTAINER_NAME} --format={{{{.ID}}}}",
f"docker ps --all --quiet --filter name={CONTAINER_NAME}",
shell=True,
universal_newlines=True,
).splitlines()

View File

@ -582,75 +582,83 @@ def test_sqlite_odbc_cached_dictionary(started_cluster):
def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster):
skip_test_msan(node1)
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')"
)
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed")
node1.exec_in_container(
["ss", "-K", "dport", "postgresql"], privileged=True, user="root"
)
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed")
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))",
"hello",
)
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))",
"world",
)
cursor.execute("truncate table clickhouse.test_table")
try:
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')"
)
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed")
node1.exec_in_container(
["ss", "-K", "dport", "postgresql"], privileged=True, user="root"
)
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_hashed")
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(1))",
"hello",
)
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(2))",
"world",
)
finally:
cursor.execute("truncate table clickhouse.test_table")
def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster):
skip_test_msan(node1)
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')")
for i in range(100):
try:
node1.query("system reload dictionary postgres_odbc_hashed", timeout=15)
except Exception as ex:
assert False, "Exception occured -- odbc-bridge hangs: " + str(ex)
try:
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute("insert into clickhouse.test_table values(3, 3, 'xxx')")
for i in range(100):
try:
node1.query("system reload dictionary postgres_odbc_hashed", timeout=15)
except Exception as ex:
assert False, "Exception occured -- odbc-bridge hangs: " + str(ex)
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))",
"xxx",
)
cursor.execute("truncate table clickhouse.test_table")
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_hashed', 'column2', toUInt64(3))",
"xxx",
)
finally:
cursor.execute("truncate table clickhouse.test_table")
def test_no_connection_pooling(started_cluster):
skip_test_msan(node1)
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')"
)
node1.exec_in_container(["ss", "-K", "dport", "5432"], privileged=True, user="root")
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_nopool")
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(1))",
"hello",
)
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(2))",
"world",
)
try:
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"insert into clickhouse.test_table values(1, 1, 'hello'),(2, 2, 'world')"
)
node1.exec_in_container(
["ss", "-K", "dport", "5432"], privileged=True, user="root"
)
node1.query("SYSTEM RELOAD DICTIONARY postgres_odbc_nopool")
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(1))",
"hello",
)
assert_eq_with_retry(
node1,
"select dictGetString('postgres_odbc_nopool', 'column2', toUInt64(2))",
"world",
)
# No open connections should be left because we don't use connection pooling.
assert "" == node1.exec_in_container(
["ss", "-H", "dport", "5432"], privileged=True, user="root"
)
cursor.execute("truncate table clickhouse.test_table")
# No open connections should be left because we don't use connection pooling.
assert "" == node1.exec_in_container(
["ss", "-H", "dport", "5432"], privileged=True, user="root"
)
finally:
cursor.execute("truncate table clickhouse.test_table")
def test_postgres_insert(started_cluster):
@ -662,112 +670,119 @@ def test_postgres_insert(started_cluster):
# postgres .yml file). This is needed to check parsing, validation and
# reconstruction of connection string.
node1.query(
"create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')"
)
node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')")
assert node1.query("select * from pg_insert") == "1\t1\thello\n2\t2\tworld\n"
node1.query(
"insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test"
)
node1.query(
"insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')"
" select number, number, 's' || toString(number) from numbers (4, 7)"
)
assert (
node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n"
)
assert (
try:
node1.query(
"select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))"
"create table pg_insert (id UInt64, column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')"
)
== "55\t10\n"
)
node1.query("DROP TABLE pg_insert")
conn.cursor().execute("truncate table clickhouse.test_table")
node1.query("insert into pg_insert values (1, 1, 'hello'), (2, 2, 'world')")
assert node1.query("select * from pg_insert") == "1\t1\thello\n2\t2\tworld\n"
node1.query(
"insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,3,test"
)
node1.query(
"insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')"
" select number, number, 's' || toString(number) from numbers (4, 7)"
)
assert (
node1.query("select sum(column1), count(column1) from pg_insert")
== "55\t10\n"
)
assert (
node1.query(
"select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))"
)
== "55\t10\n"
)
finally:
node1.query("DROP TABLE IF EXISTS pg_insert")
conn.cursor().execute("truncate table clickhouse.test_table")
def test_odbc_postgres_date_data_type(started_cluster):
skip_test_msan(node1)
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)"
)
try:
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"CREATE TABLE clickhouse.test_date (id integer, column1 integer, column2 date)"
)
cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')")
conn.commit()
cursor.execute("INSERT INTO clickhouse.test_date VALUES (1, 1, '2020-12-01')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (2, 2, '2020-12-02')")
cursor.execute("INSERT INTO clickhouse.test_date VALUES (3, 3, '2020-12-03')")
conn.commit()
node1.query(
"""
CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date)
ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')"""
)
node1.query(
"""
CREATE TABLE test_date (id UInt64, column1 UInt64, column2 Date)
ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_date')"""
)
expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n"
result = node1.query("SELECT * FROM test_date")
assert result == expected
cursor.execute("DROP TABLE clickhouse.test_date")
node1.query("DROP TABLE test_date")
expected = "1\t1\t2020-12-01\n2\t2\t2020-12-02\n3\t3\t2020-12-03\n"
result = node1.query("SELECT * FROM test_date")
assert result == expected
finally:
cursor.execute("DROP TABLE clickhouse.test_date")
node1.query("DROP TABLE IF EXISTS test_date")
def test_odbc_postgres_conversions(started_cluster):
skip_test_msan(node1)
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
try:
conn = get_postgres_conn(started_cluster)
cursor = conn.cursor()
cursor.execute(
"""CREATE TABLE clickhouse.test_types (
a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial,
h timestamp)"""
)
cursor.execute(
"""CREATE TABLE clickhouse.test_types (
a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial,
h timestamp)"""
)
node1.query(
"""
INSERT INTO TABLE FUNCTION
odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')
VALUES (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12')"""
)
node1.query(
"""
INSERT INTO TABLE FUNCTION
odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')
VALUES (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12')"""
)
result = node1.query(
"""
SELECT a, b, c, d, e, f, g, h
FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')
"""
)
result = node1.query(
"""
SELECT a, b, c, d, e, f, g, h
FROM odbc('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')
"""
)
assert (
result
== "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\n"
)
cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types")
assert (
result
== "-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\n"
)
cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types")
cursor.execute(
"""CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)"""
)
cursor.execute(
"""CREATE TABLE clickhouse.test_types (column1 Timestamp, column2 Numeric)"""
)
node1.query(
"""
CREATE TABLE test_types (column1 DateTime64, column2 Decimal(5, 1))
ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')"""
)
node1.query(
"""
CREATE TABLE test_types (column1 DateTime64, column2 Decimal(5, 1))
ENGINE=ODBC('DSN=postgresql_odbc; Servername=postgre-sql.local', 'clickhouse', 'test_types')"""
)
node1.query(
"""INSERT INTO test_types
SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)"""
)
node1.query(
"""INSERT INTO test_types
SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)"""
)
expected = node1.query(
"SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)"
)
result = node1.query("SELECT * FROM test_types")
cursor.execute("DROP TABLE clickhouse.test_types")
node1.query("DROP TABLE test_types")
assert result == expected
expected = node1.query(
"SELECT toDateTime64('2019-01-01 00:00:00', 3, 'Etc/UTC'), toDecimal32(1.1, 1)"
)
result = node1.query("SELECT * FROM test_types")
assert result == expected
finally:
cursor.execute("DROP TABLE IF EXISTS clickhouse.test_types")
node1.query("DROP TABLE IF EXISTS test_types")
def test_odbc_cyrillic_with_varchar(started_cluster):