Skip to content

Commit 5046832

Browse files
committed
Stateless tests: add test job with enabled async inserts
1 parent 2768ae9 commit 5046832

26 files changed

+183
-30
lines changed

tests/async_insert_blacklist.txt

Lines changed: 108 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,108 @@
1+
Deduplication in dependent materialized view cannot work together with async inserts. Please disable either `deduplicate_blocks_in_dependent_materialized_views` or `async_insert` setting
2+
00633_materialized_view_and_too_many_parts_zookeeper
3+
02912_ingestion_mv_deduplication
4+
03008_deduplication_cases_from_docs
5+
03008_deduplication_insert_into_partitioned_table
6+
01927_query_views_log_matview_exceptions
7+
8+
!!!insert deduplication doesn't work
9+
02124_insert_deduplication_token_multiple_blocks
10+
02124_insert_deduplication_token_multiple_blocks_replica
11+
03008_deduplication_random_setttings
12+
13+
Cannot insert column value, because it is MATERIALIZED column
14+
00952_insert_into_distributed_with_materialized_column
15+
16+
!!!performance degradation (timeout)
17+
02046_low_cardinality_parallel_group_by
18+
03221_insert_timeout_overflow_mode
19+
20+
Not bugs
21+
22+
Transactions
23+
Async inserts inside transactions are not supported. (NOT_IMPLEMENTED)
24+
01133_begin_commit_race
25+
01169_old_alter_partition_isolation_stress
26+
01170_alter_partition_isolation
27+
01171_mv_select_insert_isolation_long
28+
01172_transaction_counters
29+
01173_transaction_control_queries
30+
01174_select_insert_isolation
31+
02421_truncate_isolation_no_merges
32+
02421_truncate_isolation_with_mutations
33+
03167_transactions_are_really_disabled
34+
01167_isolation_hermitage
35+
36+
Async inserts with 'implicit_transaction' are not supported. (NOT_IMPLEMENTED)
37+
01168_mutations_isolation
38+
Cannot execute query because current transaction failed. Expecting ROLLBACK statement. (INVALID_TRANSACTION
39+
01169_alter_partition_isolation_stress
40+
41+
Processing async inserts with both inlined and external data (from stdin or infile) is not supported. (NOT_IMPLEMENTED)
42+
02270_stdin_with_query_or_infile_data
43+
02435_rollback_cancelled_queries
44+
45+
AsyncInsertRows profile event is missing
46+
01413_rows_events
47+
48+
https://github.com/ClickHouse/ClickHouse/issues/57768
49+
02423_insert_summary_behaviour
50+
51+
https://github.com/ClickHouse/ClickHouse/issues/79790
52+
02046_remote_table_function_named_collections
53+
54+
55+
FAIL 02417_opentelemetry_insert_on_distributed_table 5.20
56+
2025-04-04 03:39:33 Reason: result differs with reference:
57+
2025-04-04 03:39:33 --- /repo/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.reference 2025-04-04 03:23:18.320164897 +1030
58+
2025-04-04 03:39:33 +++ /repo/tests/queries/0_stateless/02417_opentelemetry_insert_on_distributed_table.stdout 2025-04-04 03:39:33.334212935 +1030
59+
2025-04-04 03:39:33 @@ -1,15 +1,9 @@
60+
2025-04-04 03:39:33 ===1===
61+
2025-04-04 03:39:33 -{"operation_name":"void DB::DistributedSink::writeToLocal(const Cluster::ShardInfo &, const Block &, size_t)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"}
62+
2025-04-04 03:39:33 -{"operation_name":"void DB::DistributedSink::writeToLocal(const Cluster::ShardInfo &, const Block &, size_t)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"}
63+
2025-04-04 03:39:33 1
64+
2025-04-04 03:39:33 ===2===
65+
2025-04-04 03:39:33 -{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(std::string &, const SettingsChanges &)","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"}
66+
2025-04-04 03:39:33 -{"operation_name":"void DB::DistributedAsyncInsertDirectoryQueue::processFile(std::string &, const SettingsChanges &)","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"}
67+
2025-04-04 03:39:33 -3
68+
2025-04-04 03:39:33 -2
69+
2025-04-04 03:39:33 +1
70+
2025-04-04 03:39:33 +0
71+
2025-04-04 03:39:33 ===3===
72+
2025-04-04 03:39:33 -{"operation_name":"auto DB::DistributedSink::runWritingJob(JobReplica &, const Block &, size_t)::(anonymous class)::operator()() const","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"}
73+
2025-04-04 03:39:33 -{"operation_name":"auto DB::DistributedSink::runWritingJob(JobReplica &, const Block &, size_t)::(anonymous class)::operator()() const","cluster":"test_cluster_two_shards_localhost","shard":"2","rows":"1","bytes":"8"}
74+
2025-04-04 03:39:33 1
75+
2025-04-04 03:39:33 ===4===
76+
2025-04-04 03:39:33 {"operation_name":"auto DB::DistributedSink::runWritingJob(JobReplica &, const Block &, size_t)::(anonymous class)::operator()() const","cluster":"test_cluster_two_shards_localhost","shard":"1","rows":"1","bytes":"8"}
77+
2025-04-04 03:39:33
78+
2025-04-04 03:39:33
79+
2025-04-04 03:39:33 Settings used in the test: --max_insert_threads 1 --group_by_two_level_threshold 1 --group_by_two_level_threshold_bytes 1 --distributed_aggregation_memory_efficient 1 --fsync_metadata 0 --output_format_parallel_formatting 0 --input_format_parallel_parsing 0 --min_chunk_bytes_for_parallel_parsing 11050395 --max_read_buffer_size 867254 --prefer_localhost_replica 1 --max_block_size 33216 --max_joined_block_size_rows 8084 --max_threads 1 --optimize_append_index 0 --optimize_if_chain_to_multiif 1 --optimize_if_transform_strings_to_enum 1 --optimize_read_in_order 1 --optimize_or_like_chain 0 --optimize_substitute_columns 1 --enable_multiple_prewhere_read_steps 0 --read_in_order_two_level_merge_threshold 28 --optimize_aggregation_in_order 1 --aggregation_in_order_max_block_bytes 17150565 --use_uncompressed_cache 0 --min_bytes_to_use_direct_io 10737418240 --min_bytes_to_use_mmap_io 7955247908 --local_filesystem_read_method pread_threadpool --remote_filesystem_read_method threadpool --local_filesystem_read_prefetch 1 --filesystem_cache_segments_batch_size 2 --read_from_filesystem_cache_if_exists_otherwise_bypass_cache 1 --throw_on_error_from_cache_on_write_operations 0 --remote_filesystem_read_prefetch 0 --allow_prefetched_read_pool_for_remote_filesystem 1 --filesystem_prefetch_max_memory_usage 64Mi --filesystem_prefetches_limit 10 --filesystem_prefetch_min_bytes_for_single_read_task 1Mi --filesystem_prefetch_step_marks 0 --filesystem_prefetch_step_bytes 100Mi --compile_aggregate_expressions 1 --compile_sort_description 1 --merge_tree_coarse_index_granularity 20 --optimize_distinct_in_order 0 --max_bytes_before_remerge_sort 1807198608 --min_compress_block_size 264715 --max_compress_block_size 486163 --merge_tree_compact_parts_min_granules_to_multibuffer_read 14 --optimize_sorting_by_input_stream_properties 0 --http_response_buffer_size 9535929 --http_wait_end_of_query True --enable_memory_bound_merging_of_aggregation_results 0 --min_count_to_compile_expression 3 --min_count_to_compile_aggregate_expression 0 --min_count_to_compile_sort_description 3 --session_timezone Mexico/BajaSur --use_page_cache_for_disks_without_file_cache False --page_cache_inject_eviction True --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.48 --prefer_external_sort_block_bytes 100000000 --cross_join_min_rows_to_compress 100000000 --cross_join_min_bytes_to_compress 100000000 --min_external_table_block_size_bytes 100000000 --max_parsing_threads 10 --optimize_functions_to_subcolumns 0 --parallel_replicas_local_plan 0 --query_plan_join_swap_table auto --enable_vertical_final 1 --optimize_extract_common_expressions 1 --use_query_condition_cache 0 --max_bytes_before_external_sort 0 --max_bytes_before_external_group_by 0 --max_bytes_ratio_before_external_sort 0.2 --max_bytes_ratio_before_external_group_by 0.2
80+
2025-04-04 03:39:33
81+
2025-04-04 03:39:33 MergeTree settings used in test: --ratio_of_defaults_for_sparse_serialization 0.0 --prefer_fetch_merged_part_size_threshold 10737418240 --vertical_merge_algorithm_min_rows_to_activate 1 --vertical_merge_algorithm_min_columns_to_activate 100 --allow_vertical_merges_from_compact_to_wide_parts 1 --min_merge_bytes_to_use_direct_io 1 --index_granularity_bytes 13714526 --merge_max_block_size 4995 --index_granularity 29598 --min_bytes_for_wide_part 915939826 --marks_compress_block_size 26904 --primary_key_compress_block_size 61136 --replace_long_file_name_to_hash 0 --max_file_name_length 24 --min_bytes_for_full_part_storage 536870912 --compact_parts_max_bytes_to_buffer 52075424 --compact_parts_max_granules_to_buffer 1 --compact_parts_merge_max_bytes_to_prefetch_part 15860469 --cache_populated_by_fetch 1 --concurrent_part_removal_threshold 10 --old_parts_lifetime 299 --prewarm_mark_cache 1 --use_const_adaptive_granularity 1 --enable_index_granularity_compression 1 --enable_block_number_column 0 --enable_block_offset_column 0 --use_primary_key_cache 0 --prewarm_primary_key_cache 0
82+
2025-04-04 03:39:33
83+
2025-04-04 03:39:33 Database: test_swjjbrc6
84+
85+
FAIL 02421_simple_queries_for_opentelemetry 4.00
86+
2025-04-04 03:30:21 Reason: result differs with reference:
87+
2025-04-04 03:30:21 --- /repo/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.reference 2025-04-04 03:23:18.320164897 +1030
88+
2025-04-04 03:30:21 +++ /repo/tests/queries/0_stateless/02421_simple_queries_for_opentelemetry.stdout 2025-04-04 03:30:21.155889098 +1030
89+
2025-04-04 03:30:21 @@ -1,6 +1,6 @@
90+
2025-04-04 03:30:21 {"query":"show processlist format Null
91+
"}
92+
2025-04-04 03:30:21 {"query":"show databases format Null
93+
"}
94+
2025-04-04 03:30:21 -{"query":"insert into opentelemetry_test values","read_rows":"3","written_rows":"3"}
95+
2025-04-04 03:30:21 +{"query":"insert into opentelemetry_test values","read_rows":"","written_rows":""}
96+
2025-04-04 03:30:21 {"query":"select * from opentelemetry_test format Null
97+
","read_rows":"3","written_rows":""}
98+
2025-04-04 03:30:21 {"query":"SELECT * FROM opentelemetry_test FORMAT Null
99+
","read_rows":"3","written_rows":""}
100+
2025-04-04 03:30:21 {"min_compress_block_size":"present","max_block_size":"present","max_execution_time":"present"}
101+
2025-04-04 03:30:21
102+
2025-04-04 03:30:21
103+
2025-04-04 03:30:21 Settings used in the test: --max_insert_threads 1 --group_by_two_level_threshold 440855 --group_by_two_level_threshold_bytes 11597371 --distributed_aggregation_memory_efficient 1 --fsync_metadata 0 --output_format_parallel_formatting 1 --input_format_parallel_parsing 0 --min_chunk_bytes_for_parallel_parsing 7737435 --max_read_buffer_size 860942 --prefer_localhost_replica 1 --max_block_size 58921 --max_joined_block_size_rows 99921 --max_threads 2 --optimize_append_index 1 --optimize_if_chain_to_multiif 1 --optimize_if_transform_strings_to_enum 1 --optimize_read_in_order 0 --optimize_or_like_chain 0 --optimize_substitute_columns 1 --enable_multiple_prewhere_read_steps 0 --read_in_order_two_level_merge_threshold 40 --optimize_aggregation_in_order 0 --aggregation_in_order_max_block_bytes 43570048 --use_uncompressed_cache 0 --min_bytes_to_use_direct_io 10737418240 --min_bytes_to_use_mmap_io 10737418240 --local_filesystem_read_method read --remote_filesystem_read_method threadpool --local_filesystem_read_prefetch 0 --filesystem_cache_segments_batch_size 10 --read_from_filesystem_cache_if_exists_otherwise_bypass_cache 0 --throw_on_error_from_cache_on_write_operations 1 --remote_filesystem_read_prefetch 0 --allow_prefetched_read_pool_for_remote_filesystem 1 --filesystem_prefetch_max_memory_usage 128Mi --filesystem_prefetches_limit 10 --filesystem_prefetch_min_bytes_for_single_read_task 1Mi --filesystem_prefetch_step_marks 0 --filesystem_prefetch_step_bytes 0 --compile_aggregate_expressions 0 --compile_sort_description 1 --merge_tree_coarse_index_granularity 4 --optimize_distinct_in_order 1 --max_bytes_before_remerge_sort 569103651 --min_compress_block_size 1196021 --max_compress_block_size 1373014 --merge_tree_compact_parts_min_granules_to_multibuffer_read 59 --optimize_sorting_by_input_stream_properties 0 --http_response_buffer_size 2579816 --http_wait_end_of_query True --enable_memory_bound_merging_of_aggregation_results 0 --min_count_to_compile_expression 3 --min_count_to_compile_aggregate_expression 3 --min_count_to_compile_sort_description 3 --session_timezone America/Hermosillo --use_page_cache_for_disks_without_file_cache True --page_cache_inject_eviction True --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.34 --prefer_external_sort_block_bytes 1 --cross_join_min_rows_to_compress 0 --cross_join_min_bytes_to_compress 1 --min_external_table_block_size_bytes 100000000 --max_parsing_threads 1 --optimize_functions_to_subcolumns 1 --parallel_replicas_local_plan 1 --query_plan_join_swap_table true --enable_vertical_final 0 --optimize_extract_common_expressions 0 --use_query_condition_cache 0 --max_bytes_before_external_sort 0 --max_bytes_before_external_group_by 0 --max_bytes_ratio_before_external_sort 0.0 --max_bytes_ratio_before_external_group_by 0.11
104+
2025-04-04 03:30:21
105+
2025-04-04 03:30:21 MergeTree settings used in test: --ratio_of_defaults_for_sparse_serialization 1.0 --prefer_fetch_merged_part_size_threshold 1 --vertical_merge_algorithm_min_rows_to_activate 1000000 --vertical_merge_algorithm_min_columns_to_activate 40 --allow_vertical_merges_from_compact_to_wide_parts 1 --min_merge_bytes_to_use_direct_io 10737418240 --index_granularity_bytes 24194483 --merge_max_block_size 7504 --index_granularity 39728 --min_bytes_for_wide_part 0 --marks_compress_block_size 8442 --primary_key_compress_block_size 96506 --replace_long_file_name_to_hash 0 --max_file_name_length 128 --min_bytes_for_full_part_storage 230961642 --compact_parts_max_bytes_to_buffer 138334530 --compact_parts_max_granules_to_buffer 174 --compact_parts_merge_max_bytes_to_prefetch_part 32459157 --cache_populated_by_fetch 0 --concurrent_part_removal_threshold 100 --old_parts_lifetime 480 --prewarm_mark_cache 0 --use_const_adaptive_granularity 1 --enable_index_granularity_compression 1 --enable_block_number_column 0 --enable_block_offset_column 1 --use_primary_key_cache 1 --prewarm_primary_key_cache 1
106+
2025-04-04 03:30:21
107+
2025-04-04 03:30:21 Database: test_ho1x48nh
108+

tests/clickhouse-test

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -910,6 +910,7 @@ class FailureReason(enum.Enum):
910910
AZURE_BLOB_STORAGE = "azure-blob-storage"
911911
BUILD = "not running for current build"
912912
NO_PARALLEL_REPLICAS = "smth in not supported with parallel replicas"
913+
NO_ASYNC_INSERT = "test is not supported with async inserts enabled"
913914
SHARED_MERGE_TREE = "no-shared-merge-tree"
914915
DISTRIBUTED_CACHE = "distributed-cache"
915916
NO_STATELESS = "no-stateless"
@@ -1450,6 +1451,9 @@ class TestCase:
14501451
if "no-parallel-replicas" in tags and args.no_parallel_replicas:
14511452
return FailureReason.NO_PARALLEL_REPLICAS
14521453

1454+
if "no-async-insert" in tags and args.no_async_insert:
1455+
return FailureReason.NO_ASYNC_INSERT
1456+
14531457
if args.skip and any(s in self.name for s in args.skip):
14541458
return FailureReason.SKIP
14551459

@@ -1870,7 +1874,10 @@ class TestCase:
18701874
+ database
18711875
+ "_1"
18721876
)
1873-
for env_to_param in ["CLICKHOUSE_USER_FILES", "CLICKHOUSE_USER_FILES_UNIQUE"]:
1877+
for env_to_param in [
1878+
"CLICKHOUSE_USER_FILES",
1879+
"CLICKHOUSE_USER_FILES_UNIQUE",
1880+
]:
18741881
value = os.environ.get(env_to_param, None)
18751882
if value:
18761883
query_params += f" --param_{env_to_param}={value}"
@@ -3324,6 +3331,12 @@ def main(args):
33243331
blacklist = try_get_skip_list(base_dir, "../parallel_replicas_blacklist.txt")
33253332
blacklist_check.extend(blacklist)
33263333

3334+
if args.no_async_insert is True:
3335+
skip_list_async_inserts = try_get_skip_list(
3336+
base_dir, "../async_insert_blacklist.txt"
3337+
)
3338+
skip_list.extend(skip_list_async_inserts)
3339+
33273340
for suite in sorted(os.listdir(base_dir), key=suite_key_func):
33283341
if server_died.is_set():
33293342
break
@@ -3801,6 +3814,12 @@ def parse_args():
38013814
default=False,
38023815
help="Do not include tests that are not supported with parallel replicas feature",
38033816
)
3817+
parser.add_argument(
3818+
"--no-async-insert",
3819+
action="store_true",
3820+
default=False,
3821+
help="Do not include tests that are not supported with async insert feature",
3822+
)
38043823
parser.add_argument(
38053824
"--replace-replicated-with-shared",
38063825
action="store_true",

tests/config/users.d/enable_async_inserts.xml

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2,6 +2,7 @@
22
<profiles>
33
<default>
44
<async_insert>1</async_insert>
5+
<apply_settings_from_server>1</apply_settings_from_server>
56
</default>
67
</profiles>
78
</clickhouse>

tests/docker_scripts/stateless_runner.sh

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -333,6 +333,10 @@ function run_tests()
333333
ADDITIONAL_OPTIONS+=('--zookeeper')
334334
ADDITIONAL_OPTIONS+=('--shard')
335335
fi
336+
if [[ "$USE_ASYNC_INSERTS" -eq 1 ]]; then
337+
ADDITIONAL_OPTIONS+=('--no-async-insert')
338+
fi
339+
336340
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
337341
ADDITIONAL_OPTIONS+=('--no-stateful')
338342
fi

tests/queries/0_stateless/00453_cast_enum.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,6 @@ INSERT INTO cast_enums SELECT 2 AS type, toDate('2017-01-01') AS date, number AS
1212

1313
SELECT type, date, id FROM cast_enums ORDER BY type, id;
1414

15-
INSERT INTO cast_enums VALUES ('wrong_value', '2017-01-02', 7); -- { clientError UNKNOWN_ELEMENT_OF_ENUM }
15+
INSERT INTO cast_enums VALUES ('wrong_value', '2017-01-02', 7); -- { error UNKNOWN_ELEMENT_OF_ENUM }
1616

1717
DROP TABLE IF EXISTS cast_enums;

tests/queries/0_stateless/00634_performance_introspection_and_logging.sh

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,7 @@
11
#!/usr/bin/env bash
2+
# Tags: no-async-insert
3+
# no-async-insert: Test expects new part for each insert
4+
25
set -e
36

47
# Get all server logs

tests/queries/0_stateless/00748_insert_array_with_null.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@ set input_format_null_as_default=0;
55
CREATE TABLE arraytest ( created_date Date DEFAULT toDate(created_at), created_at DateTime DEFAULT now(), strings Array(String) DEFAULT emptyArrayString()) ENGINE = MergeTree(created_date, cityHash64(created_at), (created_date, cityHash64(created_at)), 8192);
66

77
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', 'ccccc']);
8-
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', null]); -- { clientError CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN }
8+
INSERT INTO arraytest (created_at, strings) VALUES (now(), ['aaaaa', 'bbbbb', null]); -- { error CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN }
99

1010
SELECT strings from arraytest;
1111

tests/queries/0_stateless/01158_zookeeper_log_long.sql

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,6 +1,7 @@
1-
-- Tags: long, zookeeper, no-replicated-database, no-polymorphic-parts, no-random-merge-tree-settings, no-shared-merge-tree
1+
-- Tags: long, zookeeper, no-replicated-database, no-polymorphic-parts, no-random-merge-tree-settings, no-shared-merge-tree, no-async-insert
22
-- Tag no-replicated-database: Fails due to additional replicas or shards
33
-- no-shared-merge-tree: depends on structure in zookeeper of replicated merge tree
4+
-- no-async-insert: Test expects new part for each insert
45

56
SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries
67

tests/queries/0_stateless/01165_lost_part_empty_partition.sql

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -7,7 +7,7 @@ create table rmt1 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{d
77
create table rmt2 (d DateTime, n int) engine=ReplicatedMergeTree('/test/01165/{database}/rmt', '2') order by n partition by toYYYYMMDD(d);
88

99
system stop replicated sends rmt1;
10-
insert into rmt1 values (now(), arrayJoin([1, 2])); -- { clientError BAD_ARGUMENTS }
10+
insert into rmt1 values (now(), arrayJoin([1, 2])); -- { error BAD_ARGUMENTS }
1111
insert into rmt1(n) select * from system.numbers limit arrayJoin([1, 2]); -- { serverError BAD_ARGUMENTS, INVALID_LIMIT_EXPRESSION }
1212
insert into rmt1 values (now(), rand());
1313
drop table rmt1;

0 commit comments

Comments
 (0)