Convert skip_list.json into first line comments.

This commit is contained in:
Vitaly Baranov 2021-09-12 15:35:00 +03:00
parent a4153e5629
commit bbb192ee85
13 changed files with 739 additions and 320 deletions

View File

@ -80,7 +80,7 @@ LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-client --query "RENAM
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits"
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-client --query "SHOW TABLES FROM test"
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test -j 8 --testname --shard --zookeeper --print-time --use-skip-list 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_result.txt
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test -j 8 --testname --shard --zookeeper --print-time 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee /test_result.txt
readarray -t FAILED_TESTS < <(awk '/FAIL|TIMEOUT|ERROR/ { print substr($3, 1, length($3)-1) }' "/test_result.txt")
@ -97,7 +97,7 @@ then
echo "Going to run again: ${FAILED_TESTS[*]}"
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test --order=random --testname --shard --zookeeper --use-skip-list "${FAILED_TESTS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a /test_result.txt
LLVM_PROFILE_FILE='client_coverage_%5m.profraw' clickhouse-test --order=random --testname --shard --zookeeper "${FAILED_TESTS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a /test_result.txt
else
echo "No failed tests"
fi

View File

@ -275,11 +275,10 @@ function run_tests
00834_cancel_http_readonly_queries_on_client_close
00911_tautological_compare
# Hyperscan
00926_multimatch
00929_multi_match_edit_distance
01681_hyperscan_debug_assertion
02004_max_hyperscan_regex_length
00926_multimatch # Hyperscan
00929_multi_match_edit_distance # Hyperscan
01681_hyperscan_debug_assertion # Hyperscan
02004_max_hyperscan_regex_length # Hyperscan
01176_mysql_client_interactive # requires mysql client
01031_mutations_interpreter_and_context
@ -344,70 +343,54 @@ function run_tests
sha256
xz
# Not sure why these two fail even in sequential mode. Disabled for now
# to make some progress.
00646_url_engine
00974_query_profiler
00646_url_engine # Not sure why fail even in sequential mode. Disabled for now to make some progress.
00974_query_profiler # Not sure why fail even in sequential mode. Disabled for now to make some progress.
# In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default
01504_rocksdb
01686_rocksdb
01504_rocksdb # In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default
01686_rocksdb # In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default
# Look at DistributedFilesToInsert, so cannot run in parallel.
01460_DistributedFilesToInsert
01460_DistributedFilesToInsert # Look at DistributedFilesToInsert, so cannot run in parallel.
01541_max_memory_usage_for_user_long
# Require python libraries like scipy, pandas and numpy
01322_ttest_scipy
01561_mann_whitney_scipy
01322_ttest_scipy # Require python libraries like scipy, pandas and numpy
01561_mann_whitney_scipy # Require python libraries like scipy, pandas and numpy
01545_system_errors
# Checks system.errors
01563_distributed_query_finish
01545_system_errors # Checks system.errors
01563_distributed_query_finish # Checks system.errors
# nc - command not found
01601_proxy_protocol
01622_defaults_for_url_engine
01601_proxy_protocol # nc - command not found
01622_defaults_for_url_engine # nc - command not found
# JSON functions
01666_blns
01666_blns # JSON functions
# Requires postgresql-client
01802_test_postgresql_protocol_with_row_policy
01802_test_postgresql_protocol_with_row_policy # Requires postgresql-client
# Depends on AWS
01801_s3_cluster
02012_settings_clause_for_s3
01801_s3_cluster # Depends on AWS
02012_settings_clause_for_s3 # Depends on AWS
# needs psql
01889_postgresql_protocol_null_fields
01889_postgresql_protocol_null_fields # needs psql
# needs pv
01923_network_receive_time_metric_insert
01923_network_receive_time_metric_insert # needs pv
01889_sqlite_read_write
# needs s2
01849_geoToS2
01851_s2_to_geo
01852_s2_get_neighbours
01853_s2_cells_intersect
01854_s2_cap_contains
01854_s2_cap_union
01849_geoToS2 # needs s2
01851_s2_to_geo # needs s2
01852_s2_get_neighbours # needs s2
01853_s2_cells_intersect # needs s2
01854_s2_cap_contains # needs s2
01854_s2_cap_union # needs s2
# needs s3
01944_insert_partition_by
01944_insert_partition_by # needs s3
# depends on Go
02013_zlib_read_after_eof
02013_zlib_read_after_eof # depends on Go
# Accesses CH via mysql table function (which is unavailable)
01747_system_session_log_long
01747_system_session_log_long # Accesses CH via mysql table function (which is unavailable)
)
time clickhouse-test --hung-check -j 8 --order=random --use-skip-list \
--no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" \
time clickhouse-test --hung-check -j 8 --order=random \
--fast-tests-only --no-long --testname --shard --zookeeper \
-- "$FASTTEST_FOCUS" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee "$FASTTEST_OUTPUT/test_log.txt"

View File

@ -108,7 +108,7 @@ function run_tests()
ADDITIONAL_OPTIONS+=('--replicated-database')
fi
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --use-skip-list --print-time "${ADDITIONAL_OPTIONS[@]}" \
clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \
"$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
}

View File

@ -97,7 +97,7 @@ function run_tests()
fi
clickhouse-test --testname --shard --zookeeper --hung-check --print-time \
--use-skip-list --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
--test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \
| ts '%Y-%m-%d %H:%M:%S' \
| tee -a test_output/test_result.txt
}

View File

@ -13,8 +13,4 @@ dpkg -i package_folder/clickhouse-test_*.deb
service clickhouse-server start && sleep 5
if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then
SKIP_LIST_OPT="--use-skip-list"
fi
clickhouse-test --testname --shard --zookeeper "$SKIP_LIST_OPT" "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt
clickhouse-test --testname --shard --zookeeper "$ADDITIONAL_OPTIONS" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt

