diff --git a/contrib/avro b/contrib/avro index 1ee16d8c5a7..e43c46e87fd 160000 --- a/contrib/avro +++ b/contrib/avro @@ -1 +1 @@ -Subproject commit 1ee16d8c5a7808acff5cf0475f771195d9aa3faa +Subproject commit e43c46e87fd32eafdc09471e95344555454c5ef8 diff --git a/tests/queries/.gitignore b/tests/queries/.gitignore deleted file mode 100644 index df2c6d60133..00000000000 --- a/tests/queries/.gitignore +++ /dev/null @@ -1,2 +0,0 @@ -*.stderr -*.stdout diff --git a/tests/queries/0_stateless/01710_aggregate_projections.sh b/tests/queries/0_stateless/01710_aggregate_projections.sh index 1b09a3f1383..a8b3e6bf99d 100755 --- a/tests/queries/0_stateless/01710_aggregate_projections.sh +++ b/tests/queries/0_stateless/01710_aggregate_projections.sh @@ -25,3 +25,5 @@ $CLICKHOUSE_CLIENT -q "select x + y + 1, argMax(y, x), sum(x - y) as s from test $CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj prewhere (x + y) % 2 = 1 group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1" $CLICKHOUSE_CLIENT -q "select x + y, sum(x - y) as s from test_agg_proj prewhere (x + y) % 2 = 1 group by x + y order by s desc limit 5 settings allow_experimental_projection_optimization=1 format JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT -q "drop table test_agg_proj" diff --git a/tests/queries/0_stateless/01710_normal_projections.sh b/tests/queries/0_stateless/01710_normal_projections.sh index 89795d12bc9..6b16021de17 100755 --- a/tests/queries/0_stateless/01710_normal_projections.sh +++ b/tests/queries/0_stateless/01710_normal_projections.sh @@ -77,3 +77,4 @@ echo "optimize_move_to_prewhere = 1, allow_experimental_projection_optimization $CLICKHOUSE_CLIENT -q "SELECT * FROM test_sort_proj WHERE y > 4294967286 order by x FORMAT JSON SETTINGS optimize_move_to_prewhere = 1, allow_experimental_projection_optimization = 1" | grep rows_read +$CLICKHOUSE_CLIENT -q "DROP TABLE test_sort_proj" diff --git a/tests/queries/0_stateless/01785_pmj_lc_bug.sql b/tests/queries/0_stateless/01785_pmj_lc_bug.sql index 722faa9b40d..3020692c80a 100644 --- a/tests/queries/0_stateless/01785_pmj_lc_bug.sql +++ b/tests/queries/0_stateless/01785_pmj_lc_bug.sql @@ -12,3 +12,6 @@ SELECT 1025 == count(n) FROM foo AS t1 ANY LEFT JOIN foo_lc AS t2 ON t1.n == t2. SELECT 1025 == count(n) FROM foo_lc AS t1 ANY LEFT JOIN foo AS t2 ON t1.n == t2.n; SELECT 1025 == count(n) FROM foo_lc AS t1 ALL LEFT JOIN foo_lc AS t2 ON t1.n == t2.n; + +DROP TABLE foo; +DROP TABLE foo_lc; diff --git a/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh b/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh index edd73131020..4b3e35047bc 100755 --- a/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh +++ b/tests/queries/0_stateless/01802_test_postgresql_protocol_with_row_policy.sh @@ -41,3 +41,5 @@ SELECT 'after row policy with plaintext_password'; psql "postgresql://postgresql_user:qwerty@localhost:${CLICKHOUSE_PORT_POSTGRESQL}/db01802" -c "SELECT * FROM postgresql;" +$CLICKHOUSE_CLIENT -q "DROP TABLE db01802.postgresql" +$CLICKHOUSE_CLIENT -q "DROP DATABASE db01802" diff --git a/tests/queries/0_stateless/01832_memory_write_suffix.sql b/tests/queries/0_stateless/01832_memory_write_suffix.sql index 718a4e4ac9d..274736c5c06 100644 --- a/tests/queries/0_stateless/01832_memory_write_suffix.sql +++ b/tests/queries/0_stateless/01832_memory_write_suffix.sql @@ -5,3 +5,5 @@ drop table if exists data_01832; create table data_01832 (key Int) Engine=Memory; insert into data_01832 values (1); select * from data_01832; + +drop table data_01832; diff --git a/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql b/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql index 97d96f643cf..bcc26491ef2 100644 --- a/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql +++ b/tests/queries/0_stateless/01838_system_dictionaries_virtual_key_column.sql @@ -24,3 +24,6 @@ LAYOUT(COMPLEX_KEY_DIRECT()); SELECT 'complex key'; SELECT name, key FROM system.dictionaries WHERE name='example_complex_key_dictionary' AND database=currentDatabase(); + +DROP DICTIONARY example_complex_key_dictionary; +DROP DICTIONARY example_simple_key_dictionary; diff --git a/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql b/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql index d6f3b8a2136..44a1537f720 100644 --- a/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql +++ b/tests/queries/0_stateless/01846_null_as_default_for_insert_select.sql @@ -28,3 +28,5 @@ CREATE TABLE test_null_as_default (a Int8, b Int64 DEFAULT c - 500, c Int32 DEFA INSERT INTO test_null_as_default(a, c) SELECT 1, NULL UNION ALL SELECT 2, NULL; SELECT * FROM test_null_as_default ORDER BY a; + +DROP TABLE test_null_as_default; diff --git a/tests/queries/0_stateless/01848_http_insert_segfault.sh b/tests/queries/0_stateless/01848_http_insert_segfault.sh index a263ded44eb..c766e9794ae 100755 --- a/tests/queries/0_stateless/01848_http_insert_segfault.sh +++ b/tests/queries/0_stateless/01848_http_insert_segfault.sh @@ -4,5 +4,4 @@ # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh - ${CLICKHOUSE_LOCAL} -q "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | curl -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||: - + ${CLICKHOUSE_LOCAL} -q "select col1, initializeAggregation('argMaxState', col2, insertTime) as col2, now() as insertTime FROM generateRandom('col1 String, col2 Array(Float64)') LIMIT 1000000 FORMAT CSV" | ${CLICKHOUSE_CURL} -s 'http://localhost:8123/?query=INSERT%20INTO%20non_existing_table%20SELECT%20col1%2C%20initializeAggregation(%27argMaxState%27%2C%20col2%2C%20insertTime)%20as%20col2%2C%20now()%20as%20insertTime%20FROM%20input(%27col1%20String%2C%20col2%20Array(Float64)%27)%20FORMAT%20CSV' --data-binary @- | grep -q "Table default.non_existing_table doesn't exist" && echo 'Ok.' || echo 'FAIL' ||: diff --git a/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql b/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql index 6ed4eafcc8f..ab4c63ccdf2 100644 --- a/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql +++ b/tests/queries/0_stateless/01850_dist_INSERT_preserve_error.sql @@ -13,3 +13,6 @@ insert into dist_01850 values (1); -- { serverError 60 } drop table if exists dist_01850; drop table shard_0.data_01850; + +drop database shard_0; +drop database shard_1; diff --git a/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql b/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql index 2501e21b95e..a51309b3ffd 100644 --- a/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql +++ b/tests/queries/0_stateless/01852_dictionary_found_rate_long.sql @@ -307,3 +307,4 @@ SELECT name, found_rate FROM system.dictionaries WHERE database = currentDatabas DROP TABLE polygons_01862; DROP TABLE points_01862; +DROP DICTIONARY polygon_dictionary_01862; diff --git a/tests/queries/0_stateless/01852_jit_if.sql b/tests/queries/0_stateless/01852_jit_if.sql index cdfeba5b895..9a9d74274c2 100644 --- a/tests/queries/0_stateless/01852_jit_if.sql +++ b/tests/queries/0_stateless/01852_jit_if.sql @@ -14,3 +14,6 @@ INSERT INTO test_jit_nullable VALUES (0), (1), (NULL); SELECT 'test_jit_nullable'; SELECT value, multiIf(value = 1, 2, value, 1, 0), if (value, 1, 0) FROM test_jit_nullable; + +DROP TABLE test_jit_nonnull; +DROP TABLE test_jit_nullable; diff --git a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh index a585785d710..409f8d5ecf5 100755 --- a/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh +++ b/tests/queries/0_stateless/01853_dictionary_cache_duplicates.sh @@ -36,6 +36,7 @@ function run_test_once() $CLICKHOUSE_CLIENT -nm -q " DROP DICTIONARY simple_key_cache_dictionary_01863; + DROP TABLE simple_key_source_table_01863; " if [ "$prev" == "$curr" ]; then diff --git a/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh b/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh index cca710e85cf..75c0e4de29f 100755 --- a/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh +++ b/tests/queries/0_stateless/01854_HTTP_dict_decompression.sh @@ -5,3 +5,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh python3 "$CURDIR"/01854_HTTP_dict_decompression.python + +$CLICKHOUSE_CLIENT -q "DROP DICTIONARY test_table_select" diff --git a/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql b/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql index e6782656887..15edf8b2f9d 100644 --- a/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql +++ b/tests/queries/0_stateless/01866_datetime64_cmp_with_constant.sql @@ -38,3 +38,5 @@ SELECT 'dt64 != const dt' FROM dt64test WHERE dt64_column != toDateTime('2020-01 SELECT 'dt64 != dt' FROM dt64test WHERE dt64_column != materialize(toDateTime('2020-01-13 13:37:00')); SELECT 'dt != const dt64' FROM dt64test WHERE dt_column != toDateTime64('2020-01-13 13:37:00', 3); SELECT 'dt != dt64' FROM dt64test WHERE dt_column != materialize(toDateTime64('2020-01-13 13:37:00', 3)); + +DROP TABLE dt64test; diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 344b32659a8..0226bcc0a3c 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -48,3 +48,7 @@ SELECT count() FROM table4 WHERE id % 10 = 7; SELECT 'comparison:'; SELECT v, v-205 as vv, modulo(vv, 200), moduloLegacy(vv, 200) FROM table1 ORDER BY v; +DROP TABLE table1; +DROP TABLE table2; +DROP TABLE table3; +DROP TABLE table4; diff --git a/tests/queries/0_stateless/01882_scalar_subquery_exception.sql b/tests/queries/0_stateless/01882_scalar_subquery_exception.sql index c7a4555d876..0fb50846502 100644 --- a/tests/queries/0_stateless/01882_scalar_subquery_exception.sql +++ b/tests/queries/0_stateless/01882_scalar_subquery_exception.sql @@ -14,3 +14,6 @@ select / (select count() from nums_in_mem_dist where rand() > 0) from system.one; -- { serverError 158 } + +drop table nums_in_mem; +drop table nums_in_mem_dist; diff --git a/tests/queries/0_stateless/01882_total_rows_approx.sh b/tests/queries/0_stateless/01882_total_rows_approx.sh index 2c03298b324..f51e95b15c0 100755 --- a/tests/queries/0_stateless/01882_total_rows_approx.sh +++ b/tests/queries/0_stateless/01882_total_rows_approx.sh @@ -51,3 +51,5 @@ check_background_query & $CLICKHOUSE_CLIENT -q "select *, sleepEachRow(1) from data_01882" --max_threads=1 --format Null --query_id="$QUERY_ID" --max_block_size=1 wait + +$CLICKHOUSE_CLIENT -q "drop table data_01882" diff --git a/tests/queries/conftest.py b/tests/queries/conftest.py index 40a9a6b3a2e..a18ba846066 100644 --- a/tests/queries/conftest.py +++ b/tests/queries/conftest.py @@ -1,18 +1,35 @@ -import pytest - import os import sys -import tempfile +import re + +import pytest from .server import ServerThread +# Command-line arguments + def pytest_addoption(parser): parser.addoption( "--builddir", action="store", default=None, help="Path to build directory to use binaries from", ) +# HTML report hooks + +def pytest_html_report_title(report): + report.title = "ClickHouse Functional Stateless Tests (PyTest)" + + +RE_TEST_NAME = re.compile(r"\[(.*)\]") +def pytest_itemcollected(item): + match = RE_TEST_NAME.search(item.name) + if match: + item._nodeid = match.group(1) + + +# Fixtures + @pytest.fixture(scope='module') def cmdopts(request): return { diff --git a/tests/queries/pytest.ini b/tests/queries/pytest.ini new file mode 100644 index 00000000000..13a2ebbaf83 --- /dev/null +++ b/tests/queries/pytest.ini @@ -0,0 +1,2 @@ +[pytest] +render_collapsed = True diff --git a/tests/queries/query_test.py b/tests/queries/query_test.py index 6ebeccbeeac..d344abe0a1e 100644 --- a/tests/queries/query_test.py +++ b/tests/queries/query_test.py @@ -15,6 +15,7 @@ SKIP_LIST = [ # just fail "00133_long_shard_memory_tracker_and_exception_safety", + "00463_long_sessions_in_http_interface", "00505_secure", "00505_shard_secure", "00646_url_engine", @@ -56,8 +57,10 @@ SKIP_LIST = [ "01098_temporary_and_external_tables", "01099_parallel_distributed_insert_select", "01103_check_cpu_instructions_at_startup", + "01107_atomic_db_detach_attach", "01114_database_atomic", "01148_zookeeper_path_macros_unfolding", + "01152_cross_replication", # tcp port in reference "01175_distributed_ddl_output_mode_long", "01181_db_atomic_drop_on_cluster", # tcp port in reference "01280_ssd_complex_key_dictionary", @@ -99,19 +102,30 @@ SKIP_LIST = [ "01601_custom_tld", "01606_git_import", "01610_client_spawn_editor", # expect-test + "01654_test_writer_block_sequence", # No module named 'pandas' "01658_read_file_to_stringcolumn", "01666_merge_tree_max_query_limit", "01674_unicode_asan", "01676_clickhouse_client_autocomplete", # expect-test (partially) + "01676_long_clickhouse_client_autocomplete", "01683_text_log_deadlock", # secure tcp "01684_ssd_cache_dictionary_simple_key", "01685_ssd_cache_dictionary_complex_key", + "01737_clickhouse_server_wait_server_pool_long", "01746_executable_pool_dictionary", "01747_executable_pool_dictionary_implicit_key", "01747_join_view_filter_dictionary", "01748_dictionary_table_dot", "01754_cluster_all_replicas_shard_num", "01759_optimize_skip_unused_shards_zero_shards", + "01763_max_distributed_depth", # BROKEN + "01780_clickhouse_dictionary_source_loop", + "01801_s3_cluster", + "01802_test_postgresql_protocol_with_row_policy", + "01804_dictionary_decimal256_type", # hardcoded path + "01848_http_insert_segfault", + "01875_ssd_cache_dictionary_decimal256_type", + "01880_remote_ipv6", ] @@ -154,6 +168,7 @@ def run_shell(bin_prefix, server, database, path, reference, replace_map=None): 'CLICKHOUSE_PORT_TCP_WITH_PROXY': str(server.proxy_port), 'CLICKHOUSE_PORT_HTTP': str(server.http_port), 'CLICKHOUSE_PORT_INTERSERVER': str(server.inter_port), + 'CLICKHOUSE_PORT_POSTGRESQL': str(server.postgresql_port), 'CLICKHOUSE_TMP': server.tmp_dir, 'CLICKHOUSE_CONFIG_CLIENT': server.client_config } diff --git a/tests/queries/server.py b/tests/queries/server.py index ed12931e658..50110bc41b9 100644 --- a/tests/queries/server.py +++ b/tests/queries/server.py @@ -38,6 +38,7 @@ class ServerThread(threading.Thread): self.https_port = port_base + 5 self.odbc_port = port_base + 6 self.proxy_port = port_base + 7 + self.postgresql_port = port_base + 8 self._args = [ '--config-file={config_path}'.format(config_path=self.server_config), @@ -46,6 +47,7 @@ class ServerThread(threading.Thread): '--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), + '--postgresql_port={psql_port}'.format(psql_port=self.postgresql_port), # TODO: SSL certificate is not specified '--tcp_port_secure={tcps_port}'.format(tcps_port=self.tcps_port), ] @@ -75,7 +77,7 @@ class ServerThread(threading.Thread): time.sleep(ServerThread.DEFAULT_SERVER_DELAY) s = socket.create_connection(('localhost', self.tcp_port), ServerThread.DEFAULT_CONNECTION_TIMEOUT) s.sendall(b'G') # trigger expected "bad" HELLO response - print('Successful server response:', s.recv(1024)) # FIXME: read whole buffered response + s.recv(1024) # FIXME: read whole buffered response s.shutdown(socket.SHUT_RDWR) s.close() except Exception: @@ -116,7 +118,7 @@ class ServerThread(threading.Thread): if self._proc.returncode is None: self._proc.terminate() self.join() - print('Stop clickhouse-server') + print('Stopped clickhouse-server') ServerThread.DEFAULT_SERVER_CONFIG = \ @@ -289,6 +291,21 @@ ServerThread.DEFAULT_SERVER_CONFIG = \ + + + + + shard_0 + localhost + {tcp_port} + + + shard_1 + localhost + {tcp_port} + + +