mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
Update Pytest check (#18972)
* [WIP] * Update skip-list * Update ci_config.json * Do not sync inserts for test * Fix more tests * Fix another test * Enable one more test * More fixed tests * More test fixes * Do not absolutize server path for now * More test fixes * Unset CLICKHOUSE_LOG_COMMENT where necessary * Remove debugging set -e * Fix more tests * Fix test reference * Fix style check
This commit is contained in:
parent
ece7e87cfb
commit
be831d09f7
@ -986,7 +986,7 @@ void BaseDaemon::setupWatchdog()
|
||||
if (errno == ECHILD)
|
||||
{
|
||||
logger().information("Child process no longer exists.");
|
||||
_exit(status);
|
||||
_exit(WEXITSTATUS(status));
|
||||
}
|
||||
|
||||
if (WIFEXITED(status))
|
||||
@ -1020,7 +1020,7 @@ void BaseDaemon::setupWatchdog()
|
||||
|
||||
/// Automatic restart is not enabled but you can play with it.
|
||||
#if 1
|
||||
_exit(status);
|
||||
_exit(WEXITSTATUS(status));
|
||||
#else
|
||||
logger().information("Will restart.");
|
||||
if (argv0)
|
||||
|
@ -5,7 +5,10 @@ RUN apt-get update -y && \
|
||||
apt-get install -y --no-install-recommends \
|
||||
python3-pip \
|
||||
python3-setuptools \
|
||||
python3-wheel
|
||||
python3-wheel \
|
||||
brotli \
|
||||
netcat-openbsd \
|
||||
zstd
|
||||
|
||||
RUN python3 -m pip install \
|
||||
wheel \
|
||||
@ -15,7 +18,10 @@ RUN python3 -m pip install \
|
||||
pytest-randomly \
|
||||
pytest-rerunfailures \
|
||||
pytest-timeout \
|
||||
pytest-xdist
|
||||
pytest-xdist \
|
||||
pandas \
|
||||
numpy \
|
||||
scipy
|
||||
|
||||
CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
||||
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb; \
|
||||
|
@ -4,6 +4,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
if ! command -v gzip &> /dev/null; then echo "gzip not found" 1>&2; exit 1; fi
|
||||
if ! command -v brotli &> /dev/null; then echo "brotli not found" 1>&2; exit 1; fi
|
||||
if ! command -v xz &> /dev/null; then echo "xz not found" 1>&2; exit 1; fi
|
||||
if ! command -v zstd &> /dev/null; then echo "zstd not found" 1>&2; exit 1; fi
|
||||
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&enable_http_compression=0" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10';
|
||||
${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&enable_http_compression=1" -H 'Accept-Encoding: gzip' -d 'SELECT number FROM system.numbers LIMIT 10' | gzip -d;
|
||||
|
@ -1,6 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CLICKHOUSE_LOG_COMMENT=''
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,6 +1,7 @@
|
||||
|
||||
-- https://github.com/ClickHouse/ClickHouse/issues/1059
|
||||
|
||||
SET insert_distributed_sync = 1;
|
||||
|
||||
DROP TABLE IF EXISTS union1;
|
||||
DROP TABLE IF EXISTS union2;
|
||||
DROP TABLE IF EXISTS union3;
|
||||
|
@ -1,4 +1,6 @@
|
||||
set insert_distributed_sync = 1;
|
||||
set allow_suspicious_low_cardinality_types = 1;
|
||||
|
||||
DROP TABLE IF EXISTS test_low_null_float;
|
||||
DROP TABLE IF EXISTS dist_00717;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET insert_distributed_sync = 1;
|
||||
|
||||
DROP TABLE IF EXISTS low_cardinality;
|
||||
DROP TABLE IF EXISTS low_cardinality_all;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
set insert_distributed_sync=1;
|
||||
|
||||
DROP TABLE IF EXISTS dist_00967;
|
||||
DROP TABLE IF EXISTS underlying_00967;
|
||||
|
||||
@ -5,7 +7,6 @@ DROP TABLE IF EXISTS underlying_00967;
|
||||
SET send_logs_level='error';
|
||||
|
||||
CREATE TABLE dist_00967 (key UInt64) Engine=Distributed('test_shard_localhost', currentDatabase(), underlying_00967);
|
||||
-- fails for TinyLog()/MergeTree()/... but not for Memory()
|
||||
CREATE TABLE underlying_00967 (key Nullable(UInt64)) Engine=TinyLog();
|
||||
INSERT INTO dist_00967 SELECT toUInt64(number) FROM system.numbers LIMIT 1;
|
||||
|
||||
|
@ -1,5 +1,8 @@
|
||||
-- Create dictionary, since dictGet*() uses DB::Context in executeImpl()
|
||||
-- (To cover scope of the Context in DB::PushingToViewsBlockOutputStream::process)
|
||||
|
||||
set insert_distributed_sync=1;
|
||||
|
||||
DROP TABLE IF EXISTS mv;
|
||||
DROP DATABASE IF EXISTS dict_in_01023;
|
||||
CREATE DATABASE dict_in_01023;
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- from https://github.com/ClickHouse/ClickHouse/issues/5142
|
||||
|
||||
set insert_distributed_sync = 1;
|
||||
|
||||
DROP TABLE IF EXISTS t;
|
||||
DROP TABLE IF EXISTS t_d;
|
||||
DROP TABLE IF EXISTS t_v;
|
||||
|
@ -1,3 +1,5 @@
|
||||
-- set insert_distributed_sync = 1; -- see https://github.com/ClickHouse/ClickHouse/issues/18971
|
||||
|
||||
DROP TABLE IF EXISTS local_01099_a;
|
||||
DROP TABLE IF EXISTS local_01099_b;
|
||||
DROP TABLE IF EXISTS distributed_01099_a;
|
||||
|
@ -1,6 +1,6 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CLICKHOUSE_LOG_COMMENT=''
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -1,3 +1,4 @@
|
||||
set insert_distributed_sync = 1;
|
||||
SET allow_experimental_live_view = 1;
|
||||
|
||||
DROP TABLE IF EXISTS lv;
|
||||
@ -7,7 +8,7 @@ DROP TABLE IF EXISTS visits_layer;
|
||||
CREATE TABLE visits(StartDate Date) ENGINE MergeTree ORDER BY(StartDate);
|
||||
CREATE TABLE visits_layer(StartDate Date) ENGINE Distributed(test_cluster_two_shards_localhost, currentDatabase(), 'visits', rand());
|
||||
|
||||
CREATE LIVE VIEW lv AS SELECT * FROM visits_layer ORDER BY StartDate;
|
||||
CREATE LIVE VIEW lv AS SELECT * FROM visits_layer ORDER BY StartDate;
|
||||
|
||||
INSERT INTO visits_layer (StartDate) VALUES ('2020-01-01');
|
||||
INSERT INTO visits_layer (StartDate) VALUES ('2020-01-02');
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET allow_experimental_live_view = 1;
|
||||
SET insert_distributed_sync = 1;
|
||||
|
||||
DROP TABLE IF EXISTS lv;
|
||||
DROP TABLE IF EXISTS visits;
|
||||
|
@ -1,3 +1,5 @@
|
||||
set insert_distributed_sync = 1;
|
||||
|
||||
DROP TABLE IF EXISTS visits;
|
||||
DROP TABLE IF EXISTS visits_dist;
|
||||
|
||||
|
@ -1,7 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
set -ue
|
||||
|
||||
CLICKHOUSE_LOG_COMMENT=''
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -61,3 +61,8 @@ SELECT ProfileEvents.Values[indexOf(ProfileEvents.Names, 'FileOpen')]
|
||||
FROM system.query_log
|
||||
WHERE (type = 'QueryFinish') AND (lower(query) LIKE lower('SELECT m.% FROM %t_map%'))
|
||||
AND event_time > now() - INTERVAL 10 SECOND AND current_database = currentDatabase();
|
||||
|
||||
DROP TABLE t_arr;
|
||||
DROP TABLE t_nul;
|
||||
DROP TABLE t_tup;
|
||||
DROP TABLE t_map;
|
||||
|
@ -12,7 +12,7 @@ CREATE TABLE subcolumns
|
||||
arr2 Array(Array(Nullable(String))),
|
||||
lc LowCardinality(String),
|
||||
nested Nested(col1 String, col2 Nullable(UInt32))
|
||||
)
|
||||
)
|
||||
ENGINE = MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = '10M';
|
||||
|
||||
INSERT INTO subcolumns VALUES (([1, NULL], 2, 'a'), ['foo', NULL, 'bar'], [['123'], ['456', '789']], 'qqqq', ['zzz', 'xxx'], [42, 43]);
|
||||
@ -37,7 +37,7 @@ CREATE TABLE subcolumns
|
||||
arr2 Array(Array(Nullable(String))),
|
||||
lc LowCardinality(String),
|
||||
nested Nested(col1 String, col2 Nullable(UInt32))
|
||||
)
|
||||
)
|
||||
ENGINE = MergeTree order by tuple() SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
INSERT INTO subcolumns VALUES (([1, NULL], 2, 'a'), ['foo', NULL, 'bar'], [['123'], ['456', '789']], 'qqqq', ['zzz', 'xxx'], [42, 43]);
|
||||
@ -47,3 +47,5 @@ SELECT t.a.size0, t.a.null, t.u, t.s, t.s.null FROM subcolumns;
|
||||
SELECT sumArray(arr.null), sum(arr.size0) FROM subcolumns;
|
||||
SELECT arr2, arr2.size0, arr2.size1, arr2.null FROM subcolumns;
|
||||
-- SELECT nested.col1, nested.col2, nested.size0, nested.size0, nested.col2.null FROM subcolumns;
|
||||
|
||||
DROP TABLE subcolumns;
|
||||
|
@ -37,3 +37,5 @@ SELECT count() FROM map_subcolumns PREWHERE has(m.keys, 'b');
|
||||
|
||||
SELECT id, m.size0 FROM map_subcolumns;
|
||||
SELECT count() FROM map_subcolumns WHERE m.size0 > 2;
|
||||
|
||||
DROP TABLE map_subcolumns;
|
||||
|
@ -23,3 +23,5 @@ for engine in "${ENGINES[@]}"; do
|
||||
$CLICKHOUSE_CLIENT --query "SELECT * FROM subcolumns"
|
||||
$CLICKHOUSE_CLIENT --query "SELECT n, n.null, a1, a1.size0, a2, a2.size0, a2.size1, a2.size2, a3, a3.size0, a3.null, t, t.s, t.v, m, m.keys, m.values FROM subcolumns"
|
||||
done
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "DROP TABLE subcolumns"
|
||||
|
@ -1,7 +1,7 @@
|
||||
#!/usr/bin/env bash
|
||||
set -ue
|
||||
|
||||
CLICKHOUSE_LOG_COMMENT=''
|
||||
unset CLICKHOUSE_LOG_COMMENT
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
|
@ -8,7 +8,7 @@ CREATE TABLE nested
|
||||
col2 Nested(a UInt32, n Nested(s String, b UInt32)),
|
||||
col3 Nested(n1 Nested(a UInt32, b UInt32), n2 Nested(s String, t String))
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ENGINE = MergeTree
|
||||
ORDER BY tuple()
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
@ -55,7 +55,7 @@ CREATE TABLE nested
|
||||
id UInt32,
|
||||
col1 Nested(a UInt32, n Nested(s String, b UInt32))
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ENGINE = MergeTree
|
||||
ORDER BY id
|
||||
SETTINGS min_bytes_for_wide_part = 0;
|
||||
|
||||
@ -64,3 +64,5 @@ SELECT id % 10, sum(length(col1)), sumArray(arrayMap(x -> length(x), col1.n.b))
|
||||
|
||||
SELECT arraySum(col1.a), arrayMap(x -> x * x * 2, col1.a) FROM nested ORDER BY id LIMIT 5;
|
||||
SELECT untuple(arrayJoin(arrayJoin(col1.n))) FROM nested ORDER BY id LIMIT 10 OFFSET 10;
|
||||
|
||||
DROP TABLE nested;
|
||||
|
@ -4,6 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
yes http://foobarfoobarfoobarfoobarfoobarfoobarfoobar.com | head -c1G > 1g.csv
|
||||
yes http://foobarfoobarfoobarfoobarfoobarfoobarfoobar.com | head -c1G > ${CLICKHOUSE_TMP}/1g.csv
|
||||
|
||||
$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=100Mi -q "select count() from file('1g.csv', 'TSV', 'URL String')"
|
||||
$CLICKHOUSE_LOCAL --stacktrace --input_format_parallel_parsing=1 --max_memory_usage=100Mi -q "select count() from file('${CLICKHOUSE_TMP}/1g.csv', 'TSV', 'URL String')"
|
||||
|
@ -29,3 +29,5 @@ INSERT INTO t0(c0, c1) VALUES ("1",1) ; -- { clientError 47 }
|
||||
INSERT INTO t0(c0, c1) VALUES ('1', 1) ;
|
||||
-- the return code must be zero after the final query has failed with expected error
|
||||
insert into values_01564 values (11); -- { serverError 469 }
|
||||
drop table t0;
|
||||
drop table values_01564;
|
||||
|
@ -49,3 +49,6 @@ INSERT INTO t0(c0, c1) VALUES ('1', 1) ;
|
||||
|
||||
-- the return code must be zero after the final query has failed with expected error
|
||||
insert into values_01564 values (11); -- { serverError 469 }
|
||||
|
||||
drop table t0;
|
||||
drop table values_01564;
|
||||
|
@ -9,3 +9,5 @@ INSERT INTO low_card VALUES ( '1' );
|
||||
SELECT * FROM low_card;
|
||||
SELECT * FROM low_card WHERE lc = '1';
|
||||
SELECT CAST(lc AS String) FROM low_card;
|
||||
|
||||
DROP TABLE low_card;
|
||||
|
@ -6,6 +6,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
# Clone some not too large repository and create a database from it.
|
||||
|
||||
cd $CLICKHOUSE_TMP || exit
|
||||
|
||||
# Protection for network errors
|
||||
for _ in {1..10}; do
|
||||
rm -rf ./clickhouse-odbc
|
||||
|
@ -7,27 +7,18 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
set -e
|
||||
|
||||
TEMP_DIR="$(mktemp -d /tmp/clickhouse.test..XXXXXX)"
|
||||
cd "${TEMP_DIR:?}"
|
||||
|
||||
function cleanup()
|
||||
{
|
||||
rm -fr "${TEMP_DIR:?}"
|
||||
}
|
||||
trap cleanup EXIT
|
||||
|
||||
# This is random garbage, so compression ratio will be very low.
|
||||
tr -cd 'a-z0-9' < /dev/urandom | head -c1M > input
|
||||
tr -cd 'a-z0-9' < /dev/urandom | head -c1M > ${CLICKHOUSE_TMP}/input
|
||||
|
||||
# stdin/stdout streams
|
||||
$CLICKHOUSE_COMPRESSOR < input > output
|
||||
diff -q <($CLICKHOUSE_COMPRESSOR --decompress < output) input
|
||||
$CLICKHOUSE_COMPRESSOR < ${CLICKHOUSE_TMP}/input > ${CLICKHOUSE_TMP}/output
|
||||
diff -q <($CLICKHOUSE_COMPRESSOR --decompress < ${CLICKHOUSE_TMP}/output) ${CLICKHOUSE_TMP}/input
|
||||
|
||||
# positional arguments, and that fact that input/output will be overwritten
|
||||
$CLICKHOUSE_COMPRESSOR input output
|
||||
diff -q <($CLICKHOUSE_COMPRESSOR --decompress output) input
|
||||
$CLICKHOUSE_COMPRESSOR ${CLICKHOUSE_TMP}/input ${CLICKHOUSE_TMP}/output
|
||||
diff -q <($CLICKHOUSE_COMPRESSOR --decompress ${CLICKHOUSE_TMP}/output) ${CLICKHOUSE_TMP}/input
|
||||
|
||||
# --offset-in-decompressed-block
|
||||
diff -q <($CLICKHOUSE_COMPRESSOR --decompress --offset-in-decompressed-block 10 output) <(tail -c+$((10+1)) input)
|
||||
diff -q <($CLICKHOUSE_COMPRESSOR --decompress --offset-in-decompressed-block 10 ${CLICKHOUSE_TMP}/output) <(tail -c+$((10+1)) ${CLICKHOUSE_TMP}/input)
|
||||
|
||||
# TODO: --offset-in-compressed-file using some .bin file (via clickhouse-local + check-marks)
|
||||
|
@ -7,12 +7,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
|
||||
PORT="$(($RANDOM%63000+2001))"
|
||||
|
||||
TEMP_FILE="$CURDIR/01622_defaults_for_url_engine.tmp"
|
||||
TEMP_FILE="${CLICKHOUSE_TMP}/01622_defaults_for_url_engine.tmp"
|
||||
|
||||
function thread1
|
||||
{
|
||||
while true; do
|
||||
echo -e "HTTP/1.1 200 OK\n\n{\"a\": 1}" | nc -l -p $1 -q 1;
|
||||
while true; do
|
||||
echo -e "HTTP/1.1 200 OK\n\n{\"a\": 1}" | nc -l -p $1 -q 1;
|
||||
done
|
||||
}
|
||||
|
||||
|
@ -4,3 +4,5 @@ SELECT ignore(10, ignore(*), ignore(ignore(-2, 1025, *)), NULL, *), * FROM lc_nu
|
||||
|
||||
|
||||
SELECT ignore(toLowCardinality(1), toLowCardinality(2), 3);
|
||||
|
||||
DROP TABLE lc_null_int8_defnull;
|
||||
|
@ -56,3 +56,5 @@ WHERE current_database = currentDatabase() AND type == 'QueryFinish' AND (query
|
||||
ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames;
|
||||
SELECT '';
|
||||
|
||||
DROP TABLE test_query_log_factories_info1.memory_table;
|
||||
DROP DATABASE test_query_log_factories_info1;
|
||||
|
@ -4,3 +4,5 @@ INSERT INTO data_01655 VALUES (1);
|
||||
SELECT * FROM data_01655 SETTINGS merge_tree_min_rows_for_concurrent_read=0, merge_tree_min_bytes_for_concurrent_read=0;
|
||||
-- UINT64_MAX
|
||||
SELECT * FROM data_01655 SETTINGS merge_tree_min_rows_for_concurrent_read=18446744073709551615, merge_tree_min_bytes_for_concurrent_read=18446744073709551615;
|
||||
|
||||
DROP TABLE data_01655;
|
||||
|
@ -25,6 +25,9 @@ def bin_prefix(cmdopts):
|
||||
prefix = 'clickhouse'
|
||||
if cmdopts['builddir'] is not None:
|
||||
prefix = os.path.join(cmdopts['builddir'], 'programs', prefix)
|
||||
# FIXME: does this hangs the server start for some reason?
|
||||
# if not os.path.isabs(prefix):
|
||||
# prefix = os.path.abspath(prefix)
|
||||
return prefix
|
||||
|
||||
|
||||
|
@ -14,13 +14,10 @@ SKIP_LIST = [
|
||||
"00987_distributed_stack_overflow",
|
||||
|
||||
# just fail
|
||||
"00302_http_compression",
|
||||
"00463_long_sessions_in_http_interface",
|
||||
"00505_secure",
|
||||
"00505_shard_secure",
|
||||
"00506_union_distributed", # flaky
|
||||
"00646_url_engine",
|
||||
"00821_distributed_storage_with_join_on.sql", # flaky
|
||||
"00725_memory_tracking", # BROKEN
|
||||
"00834_cancel_http_readonly_queries_on_client_close",
|
||||
"00933_test_fix_extra_seek_on_compressed_cache",
|
||||
"00965_logs_level_bugfix",
|
||||
@ -30,10 +27,6 @@ SKIP_LIST = [
|
||||
"01014_lazy_database_concurrent_recreate_reattach_and_show_tables",
|
||||
"01018_Distributed__shard_num",
|
||||
"01018_ip_dictionary",
|
||||
"01023_materialized_view_query_context", # flaky
|
||||
"01035_lc_empty_part_bug", # flaky
|
||||
"01037_polygon_dicts_simple_functions.sh", # flaky
|
||||
"01046_materialized_view_with_join_over_distributed", # flaky
|
||||
"01050_clickhouse_dict_source_with_subquery",
|
||||
"01053_ssd_dictionary",
|
||||
"01054_cache_dictionary_overflow_cell",
|
||||
@ -43,25 +36,25 @@ SKIP_LIST = [
|
||||
"01086_odbc_roundtrip",
|
||||
"01088_benchmark_query_id",
|
||||
"01098_temporary_and_external_tables",
|
||||
"01099_parallel_distributed_insert_select", # flaky
|
||||
"01099_parallel_distributed_insert_select",
|
||||
"01103_check_cpu_instructions_at_startup",
|
||||
"01114_database_atomic",
|
||||
"01148_zookeeper_path_macros_unfolding",
|
||||
"01193_metadata_loading.sh", # flaky
|
||||
"01274_alter_rename_column_distributed", # flaky
|
||||
"01181_db_atomic_drop_on_cluster", # tcp port in reference
|
||||
"01280_ssd_complex_key_dictionary",
|
||||
"01293_client_interactive_vertical_multiline", # expect-test
|
||||
"01293_client_interactive_vertical_singleline", # expect-test
|
||||
"01293_system_distribution_queue", # FLAKY
|
||||
"01293_show_clusters",
|
||||
"01294_lazy_database_concurrent_recreate_reattach_and_show_tables",
|
||||
"01294_system_distributed_on_cluster",
|
||||
"01300_client_save_history_when_terminated", # expect-test
|
||||
"01304_direct_io",
|
||||
"01306_benchmark_json",
|
||||
"01035_lc_empty_part_bug", # FLAKY
|
||||
"01320_create_sync_race_condition_zookeeper",
|
||||
"01355_CSV_input_format_allow_errors",
|
||||
"01370_client_autocomplete_word_break_characters", # expect-test
|
||||
"01375_storage_file_tsv_csv_with_names_write_prefix", # flaky
|
||||
"01376_GROUP_BY_injective_elimination_dictGet",
|
||||
"01393_benchmark_secure_port",
|
||||
"01418_custom_settings",
|
||||
@ -72,6 +65,7 @@ SKIP_LIST = [
|
||||
"01507_clickhouse_server_start_with_embedded_config",
|
||||
"01514_distributed_cancel_query_on_error",
|
||||
"01520_client_print_query_id", # expect-test
|
||||
"01526_client_start_and_exit", # expect-test
|
||||
"01527_dist_sharding_key_dictGet_reload",
|
||||
"01545_url_file_format_settings",
|
||||
"01553_datetime64_comparison",
|
||||
@ -79,17 +73,18 @@ SKIP_LIST = [
|
||||
"01558_ttest_scipy",
|
||||
"01561_mann_whitney_scipy",
|
||||
"01582_distinct_optimization",
|
||||
"01586_storage_join_low_cardinality_key",
|
||||
"01599_multiline_input_and_singleline_comments",
|
||||
"01600_benchmark_query",
|
||||
"01599_multiline_input_and_singleline_comments", # expect-test
|
||||
"01601_custom_tld",
|
||||
"01601_proxy_protocol",
|
||||
"01610_client_spawn_editor", # expect-test
|
||||
"01676_clickhouse_client_autocomplete", # expect-test (partially)
|
||||
"01683_text_log_deadlock", # secure tcp
|
||||
]
|
||||
|
||||
|
||||
def check_result(result, error, return_code, reference, replace_map):
|
||||
for old, new in replace_map.items():
|
||||
result = result.replace(old.encode('utf-8'), new.encode('utf-8'))
|
||||
if replace_map:
|
||||
for old, new in replace_map.items():
|
||||
result = result.replace(old.encode('utf-8'), new.encode('utf-8'))
|
||||
|
||||
if return_code != 0:
|
||||
try:
|
||||
@ -106,9 +101,9 @@ def check_result(result, error, return_code, reference, replace_map):
|
||||
pytrace=False)
|
||||
|
||||
|
||||
def run_client(bin_prefix, port, query, reference, replace_map={}):
|
||||
def run_client(bin_prefix, port, database, query, reference, replace_map=None):
|
||||
# We can't use `text=True` since some tests may return binary data
|
||||
client = subprocess.Popen([bin_prefix + '-client', '--port', str(port), '-m', '-n', '--testmode'],
|
||||
client = subprocess.Popen([bin_prefix + '-client', '--port', str(port), '-d', database, '-m', '-n', '--testmode'],
|
||||
stdin=subprocess.PIPE, stdout=subprocess.PIPE, stderr=subprocess.PIPE)
|
||||
result, error = client.communicate(query.encode('utf-8'))
|
||||
assert client.returncode is not None, "Client should exit after processing all queries"
|
||||
@ -116,12 +111,13 @@ def run_client(bin_prefix, port, query, reference, replace_map={}):
|
||||
check_result(result, error, client.returncode, reference, replace_map)
|
||||
|
||||
|
||||
def run_shell(bin_prefix, server, database, path, reference, replace_map={}):
|
||||
def run_shell(bin_prefix, server, database, path, reference, replace_map=None):
|
||||
env = {
|
||||
'CLICKHOUSE_BINARY': bin_prefix,
|
||||
'CLICKHOUSE_DATABASE': database,
|
||||
'CLICKHOUSE_PORT_TCP': str(server.tcp_port),
|
||||
'CLICKHOUSE_PORT_TCP_SECURE': str(server.tcps_port),
|
||||
'CLICKHOUSE_PORT_TCP_WITH_PROXY': str(server.proxy_port),
|
||||
'CLICKHOUSE_PORT_HTTP': str(server.http_port),
|
||||
'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port),
|
||||
'CLICKHOUSE_TMP': server.tmp_dir,
|
||||
@ -136,6 +132,7 @@ def run_shell(bin_prefix, server, database, path, reference, replace_map={}):
|
||||
|
||||
def random_str(length=10):
|
||||
alphabet = string.ascii_lowercase + string.digits
|
||||
random.seed(os.urandom(8))
|
||||
return ''.join(random.choice(alphabet) for _ in range(length))
|
||||
|
||||
|
||||
@ -159,17 +156,18 @@ def test_sql_query(bin_prefix, sql_query, standalone_server):
|
||||
reference = file.read()
|
||||
|
||||
random_name = 'test_{random}'.format(random=random_str())
|
||||
query = 'CREATE DATABASE {random}; USE {random}; {query}'.format(random=random_name, query=query)
|
||||
run_client(bin_prefix, tcp_port, query, reference, {random_name: 'default'})
|
||||
run_client(bin_prefix, tcp_port, 'default', 'CREATE DATABASE {random};'.format(random=random_name), b'')
|
||||
|
||||
run_client(bin_prefix, tcp_port, random_name, query, reference, {random_name: 'default'})
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
|
||||
run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED TABLES\n')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
|
||||
query = 'DROP DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, query, b'')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
|
||||
run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
|
||||
|
||||
def test_shell_query(bin_prefix, shell_query, standalone_server):
|
||||
@ -191,15 +189,15 @@ def test_shell_query(bin_prefix, shell_query, standalone_server):
|
||||
|
||||
random_name = 'test_{random}'.format(random=random_str())
|
||||
query = 'CREATE DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, query, b'')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
run_shell(bin_prefix, standalone_server, random_name, shell_path, reference, {random_name: 'default'})
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED TABLES'; SELECT name FROM system.tables WHERE database != 'system' ORDER BY (database, name);"
|
||||
run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED TABLES\n')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED TABLES\n')
|
||||
|
||||
query = 'DROP DATABASE {random};'.format(random=random_name)
|
||||
run_client(bin_prefix, tcp_port, query, b'')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'')
|
||||
|
||||
query = "SELECT 'SHOW ORPHANED DATABASES'; SHOW DATABASES;"
|
||||
run_client(bin_prefix, tcp_port, query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
run_client(bin_prefix, tcp_port, 'default', query, b'SHOW ORPHANED DATABASES\ndefault\nsystem\n')
|
||||
|
@ -37,6 +37,7 @@ class ServerThread(threading.Thread):
|
||||
self.tcps_port = port_base + 4
|
||||
self.https_port = port_base + 5
|
||||
self.odbc_port = port_base + 6
|
||||
self.proxy_port = port_base + 7
|
||||
|
||||
self._args = [
|
||||
'--config-file={config_path}'.format(config_path=self.server_config),
|
||||
@ -44,6 +45,7 @@ class ServerThread(threading.Thread):
|
||||
'--tcp_port={tcp_port}'.format(tcp_port=self.tcp_port),
|
||||
'--http_port={http_port}'.format(http_port=self.http_port),
|
||||
'--interserver_http_port={inter_port}'.format(inter_port=self.inter_port),
|
||||
'--tcp_with_proxy_port={proxy_port}'.format(proxy_port=self.proxy_port),
|
||||
# TODO: SSL certificate is not specified '--tcp_port_secure={tcps_port}'.format(tcps_port=self.tcps_port),
|
||||
]
|
||||
|
||||
@ -76,8 +78,8 @@ class ServerThread(threading.Thread):
|
||||
print('Successful server response:', s.recv(1024)) # FIXME: read whole buffered response
|
||||
s.shutdown(socket.SHUT_RDWR)
|
||||
s.close()
|
||||
except Exception as e:
|
||||
print('Failed to connect to server:', e, file=sys.stderr)
|
||||
except Exception:
|
||||
# Failed to connect to server - try again
|
||||
continue
|
||||
else:
|
||||
break
|
||||
@ -96,6 +98,10 @@ class ServerThread(threading.Thread):
|
||||
|
||||
self._lock.release()
|
||||
|
||||
if not retries:
|
||||
print('Failed to start server', file=sys.stderr)
|
||||
return
|
||||
|
||||
while self._proc.returncode is None:
|
||||
self._proc.communicate()
|
||||
|
||||
@ -297,6 +303,10 @@ ServerThread.DEFAULT_SERVER_CONFIG = \
|
||||
<implementation>testkeeper</implementation>
|
||||
</zookeeper>
|
||||
|
||||
<distributed_ddl>
|
||||
<path>/clickhouse/task_queue/ddl</path>
|
||||
</distributed_ddl>
|
||||
|
||||
<part_log>
|
||||
<database>system</database>
|
||||
<table>part_log</table>
|
||||
@ -1112,6 +1122,136 @@ ServerThread.DEFAULT_DICTIONARIES_CONFIG = \
|
||||
</attribute>
|
||||
</structure>
|
||||
</dictionary>
|
||||
|
||||
<dictionary>
|
||||
<name>simple_executable_cache_dictionary_no_implicit_key</name>
|
||||
<structure>
|
||||
<id>
|
||||
<name>id</name>
|
||||
<type>UInt64</type>
|
||||
</id>
|
||||
|
||||
<attribute>
|
||||
<name>value</name>
|
||||
<type>String</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
</structure>
|
||||
<source>
|
||||
<executable>
|
||||
<command>echo "1\tValue"</command>
|
||||
<format>TabSeparated</format>
|
||||
<implicit_key>false</implicit_key>
|
||||
</executable>
|
||||
</source>
|
||||
<layout>
|
||||
<cache>
|
||||
<size_in_cells>10000</size_in_cells>
|
||||
</cache>
|
||||
</layout>
|
||||
<lifetime>300</lifetime>
|
||||
</dictionary>
|
||||
|
||||
<dictionary>
|
||||
<name>simple_executable_cache_dictionary_implicit_key</name>
|
||||
<structure>
|
||||
<id>
|
||||
<name>id</name>
|
||||
<type>UInt64</type>
|
||||
</id>
|
||||
|
||||
<attribute>
|
||||
<name>value</name>
|
||||
<type>String</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
</structure>
|
||||
<source>
|
||||
<executable>
|
||||
<command>echo "Value"</command>
|
||||
<format>TabSeparated</format>
|
||||
<implicit_key>true</implicit_key>
|
||||
</executable>
|
||||
</source>
|
||||
<layout>
|
||||
<cache>
|
||||
<size_in_cells>10000</size_in_cells>
|
||||
</cache>
|
||||
</layout>
|
||||
<lifetime>300</lifetime>
|
||||
</dictionary>
|
||||
|
||||
<dictionary>
|
||||
<name>complex_executable_cache_dictionary_no_implicit_key</name>
|
||||
<structure>
|
||||
<key>
|
||||
<attribute>
|
||||
<name>id</name>
|
||||
<type>UInt64</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
<attribute>
|
||||
<name>id_key</name>
|
||||
<type>String</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
</key>
|
||||
<attribute>
|
||||
<name>value</name>
|
||||
<type>String</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
</structure>
|
||||
<source>
|
||||
<executable>
|
||||
<command>echo "1\tFirstKey\tValue"</command>
|
||||
<format>TabSeparated</format>
|
||||
<implicit_key>false</implicit_key>
|
||||
</executable>
|
||||
</source>
|
||||
<layout>
|
||||
<complex_key_cache>
|
||||
<size_in_cells>10000</size_in_cells>
|
||||
</complex_key_cache>
|
||||
</layout>
|
||||
<lifetime>300</lifetime>
|
||||
</dictionary>
|
||||
|
||||
<dictionary>
|
||||
<name>complex_executable_cache_dictionary_implicit_key</name>
|
||||
<structure>
|
||||
<key>
|
||||
<attribute>
|
||||
<name>id</name>
|
||||
<type>UInt64</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
<attribute>
|
||||
<name>id_key</name>
|
||||
<type>String</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
</key>
|
||||
<attribute>
|
||||
<name>value</name>
|
||||
<type>String</type>
|
||||
<null_value></null_value>
|
||||
</attribute>
|
||||
</structure>
|
||||
<source>
|
||||
<executable>
|
||||
<command>echo "Value"</command>
|
||||
<format>TabSeparated</format>
|
||||
<implicit_key>true</implicit_key>
|
||||
</executable>
|
||||
</source>
|
||||
<layout>
|
||||
<complex_key_cache>
|
||||
<size_in_cells>10000</size_in_cells>
|
||||
</complex_key_cache>
|
||||
</layout>
|
||||
<lifetime>300</lifetime>
|
||||
</dictionary>
|
||||
</yandex>
|
||||
"""
|
||||
|
||||
|
@ -8,11 +8,12 @@ export CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL
|
||||
[ -v CLICKHOUSE_CONFIG_CLIENT ] && CLICKHOUSE_CLIENT_OPT0+=" --config-file=${CLICKHOUSE_CONFIG_CLIENT} "
|
||||
[ -v CLICKHOUSE_HOST ] && CLICKHOUSE_CLIENT_OPT0+=" --host=${CLICKHOUSE_HOST} "
|
||||
[ -v CLICKHOUSE_PORT_TCP ] && CLICKHOUSE_CLIENT_OPT0+=" --port=${CLICKHOUSE_PORT_TCP} "
|
||||
[ -v CLICKHOUSE_PORT_TCP ] && CLICKHOUSE_BENCHMARK_OPT0+=" --port=${CLICKHOUSE_PORT_TCP} "
|
||||
[ -v CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL ] && CLICKHOUSE_CLIENT_OPT0+=" --send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL} "
|
||||
[ -v CLICKHOUSE_DATABASE ] && CLICKHOUSE_CLIENT_OPT0+=" --database=${CLICKHOUSE_DATABASE} "
|
||||
[ -n "$CLICKHOUSE_LOG_COMMENT" ] && CLICKHOUSE_CLIENT_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' "
|
||||
[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_CLIENT_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' "
|
||||
[ -v CLICKHOUSE_DATABASE ] && CLICKHOUSE_BENCHMARK_OPT0+=" --database=${CLICKHOUSE_DATABASE} "
|
||||
[ -n "$CLICKHOUSE_LOG_COMMENT" ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' "
|
||||
[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' "
|
||||
|
||||
export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"}
|
||||
[ -x "$CLICKHOUSE_BINARY-client" ] && CLICKHOUSE_CLIENT_BINARY=${CLICKHOUSE_CLIENT_BINARY:=$CLICKHOUSE_BINARY-client}
|
||||
@ -62,7 +63,7 @@ else
|
||||
export CLICKHOUSE_URL_PARAMS="database=${CLICKHOUSE_DATABASE}"
|
||||
fi
|
||||
# Note: missing url encoding of the log comment.
|
||||
[ -n "$CLICKHOUSE_LOG_COMMENT" ] && export CLICKHOUSE_URL_PARAMS="${CLICKHOUSE_URL_PARAMS}&log_comment=${CLICKHOUSE_LOG_COMMENT}"
|
||||
[ -v CLICKHOUSE_LOG_COMMENT ] && export CLICKHOUSE_URL_PARAMS="${CLICKHOUSE_URL_PARAMS}&log_comment=${CLICKHOUSE_LOG_COMMENT}"
|
||||
|
||||
export CLICKHOUSE_URL=${CLICKHOUSE_URL:="${CLICKHOUSE_PORT_HTTP_PROTO}://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTP}/"}
|
||||
export CLICKHOUSE_URL_HTTPS=${CLICKHOUSE_URL_HTTPS:="https://${CLICKHOUSE_HOST}:${CLICKHOUSE_PORT_HTTPS}/"}
|
||||
|
Loading…
Reference in New Issue
Block a user