View File

@ -10,14 +10,6 @@ import logging
import time
def get_skip_list_cmd(path):
with open(path, 'r') as f:
for line in f:
if '--use-skip-list' in line:
return '--use-skip-list'
return ''
def get_options(i):
options = []
client_options = []
@ -56,8 +48,6 @@ def get_options(i):
def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_time_limit):
skip_list_opt = get_skip_list_cmd(cmd)
global_time_limit_option = ''
if global_time_limit:
global_time_limit_option = "--global_time_limit={}".format(global_time_limit)
@ -66,7 +56,7 @@ def run_func_test(cmd, output_prefix, num_processes, skip_tests_option, global_t
pipes = []
for i in range(0, len(output_paths)):
f = open(output_paths[i], 'w')
full_command = "{} {} {} {} {}".format(cmd, skip_list_opt, get_options(i), global_time_limit_option, skip_tests_option)
full_command = "{} {} {} {}".format(cmd, get_options(i), global_time_limit_option, skip_tests_option)
logging.info("Run func tests '%s'", full_command)
p = Popen(full_command, shell=True, stdout=f, stderr=f)
pipes.append(p)

View File

@ -4,6 +4,7 @@ set (CLICKHOUSE_CLIENT_SOURCES
QueryFuzzer.cpp
Suggest.cpp
TestHint.cpp
TestTags.cpp
)
set (CLICKHOUSE_CLIENT_LINK

View File

@ -6,6 +6,7 @@
#include "QueryFuzzer.h"
#include "Suggest.h"
#include "TestHint.h"
#include "TestTags.h"
#if USE_REPLXX
# include <common/ReplxxLineReader.h>
@ -1078,12 +1079,17 @@ private:
bool echo_query = echo_queries;
/// Test tags are started with "--" so they are interpreted as comments anyway.
/// But if the echo is enabled we have to remove the test tags from `all_queries_text`
/// because we don't want test tags to be echoed.
size_t test_tags_length = test_mode ? getTestTagsLength(all_queries_text) : 0;
/// Several queries separated by ';'.
/// INSERT data is ended by the end of line, not ';'.
/// An exception is VALUES format where we also support semicolon in
/// addition to end of line.
const char * this_query_begin = all_queries_text.data();
const char * this_query_begin = all_queries_text.data() + test_tags_length;
const char * all_queries_end = all_queries_text.data() + all_queries_text.size();
while (this_query_begin < all_queries_end)

View File

@ -0,0 +1,50 @@
#include "TestTags.h"
namespace DB
{
size_t getTestTagsLength(const String & multiline_query)
{
const String & text = multiline_query;
size_t pos = 0;
bool first_line = true;
while (true)
{
size_t line_start = pos;
/// Skip spaces.
while ((pos != text.length()) && (text[pos] == ' ' || text[pos] == '\t'))
++pos;
/// Skip comment "--".
static constexpr const char comment[] = "--";
if (text.compare(pos, strlen(comment), comment) != 0)
return line_start;
pos += strlen(comment);
/// Skip the prefix "Tags:" if it's the first line.
if (first_line)
{
while ((pos != text.length()) && (text[pos] == ' ' || text[pos] == '\t'))
++pos;
static constexpr const char tags_prefix[] = "Tags:";
if (text.compare(pos, strlen(tags_prefix), tags_prefix) != 0)
return 0;
pos += strlen(tags_prefix);
first_line = false;
}
/// Skip end-of-line.
size_t eol_pos = text.find_first_of("\r\n", pos);
if (eol_pos == String::npos)
return text.length();
bool two_chars_eol = (eol_pos + 1 < text.length()) && ((text[eol_pos + 1] == '\r') || (text[eol_pos + 1] == '\n')) && (text[eol_pos + 1] != text[eol_pos]);
size_t eol_length = two_chars_eol ? 2 : 1;
pos = eol_pos + eol_length;
}
}
}

View File

@ -0,0 +1,18 @@
#pragma once
#include <Core/Types.h>
namespace DB
{
/// Returns the length of a text looking like
/// -- Tags: x, y, z
/// -- Tag x: explanation of tag x
/// -- Tag y: explanation of tag y
/// -- Tag z: explanation of tag z
///
/// at the beginning of a multiline query.
/// If there are no test tags in the multiline query the function returns 0.
size_t getTestTagsLength(const String & multiline_query);
}

View File

@ -6,12 +6,11 @@ import os
import os.path
import signal
import re
import json
import copy
import traceback
from argparse import ArgumentParser
from typing import Tuple, Union, Optional, TextIO
from typing import Tuple, Union, Optional, TextIO, Dict, Set, List
import shlex
import subprocess
from subprocess import Popen
@ -83,76 +82,6 @@ def stop_tests():
os.killpg(os.getpgid(os.getpid()), signal.SIGTERM)
signal.signal(signal.SIGTERM, signal.SIG_DFL)
def json_minify(string):
"""
Removes all js-style comments from json string. Allows to have comments in skip_list.json.
The code was taken from https://github.com/getify/JSON.minify/tree/python under the MIT license.
"""
tokenizer = re.compile(r'"|(/\*)|(\*/)|(//)|\n|\r')
end_slashes_re = re.compile(r'(\\)*$')
in_string = False
in_multi = False
in_single = False
new_str = []
index = 0
for match in re.finditer(tokenizer, string):
if not (in_multi or in_single):
tmp = string[index:match.start()]
new_str.append(tmp)
else:
# Replace comments with white space so that the JSON parser reports
# the correct column numbers on parsing errors.
new_str.append(' ' * (match.start() - index))
index = match.end()
val = match.group()
if val == '"' and not (in_multi or in_single):
escaped = end_slashes_re.search(string, 0, match.start())
# start of string or unescaped quote character to end string
if not in_string or (escaped is None or len(escaped.group()) % 2 == 0): # noqa
in_string = not in_string
index -= 1 # include " character in next catch
elif not (in_string or in_multi or in_single):
if val == '/*':
in_multi = True
elif val == '//':
in_single = True
elif val == '*/' and in_multi and not (in_string or in_single):
in_multi = False
new_str.append(' ' * len(val))
elif val in '\r\n' and not (in_multi or in_string) and in_single:
in_single = False
elif not in_multi or in_single: # noqa
new_str.append(val)
if val in '\r\n':
new_str.append(val)
elif in_multi or in_single:
new_str.append(' ' * len(val))
new_str.append(string[index:])
return ''.join(new_str)
def remove_control_characters(s):
"""
https://github.com/html5lib/html5lib-python/issues/96#issuecomment-43438438
"""
def str_to_int(s, default, base=10):
if int(s, base) < 0x10000:
return chr(int(s, base))
return default
s = re.sub(r"&#(\d+);?", lambda c: str_to_int(c.group(1), c.group(0)), s)
s = re.sub(r"&#[xX]([0-9a-fA-F]+);?", lambda c: str_to_int(c.group(1), c.group(0), base=16), s)
s = re.sub(r"[\x00-\x08\x0b\x0e-\x1f\x7f]", "", s)
return s
def get_db_engine(args, database_name):
if args.replicated_database:
@ -429,48 +358,69 @@ def print_test_time(test_time) -> str:
return ''
def should_skip_test_by_name(name: str, test_ext: str) -> Tuple[bool, str]:
if args.skip and any(s in name for s in args.skip):
return True, "skip"
if not args.zookeeper and ('zookeeper' in name or 'replica' in name):
return True, "no zookeeper"
if not args.shard and \
('shard' in name or 'distributed' in name or 'global' in name):
return True, "no shard"
# Tests for races and deadlocks usually are run in a loop for a significant
# amount of time
if args.no_long and \
('long' in name or 'deadlock' in name or 'race' in name):
return True, "no long"
if not USE_JINJA and test_ext.endswith("j2"):
return True, "no jinja"
return False, ""
def should_skip_disabled_test(name: str, suite_dir: str) -> Tuple[bool, str]:
disabled_file = os.path.join(suite_dir, name) + '.disabled'
if os.path.exists(disabled_file) and not args.disabled:
return True, open(disabled_file, 'r').read()
return False, ""
# should skip test, should increment skipped_total, skip reason
def should_skip_test(name: str, test_ext: str, suite_dir: str) -> Tuple[bool, bool, str]:
should_skip, skip_reason = should_skip_test_by_name(name, test_ext)
def should_skip_test(name: str, test_ext: str, suite_dir: str, all_tags: Dict[str, Set[str]]) -> Tuple[bool, bool, str]:
tags = all_tags.get(name + test_ext)
if should_skip:
return True, True, skip_reason
should_skip = False
increment_skip_count = False
skip_reason = ''
should_skip, skip_reason = should_skip_disabled_test(name, suite_dir)
if tags and ('disabled' in tags) and not args.disabled:
should_skip = True
increment_skip_count = False
skip_reason = 'disabled'
return should_skip, False, skip_reason
elif os.path.exists(os.path.join(suite_dir, name) + '.disabled') and not args.disabled:
should_skip = True
increment_skip_count = False
skip_reason = 'disabled'
elif args.skip and any(s in name for s in args.skip):
should_skip = True
increment_skip_count = True
skip_reason = 'skip'
elif not USE_JINJA and test_ext.endswith("j2"):
should_skip = True
increment_skip_count = True
skip_reason = 'no jinja'
elif tags and (('zookeeper' in tags) or ('replica' in tags)) and not args.zookeeper:
should_skip = True
increment_skip_count = True
skip_reason = 'no zookeeper'
elif tags and (('shard' in tags) or ('distributed' in tags) or ('global' in tags)) and not args.shard:
should_skip = True
increment_skip_count = True
skip_reason = 'no shard'
elif tags and ('no-fasttest' in tags) and args.fast_tests_only:
should_skip = True
increment_skip_count = True
skip_reason = 'running fast tests only'
elif tags and (('long' in tags) or ('deadlock' in tags) or ('race' in tags)) and args.no_long:
# Tests for races and deadlocks usually are run in a loop for a significant amount of time
should_skip = True
increment_skip_count = True
skip_reason = 'not running long tests'
elif tags and ('no-replicated-database' in tags) and args.replicated_database:
should_skip = True
increment_skip_count = True
skip_reason = 'replicated-database'
elif tags:
for build_flag in args.build_flags:
if 'no-' + build_flag in tags:
should_skip = True
increment_skip_count = True
skip_reason = build_flag
break
return should_skip, increment_skip_count, skip_reason
def send_test_name_failed(suite: str, case: str) -> bool:
@ -491,9 +441,9 @@ def send_test_name_failed(suite: str, case: str) -> bool:
restarted_tests = [] # (test, stderr)
# def run_tests_array(all_tests, suite, suite_dir, suite_tmp_dir, run_total):
# def run_tests_array(all_tests, num_tests, suite, suite_dir, suite_tmp_dir, all_tags):
def run_tests_array(all_tests_with_params):
all_tests, num_tests, suite, suite_dir, suite_tmp_dir = all_tests_with_params
all_tests, num_tests, suite, suite_dir, suite_tmp_dir, all_tags = all_tests_with_params
global stop_time
global exit_code
global server_died
@ -558,7 +508,7 @@ def run_tests_array(all_tests_with_params):
status = "{0:72}".format(removesuffix(name, ".gen", ".sql") + ": ")
skip_test, increment_skip_count, skip_reason = \
should_skip_test(name, ext, suite_dir)
should_skip_test(name, ext, suite_dir, all_tags)
if skip_test:
status += MSG_SKIPPED + f" - {skip_reason}\n"
@ -782,16 +732,15 @@ def check_server_started(client, retry_count):
class BuildFlags():
THREAD = 'thread-sanitizer'
ADDRESS = 'address-sanitizer'
UNDEFINED = 'ub-sanitizer'
MEMORY = 'memory-sanitizer'
DEBUG = 'debug-build'
UNBUNDLED = 'unbundled-build'
RELEASE = 'release-build'
DATABASE_ORDINARY = 'database-ordinary'
THREAD = 'tsan'
ADDRESS = 'asan'
UNDEFINED = 'ubsan'
MEMORY = 'msan'
DEBUG = 'debug'
UNBUNDLED = 'unbundled'
RELEASE = 'release'
ORDINARY_DATABASE = 'ordinary-database'
POLYMORPHIC_PARTS = 'polymorphic-parts'
DATABASE_REPLICATED = 'database-replicated'
def collect_build_flags(client):
@ -836,7 +785,7 @@ def collect_build_flags(client):
if clickhouse_proc.returncode == 0:
if b'Ordinary' in stdout:
result.append(BuildFlags.DATABASE_ORDINARY)
result.append(BuildFlags.ORDINARY_DATABASE)
else:
raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
@ -902,7 +851,7 @@ def open_client_process(
def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_tests, sequential_tests, parallel):
def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, all_tags, parallel_tests, sequential_tests, parallel):
if jobs > 1 and len(parallel_tests) > 0:
print("Found", len(parallel_tests), "parallel tests and", len(sequential_tests), "sequential tests")
run_n, run_total = parallel.split('/')
@ -920,7 +869,7 @@ def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_test
batch_size = max(1, len(parallel_tests) // jobs)
parallel_tests_array = []
for _ in range(jobs):
parallel_tests_array.append((None, batch_size, suite, suite_dir, suite_tmp_dir))
parallel_tests_array.append((None, batch_size, suite, suite_dir, suite_tmp_dir, all_tags))
with closing(multiprocessing.Pool(processes=jobs)) as pool:
pool.map_async(run_tests_array, parallel_tests_array)
@ -935,11 +884,11 @@ def do_run_tests(jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_test
pool.join()
run_tests_array((sequential_tests, len(sequential_tests), suite, suite_dir, suite_tmp_dir))
run_tests_array((sequential_tests, len(sequential_tests), suite, suite_dir, suite_tmp_dir, all_tags))
return len(sequential_tests) + len(parallel_tests)
else:
num_tests = len(all_tests)
run_tests_array((all_tests, num_tests, suite, suite_dir, suite_tmp_dir))
run_tests_array((all_tests, num_tests, suite, suite_dir, suite_tmp_dir, all_tags))
return num_tests
@ -1053,23 +1002,10 @@ def main(args):
"Server is not responding. Cannot execute 'SELECT 1' query. \
If you are using split build, you have to specify -c option.")
build_flags = collect_build_flags(args.client)
if args.replicated_database:
build_flags.append(BuildFlags.DATABASE_REPLICATED)
if args.use_skip_list:
tests_to_skip_from_list = collect_tests_to_skip(args.skip_list_path, build_flags)
else:
tests_to_skip_from_list = set()
args.build_flags = collect_build_flags(args.client)
if args.skip:
args.skip = set(args.skip) | tests_to_skip_from_list
else:
args.skip = tests_to_skip_from_list
if args.use_skip_list and not args.sequential:
args.sequential = collect_sequential_list(args.skip_list_path)
args.skip = set(args.skip)
base_dir = os.path.abspath(args.queries)
tmp_dir = os.path.abspath(args.tmp)
@ -1148,19 +1084,21 @@ def main(args):
all_tests = get_tests_list(
suite_dir, args.test, args.test_runs, tests_in_suite_key_func)
jobs = args.jobs
all_tags = read_test_tags(suite_dir, all_tests)
parallel_tests = []
sequential_tests = []
if args.sequential:
for test in all_tests:
if any(s in test for s in args.sequential):
sequential_tests.append(test)
else:
sequential_tests = collect_sequential_list(all_tags)
for test in all_tests:
if any(s in test for s in args.sequential):
sequential_tests.append(test)
else:
parallel_tests.append(test)
sequential_tests_set = set(sequential_tests)
parallel_tests = [test for test in all_tests if test not in sequential_tests_set]
total_tests_run += do_run_tests(
jobs, suite, suite_dir, suite_tmp_dir, all_tests, parallel_tests, sequential_tests, args.parallel)
args.jobs, suite, suite_dir, suite_tmp_dir, all_tests, all_tags, parallel_tests, sequential_tests, args.parallel)
if server_died.is_set():
exit_code.value = 1
@ -1232,40 +1170,58 @@ def get_additional_client_options_url(args):
return ''
def collect_tests_to_skip(skip_list_path, build_flags):
result = set([])
def read_test_tags(suite_dir: str, all_tests: List[str]) -> Dict[str, Set[str]]:
def get_comment_sign(filename):
if filename.endswith('.sql') or filename.endswith('.sql.j2'):
return '--'
elif filename.endswith('.sh') or filename.endswith('.py') or filename.endswith('.expect'):
return '#'
else:
raise Exception(f'Unknown file_extension: {filename}')
if not os.path.exists(skip_list_path):
return result
def parse_tags_from_line(line, comment_sign):
if not line.startswith(comment_sign):
return None
tags_str = line[len(comment_sign):].lstrip()
tags_prefix = "Tags:"
if not tags_str.startswith(tags_prefix):
return None
tags_str = tags_str[len(tags_prefix):]
tags = tags_str.split(',')
tags = {tag.strip() for tag in tags}
return tags
with open(skip_list_path, 'r') as skip_list_file:
content = skip_list_file.read()
def is_shebang(line):
return line.startswith('#!')
# allows to have comments in skip_list.json
skip_dict = json.loads(json_minify(content))
def load_tags_from_file(filepath):
with open(filepath, 'r') as file:
try:
line = file.readline()
if is_shebang(line):
line = file.readline()
except UnicodeDecodeError:
return []
return parse_tags_from_line(line, get_comment_sign(filepath))
for build_flag in build_flags:
result |= set(skip_dict[build_flag])
count = len(result)
if count > 0:
print(f"Found file with skip-list {skip_list_path}, {count} test will be skipped")
return result
all_tags = {}
start_time = datetime.now()
for test_name in all_tests:
tags = load_tags_from_file(os.path.join(suite_dir, test_name))
if tags:
all_tags[test_name] = tags
elapsed = (datetime.now() - start_time).total_seconds()
if elapsed > 1:
print(f"Tags for suite {suite_dir} read in {elapsed:.2f} seconds")
return all_tags
def collect_sequential_list(skip_list_path):
if not os.path.exists(skip_list_path):
return set([])
with open(skip_list_path, 'r') as skip_list_file:
content = skip_list_file.read()
# allows to have comments in skip_list.json
skip_dict = json.loads(json_minify(content))
if 'parallel' in skip_dict:
return skip_dict['parallel']
return set([])
def collect_sequential_list(all_tags: Dict[str, Set[str]]) -> List[str]:
res = []
for test_name, tags in all_tags.items():
if ('no-parallel' in tags) or ('sequential' in tags):
res.append(test_name)
return res
if __name__ == '__main__':
@ -1308,10 +1264,9 @@ if __name__ == '__main__':
parser.add_argument('--test-runs', default=1, nargs='?', type=int, help='Run each test many times (useful for e.g. flaky check)')
parser.add_argument('-U', '--unified', default=3, type=int, help='output NUM lines of unified context')
parser.add_argument('-r', '--server-check-retries', default=30, type=int, help='Num of tries to execute SELECT 1 before tests started')
parser.add_argument('--skip-list-path', help="Path to skip-list file")
parser.add_argument('--use-skip-list', action='store_true', default=False, help="Use skip list to skip tests if found")
parser.add_argument('--db-engine', help='Database engine name')
parser.add_argument('--replicated-database', action='store_true', default=False, help='Run tests with Replicated database engine')
parser.add_argument('--fast-tests-only', action='store_true', default=False, help='Run only fast tests (the tests without the "no-fasttest" tag)')
parser.add_argument('--no-stateless', action='store_true', help='Disable all stateless tests')
parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests')
parser.add_argument('--skip', nargs='+', help="Skip these tests")
@ -1358,12 +1313,6 @@ if __name__ == '__main__':
print("Using queries from '" + args.queries + "' directory")
if args.skip_list_path is None:
args.skip_list_path = os.path.join(args.queries, 'skip_list.json')
if args.sequential is None:
args.sequential = set([])
if args.tmp is None:
args.tmp = args.queries
if args.client is None:

View File

@ -112,12 +112,10 @@
"01153_attach_mv_uuid",
"01157_replace_table",
"01185_create_or_replace_table",
/// Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database.
"rocksdb",
"rocksdb", /// Sometimes cannot lock file most likely due to concurrent or adjacent tests, but we don't care how it works in Ordinary database
"01914_exchange_dictionaries" /// Requires Atomic database
],
"database-replicated": [
/// Unclassified
"memory_tracking",
"memory_usage",
"live_view",
@ -125,68 +123,67 @@
"01269_create_with_null",
"01451_replicated_detach_drop_and_quorum",
"01188_attach_table_from_path",
/// ON CLUSTER is not allowed
"01181_db_atomic_drop_on_cluster",
"01175_distributed_ddl_output_mode",
"01181_db_atomic_drop_on_cluster", /// ON CLUSTER is not allowed
"01175_distributed_ddl_output_mode", /// ON CLUSTER is not allowed
"01415_sticking_mutations",
/// user_files
"01721_engine_file_truncate_on_insert",
/// Fails due to additional replicas or shards
"quorum",
"01650_drop_part_and_deduplication_zookeeper",
"01532_execute_merges_on_single_replica",
"00652_replicated_mutations_default_database_zookeeper",
"00620_optimize_on_nonleader_replica_zookeeper",
"01158_zookeeper_log",
/// grep -c
"01018_ddl_dictionaries_bad_queries",
"00908_bloom_filter_index",
/// Unsupported type of ALTER query
"01650_fetch_patition_with_macro_in_zk_path",
"01451_detach_drop_part",
"01451_replicated_detach_drop_part",
"01417_freeze_partition_verbose",
"01417_freeze_partition_verbose_zookeeper",
"01130_in_memory_parts_partitons",
"01060_shutdown_table_after_detach",
"01021_only_tuple_columns",
"01015_attach_part",
"00955_test_final_mark",
"00753_alter_attach",
"00626_replace_partition_from_table_zookeeper",
"00626_replace_partition_from_table",
"00152_insert_different_granularity",
"00054_merge_tree_partitions",
"01781_merge_tree_deduplication",
"00980_zookeeper_merge_tree_alter_settings",
"00980_merge_alter_settings",
"02009_array_join_partition",
"02012_changed_enum_type_non_replicated",
"02012_zookeeper_changed_enum_type",
"02012_zookeeper_changed_enum_type_incompatible",
/// Old syntax is not allowed
"01062_alter_on_mutataion_zookeeper",
"00925_zookeeper_empty_replicated_merge_tree_optimize_final",
"00754_alter_modify_order_by_replicated_zookeeper",
"00652_replicated_mutations_zookeeper",
"00623_replicated_truncate_table_zookeeper",
"00516_deduplication_after_drop_partition_zookeeper",
"00446_clear_column_in_partition_concurrent_zookeeper",
"00236_replicated_drop_on_non_leader_zookeeper",
"00226_zookeeper_deduplication_and_unexpected_parts",
"00215_primary_key_order_zookeeper",
"00121_drop_column_zookeeper",
"00083_create_merge_tree_zookeeper",
"00062_replicated_merge_tree_alter_zookeeper",
/// Does not support renaming of multiple tables in single query
"00634_rename_view",
"00140_rename",
/// Different query_id
"01943_query_id_check",
/// Requires investigation
"00953_zookeeper_suetin_deduplication_bug",
"01783_http_chunk_size",
"00166_explain_estimate"
"01721_engine_file_truncate_on_insert", /// user_files
"quorum", /// Fails due to additional replicas or shards
"01650_drop_part_and_deduplication_zookeeper", /// Fails due to additional replicas or shards
"01532_execute_merges_on_single_replica", /// Fails due to additional replicas or shards
"00652_replicated_mutations_default_database_zookeeper", /// Fails due to additional replicas or shards
"00620_optimize_on_nonleader_replica_zookeeper", /// Fails due to additional replicas or shards
"01158_zookeeper_log", /// Fails due to additional replicas or shards
"01018_ddl_dictionaries_bad_queries", /// grep -c
"00908_bloom_filter_index", /// grep -c
"01650_fetch_patition_with_macro_in_zk_path", /// Unsupported type of ALTER query
"01451_detach_drop_part", /// Unsupported type of ALTER query
"01451_replicated_detach_drop_part", /// Unsupported type of ALTER query
"01417_freeze_partition_verbose", /// Unsupported type of ALTER query
"01417_freeze_partition_verbose_zookeeper", /// Unsupported type of ALTER query
"01130_in_memory_parts_partitons", /// Unsupported type of ALTER query
"01060_shutdown_table_after_detach", /// Unsupported type of ALTER query
"01021_only_tuple_columns", /// Unsupported type of ALTER query
"01015_attach_part", /// Unsupported type of ALTER query
"00955_test_final_mark", /// Unsupported type of ALTER query
"00753_alter_attach", /// Unsupported type of ALTER query
"00626_replace_partition_from_table_zookeeper", /// Unsupported type of ALTER query
"00626_replace_partition_from_table", /// Unsupported type of ALTER query
"00152_insert_different_granularity", /// Unsupported type of ALTER query
"00054_merge_tree_partitions", /// Unsupported type of ALTER query
"01781_merge_tree_deduplication", /// Unsupported type of ALTER query
"00980_zookeeper_merge_tree_alter_settings", /// Unsupported type of ALTER query
"00980_merge_alter_settings", /// Unsupported type of ALTER query
"02009_array_join_partition", /// Unsupported type of ALTER query
"02012_changed_enum_type_non_replicated", /// Unsupported type of ALTER query
"02012_zookeeper_changed_enum_type", /// Unsupported type of ALTER query
"02012_zookeeper_changed_enum_type_incompatible", /// Unsupported type of ALTER query
"01062_alter_on_mutataion_zookeeper", /// Old syntax is not allowed
"00925_zookeeper_empty_replicated_merge_tree_optimize_final", /// Old syntax is not allowed
"00754_alter_modify_order_by_replicated_zookeeper", /// Old syntax is not allowed
"00652_replicated_mutations_zookeeper", /// Old syntax is not allowed
"00623_replicated_truncate_table_zookeeper", /// Old syntax is not allowed
"00516_deduplication_after_drop_partition_zookeeper", /// Old syntax is not allowed
"00446_clear_column_in_partition_concurrent_zookeeper", /// Old syntax is not allowed
"00236_replicated_drop_on_non_leader_zookeeper", /// Old syntax is not allowed
"00226_zookeeper_deduplication_and_unexpected_parts", /// Old syntax is not allowed
"00215_primary_key_order_zookeeper", /// Old syntax is not allowed
"00121_drop_column_zookeeper", /// Old syntax is not allowed
"00083_create_merge_tree_zookeeper", /// Old syntax is not allowed
"00062_replicated_merge_tree_alter_zookeeper", /// Old syntax is not allowed
"00634_rename_view", /// Does not support renaming of multiple tables in single query
"00140_rename", /// Does not support renaming of multiple tables in single query
"01943_query_id_check", /// Different query_id
"00953_zookeeper_suetin_deduplication_bug", /// Requires investigation
"01783_http_chunk_size", /// Requires investigation
"00166_explain_estimate" /// Requires investigation
],
"polymorphic-parts": [
"01508_partition_pruning_long", /// bug, shoud be fixed

View File

@ -0,0 +1,429 @@
#!/usr/bin/python3
import os
import time
TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect']
TEST_FILE_EXTENSIONS += [ext + '.disabled' for ext in TEST_FILE_EXTENSIONS]
TAGS_PREFIX = "Tags:"
def parse_skip_list_json(text):
pos = 0
def skip_spaces():
nonlocal pos
start_pos = pos
while pos < len(text) and text[pos].isspace():
pos += 1
return pos > start_pos
def skip_comment():
nonlocal pos
if text.startswith('//', pos):
end_of_line = text.find('\n', pos + 1)
if end_of_line == -1:
pos = len(text)
return True
pos = end_of_line + 1
return True
elif text.startswith('/*', pos):
end_of_comment = text.find('*/', pos + 2)
if end_of_comment == -1:
raise Exception(f'Not found the end of a comment at pos {pos}')
pos = end_of_comment + 2
return True
else:
return False
def skip_spaces_or_comments():
while skip_spaces() or skip_comment():
pass
def skip_char_only(c):
nonlocal pos
if not text.startswith(c, pos):
return False
pos += 1
return True
def skip_char(c):
skip_spaces_or_comments()
return skip_char_only(c)
def expect_char(c):
nonlocal pos
skip_spaces_or_comments()
if not text.startswith(c, pos):
raise Exception(f"Expected '{c}' at pos {pos}")
pos += 1
def parse_quoted_string():
nonlocal pos
skip_spaces_or_comments()
expect_char('"')
end_of_string = text.find('"', pos)
if end_of_string == -1:
raise Exception(f'Not found the end of a quoted string at pos {pos-1}')
str = text[pos:end_of_string]
pos = end_of_string + 1
return str
def parse_description_comment():
nonlocal pos
while pos < len(text) and (text[pos] == ' ' or text[pos] == '\t'):
pos += 1
if not text.startswith('///', pos):
return None
end_of_line = text.find('\n', pos + 3)
if end_of_line == -1:
description = text[pos+3:]
pos = len(text)
else:
description = text[pos+3:end_of_line]
pos = end_of_line + 1
description = description.strip()
return description
res = {}
expect_char('{')
while not skip_char('}'):
build_flag = parse_quoted_string()
expect_char(':')
expect_char('[')
patterns = []
while not skip_char(']'):
pattern = parse_quoted_string()
skip_char_only(',')
description = parse_description_comment()
patterns.append((pattern, description))
skip_char(',')
res[build_flag] = patterns
return res
def load_skip_list_json(path):
if not os.path.exists(path):
raise Exception(f'File {path} not found')
with open(path, 'r') as file:
contents = file.read()
skip_dict = parse_skip_list_json(contents)
return skip_dict
def parse_fasttest_run_sh(text):
mark = 'TESTS_TO_SKIP=('
pos = text.find(mark)
if pos == -1:
raise Exception('TESTS_TO_SKIP not found in fasttest/run.sh')
pos += len(mark)
def skip_spaces():
nonlocal pos
start_pos = pos
while pos < len(text) and text[pos].isspace():
pos += 1
return pos > start_pos
def skip_comment():
nonlocal pos
if text.startswith('#', pos):
end_of_line = text.find('\n', pos + 1)
if end_of_line == -1:
pos = len(text)
return True
pos = end_of_line + 1
return True
else:
return False
def skip_spaces_or_comments():
while skip_spaces() or skip_comment():
pass
def skip_char(c):
nonlocal pos
skip_spaces_or_comments()
if not text.startswith(c, pos):
return False
pos += 1
return True
def parse_test_pattern():
nonlocal pos
skip_spaces_or_comments()
cur_pos = pos
while (cur_pos < len(text)) and (text[cur_pos].isalnum() or text[cur_pos] == '_'):
cur_pos += 1
if cur_pos == pos:
raise Exception(f"Couldn't read a test's name or pattern at pos {pos}")
pattern = text[pos:cur_pos]
pos = cur_pos
return pattern
def parse_description_comment():
nonlocal pos
while pos < len(text) and (text[pos] == ' ' or text[pos] == '\t'):
pos += 1
if not text.startswith('#', pos):
return None
end_of_line = text.find('\n', pos + 1)
if end_of_line == -1:
description = text[pos+1:]
pos = len(text)
else:
description = text[pos+1:end_of_line]
pos = end_of_line + 1
description = description.strip()
return description
patterns = []
while not skip_char(')'):
pattern = parse_test_pattern()
description = parse_description_comment()
patterns.append((pattern, description))
return {"fasttest": patterns}
def load_fasttest_run_sh(path):
if not os.path.exists(path):
raise Exception(f'File {path} not found')
with open(path, 'r') as file:
contents = file.read()
skip_dict = parse_fasttest_run_sh(contents)
return skip_dict
def get_comment_sign(filename):
if filename.endswith('.disabled'):
filename = filename[:-len('.disabled')]
if filename.endswith('.sql') or filename.endswith('.sql.j2'):
return '--'
elif filename.endswith('.sh') or filename.endswith('.py') or filename.endswith('.expect'):
return '#'
else:
raise Exception(f'Unknown file_extension: {filename}')
def is_shebang(line):
return line.startswith('#!')
def parse_tags_from_line(line, comment_sign):
if not line.startswith(comment_sign):
return None
tags_str = line[len(comment_sign):].lstrip()
if not tags_str.startswith(TAGS_PREFIX):
return None
tags_str = tags_str[len(TAGS_PREFIX):]
tags = tags_str.split(',')
tags = [tag.strip() for tag in tags]
return tags
def format_tags_to_line(tags, comment_sign):
return comment_sign + ' ' + TAGS_PREFIX + ' ' + ', '.join(tags) + '\n'
def load_tags_from_file(filepath):
with open(filepath, 'r') as file:
try:
line = file.readline()
if is_shebang(line):
line = file.readline()
except UnicodeDecodeError:
return []
return parse_tags_from_line(line, get_comment_sign(filepath))
def save_tags_to_file(filepath, tags, description = {}):
if not tags:
return
with open(filepath, 'r') as file:
contents = file.read()
# Skip shebang.
shebang_line = ''
eol = contents.find('\n')
if eol != -1 and is_shebang(contents[:eol+1]):
shebang_line = contents[:eol+1]
contents = contents[eol+1:]
# Skip tag line with description lines.
eol = contents.find('\n')
comment_sign = get_comment_sign(filepath)
if eol != -1 and parse_tags_from_line(contents[:eol+1], comment_sign):
contents = contents[eol+1:]
while True:
eol = contents.find('\n')
if eol == -1 or contents[:eol+1].isspace():
break
contents = contents[eol+1:]
# Skip an empty line after tags.
eol = contents.find('\n')
if eol != -1 and contents[:eol+1].isspace():
contents = contents[eol+1:]
empty_line = '\n'
# New tags line.
tags_line = format_tags_to_line(tags, comment_sign)
# New description lines.
tags_with_descriptions = []
for tag in tags:
if description.get(tag):
tags_with_descriptions.append(tag)
description_lines = ''
if tags_with_descriptions:
for tag in tags_with_descriptions:
description_lines += comment_sign + ' Tag ' + tag + ': ' + description[tag] + '\n'
contents = shebang_line + tags_line + description_lines + empty_line + contents
with open(filepath, 'w') as file:
file.write(contents)
print(f'Changed {filepath}')
def build_flag_to_tag(build_flag):
if build_flag == "thread-sanitizer":
return "no-tsan"
elif build_flag == "address-sanitizer":
return "no-asan"
elif build_flag == "ub-sanitizer":
return "no-ubsan"
elif build_flag == "memory-sanitizer":
return "no-msan"
elif build_flag == "database-replicated":
return "no-replicated-database"
elif build_flag == "database-ordinary":
return "no-ordinary-database"
elif build_flag == "debug-build":
return "no-debug"
elif build_flag == "release-build":
return "no-release"
elif build_flag == "unbundled-build":
return "no-unbundled"
else:
return "no-" + build_flag
def get_tags(test_name, skip_list):
tags = []
descriptions = {}
if test_name.endswith('.disabled'):
tags.append("disabled")
if 'deadlock' in test_name:
tags.append("deadlock")
elif 'race' in test_name:
tags.append("race")
elif 'long' in test_name:
tags.append("long")
if 'replica' in test_name:
tags.append("replica")
elif 'zookeeper' in test_name:
tags.append("zookeeper")
if 'distributed' in test_name:
tags.append("distributed")
elif 'shard' in test_name:
tags.append("shard")
elif 'global' in test_name:
tags.append("global")
for build_flag, patterns in skip_list.items():
for pattern, description in patterns:
if pattern in test_name:
tag = build_flag_to_tag(build_flag)
if not tag in tags:
tags.append(tag)
if description:
descriptions[tag] = description
return tags, descriptions
def load_all_tags(base_dir):
def get_comment_sign(filename):
if filename.endswith('.sql') or filename.endswith('.sql.j2'):
return '--'
elif filename.endswith('.sh') or filename.endswith('.py') or filename.endswith('.expect'):
return '#'
else:
raise Exception(f'Unknown file_extension: {filename}')
def parse_tags_from_line(line, comment_sign):
if not line.startswith(comment_sign):
return None
tags_str = line[len(comment_sign):].lstrip()
tags_prefix = "Tags:"
if not tags_str.startswith(tags_prefix):
return None
tags_str = tags_str[len(tags_prefix):]
tags = tags_str.split(',')
tags = {tag.strip() for tag in tags}
return tags
def is_shebang(line):
return line.startswith('#!')
def load_tags_from_file(filepath):
with open(filepath, 'r') as file:
try:
line = file.readline()
if is_shebang(line):
line = file.readline()
except UnicodeDecodeError:
return []
return parse_tags_from_line(line, get_comment_sign(filepath))
all_tags = {}
for suite in os.listdir(base_dir):
suite_dir = os.path.join(base_dir, suite)
if not os.path.isdir(suite_dir):
continue
for test_name in os.listdir(suite_dir):
test_path = os.path.join(suite_dir, test_name)
if not os.path.isfile(test_path) or all(not test_path.endswith(supported_ext) for supported_ext in TEST_FILE_EXTENSIONS):
continue
tags = load_tags_from_file(test_path)
if tags:
all_tags[test_path] = tags
return all_tags
def load_all_tags_timing(base_dir):
start_time = time.time()
print(type(load_all_tags(base_dir)))
print("--- %s seconds ---" % (time.time() - start_time))
def main():
base_dir = os.path.dirname(__file__)
print(f'base_dir={base_dir}')
#return load_all_tags_timing(base_dir)
skip_dict = load_skip_list_json(os.path.join(base_dir, 'skip_list.json'))
skip_dict_from_fasttest_run_sh = load_fasttest_run_sh(os.path.join(base_dir, '../../docker/test/fasttest/run.sh'))
skip_dict = {**skip_dict, **skip_dict_from_fasttest_run_sh}
#print(f'skip_dict={skip_dict}')
for suite in os.listdir(base_dir):
suite_dir = os.path.join(base_dir, suite)
if not os.path.isdir(suite_dir):
continue
for test_name in os.listdir(suite_dir):
test_path = os.path.join(suite_dir, test_name)
if not os.path.isfile(test_path) or all(not test_path.endswith(supported_ext) for supported_ext in TEST_FILE_EXTENSIONS):
continue
tags, descriptions = get_tags(test_path, skip_dict)
save_tags_to_file(test_path, tags, descriptions)
if __name__ == '__main__':
main()