From cb7be14492ed358416a35172ecc43741af605642 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 4 May 2022 20:16:42 +0300 Subject: [PATCH 001/101] FR: Expose what triggered the merge in system.part_log #26255 --- src/Interpreters/PartLog.cpp | 31 ++++++++++++++++ src/Interpreters/PartLog.h | 15 ++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 4 ++ .../02293_part_log_has_merge_reason.reference | 1 + .../02293_part_log_has_merge_reason.sql | 37 +++++++++++++++++++ 5 files changed, 88 insertions(+) create mode 100644 tests/queries/0_stateless/02293_part_log_has_merge_reason.reference create mode 100644 tests/queries/0_stateless/02293_part_log_has_merge_reason.sql diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index ce9aa0c03d1..643fd192cad 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -16,6 +16,25 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType merge_type) { + switch (merge_type) + { + case MergeType::REGULAR: + return REGULAR_MERGE; + case MergeType::TTL_DELETE: + return TTL_DELETE_MERGE; + case MergeType::TTL_RECOMPRESS: + return TTL_RECOMPRESS_MERGE; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); +} + NamesAndTypesList PartLogElement::getNamesAndTypes() { auto event_type_datatype = std::make_shared( @@ -30,11 +49,22 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() } ); + auto merge_reason_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"NotAMerge", static_cast(NOT_A_MERGE)}, + {"RegularMerge", static_cast(REGULAR_MERGE)}, + {"TTLDeleteMerge", static_cast(TTL_DELETE_MERGE)}, + {"TTLRecompressMerge", static_cast(TTL_RECOMPRESS_MERGE)}, + } + ); + ColumnsWithTypeAndName columns_with_type_and_name; return { {"query_id", std::make_shared()}, {"event_type", std::move(event_type_datatype)}, + {"merge_reason", std::move(merge_reason_datatype)}, {"event_date", std::make_shared()}, {"event_time", std::make_shared()}, @@ -71,6 +101,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(query_id); columns[i++]->insert(event_type); + columns[i++]->insert(merge_reason); columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); columns[i++]->insert(event_time); columns[i++]->insert(event_time_microseconds); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 7582f6fe9e6..48a54c55b1c 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB @@ -20,9 +21,22 @@ struct PartLogElement MOVE_PART = 6, }; + enum MergeReasonType + { + /// merge_reason is relevant only for event_type = 'MERGE_PARTS', in other cases it is NOT_A_MERGE + NOT_A_MERGE = 1, + /// Just regular merge + REGULAR_MERGE = 2, + /// Merge assigned to delete some data from parts (with TTLMergeSelector) + TTL_DELETE_MERGE = 3, + /// Merge with recompression + TTL_RECOMPRESS_MERGE = 4, + }; + String query_id; Type event_type = NEW_PART; + MergeReasonType merge_reason = NOT_A_MERGE; time_t event_time = 0; Decimal64 event_time_microseconds = 0; @@ -54,6 +68,7 @@ struct PartLogElement static std::string name() { return "PartLog"; } + static MergeReasonType getMergeReasonType(MergeType merge_type); static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d84fb9d30d3..86787635b1b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6104,6 +6104,10 @@ try part_log_elem.event_type = type; + if (part_log_elem.event_type == PartLogElement::MERGE_PARTS) + if (merge_entry) + part_log_elem.merge_reason = PartLogElement::getMergeReasonType((*merge_entry)->merge_type); + part_log_elem.error = static_cast(execution_status.code); part_log_elem.exception = execution_status.message; diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql new file mode 100644 index 00000000000..db1f4c26af4 --- /dev/null +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS t_part_log_has_merge_type_table; + +CREATE TABLE t_part_log_has_merge_type_table +( + event_time DateTime, + UserID UInt64, + Comment String +) +ENGINE = MergeTree() +ORDER BY tuple() +TTL event_time + toIntervalMonth(3) +SETTINGS min_bytes_for_wide_part = 0, merge_with_ttl_timeout = 1; + +INSERT INTO t_part_log_has_merge_type_table VALUES (now(), 1, 'username1'); +INSERT INTO t_part_log_has_merge_type_table VALUES (now() - INTERVAL 4 MONTH, 2, 'username2'); + +OPTIMIZE TABLE t_part_log_has_merge_type_table FINAL; + +SYSTEM FLUSH LOGS; + +SELECT count(*) +FROM +( + SELECT + metadata_modification_time, + event_time + FROM system.tables AS l + INNER JOIN system.part_log AS r + ON l.name = r.table + WHERE (l.database = currentDatabase()) AND + (l.name = 't_part_log_has_merge_type_table') AND + (r.event_type = 'MergeParts') AND + (r.merge_reason = 'TTLDeleteMerge') +) +WHERE (metadata_modification_time <= event_time); + +DROP TABLE t_part_log_has_merge_type_table; From d6d249d964971bf17064a5fdee0d5b953e14a42a Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 4 May 2022 20:16:42 +0300 Subject: [PATCH 002/101] FR: Expose what triggered the merge in system.part_log #26255 --- docs/en/operations/system-tables/part_log.md | 6 +++++ src/Interpreters/PartLog.cpp | 9 ++++--- .../02293_part_log_has_merge_reason.reference | 2 +- .../02293_part_log_has_merge_reason.sql | 26 +++++++------------ 4 files changed, 21 insertions(+), 22 deletions(-) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index 00eaca23862..1b567367c97 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -14,6 +14,11 @@ The `system.part_log` table contains the following columns: - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). - `MUTATE_PART` — Mutating of a data part. - `MOVE_PART` — Moving the data part from the one disk to another one. +- `merge_reason` ([Enum8](../../sql-reference/data-types/enum.md)) — The reason for the event with type `MERGE_PARTS`. Can have one of the following values: + - `NOT_A_MERGE` — The current event has the type other than `MERGE_PARTS`. + - `REGULAR_MERGE` — Some regular merge. + - `TTL_DELETE_MERGE` — Cleaning up expired data. + - `TTL_RECOMPRESS_MERGE` — Recompressing data part with the. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds precision. @@ -46,6 +51,7 @@ Row 1: ────── query_id: 983ad9c7-28d5-4ae1-844e-603116b7de31 event_type: NewPart +merge_reason: NotAMerge event_date: 2021-02-02 event_time: 2021-02-02 11:14:28 event_time_microseconds: 2021-02-02 11:14:28.861919 diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 643fd192cad..4474c22d464 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -21,14 +21,15 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType merge_type) { +PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType merge_type) +{ switch (merge_type) { - case MergeType::REGULAR: + case MergeType::Regular: return REGULAR_MERGE; - case MergeType::TTL_DELETE: + case MergeType::TTLDelete: return TTL_DELETE_MERGE; - case MergeType::TTL_RECOMPRESS: + case MergeType::TTLRecompress: return TTL_RECOMPRESS_MERGE; } diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference index d00491fd7e5..220107cf15b 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference @@ -1 +1 @@ -1 +MergeParts TTLDeleteMerge diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql index db1f4c26af4..7ef86354e71 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql @@ -8,30 +8,22 @@ CREATE TABLE t_part_log_has_merge_type_table ) ENGINE = MergeTree() ORDER BY tuple() -TTL event_time + toIntervalMonth(3) -SETTINGS min_bytes_for_wide_part = 0, merge_with_ttl_timeout = 1; +SETTINGS min_bytes_for_wide_part = 0, materialize_ttl_recalculate_only = true; INSERT INTO t_part_log_has_merge_type_table VALUES (now(), 1, 'username1'); INSERT INTO t_part_log_has_merge_type_table VALUES (now() - INTERVAL 4 MONTH, 2, 'username2'); +ALTER TABLE t_part_log_has_merge_type_table + MODIFY TTL event_time + INTERVAL 3 MONTH; + OPTIMIZE TABLE t_part_log_has_merge_type_table FINAL; SYSTEM FLUSH LOGS; -SELECT count(*) -FROM -( - SELECT - metadata_modification_time, - event_time - FROM system.tables AS l - INNER JOIN system.part_log AS r - ON l.name = r.table - WHERE (l.database = currentDatabase()) AND - (l.name = 't_part_log_has_merge_type_table') AND - (r.event_type = 'MergeParts') AND - (r.merge_reason = 'TTLDeleteMerge') -) -WHERE (metadata_modification_time <= event_time); +SELECT + event_type, + merge_reason +FROM system.part_log +WHERE (table = 't_part_log_has_merge_type_table') AND (merge_reason = 'TTLDeleteMerge'); DROP TABLE t_part_log_has_merge_type_table; From e9187ec4b7938616957bbad06c4816fcd94d7777 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 23 May 2022 14:35:09 +0000 Subject: [PATCH 003/101] Overcommit: update defaults, exception message and add ProfileEvent --- programs/server/Server.cpp | 2 +- src/Common/MemoryTracker.cpp | 36 ++++++++++++++++--- src/Common/OvercommitTracker.cpp | 24 ++++++++++--- src/Common/OvercommitTracker.h | 12 ++++++- src/Common/ProfileEvents.cpp | 1 + src/Common/tests/gtest_overcommit_tracker.cpp | 20 +++++------ 6 files changed, 73 insertions(+), 22 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index defc66b0ed9..18ab96983eb 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1095,7 +1095,7 @@ int Server::main(const std::vector & /*args*/) total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); - UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 200); + UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 5'000'000); global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 0e7803aaa71..b5a27543b4e 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #ifdef MEMORY_TRACKER_DEBUG_CHECKS @@ -52,6 +53,30 @@ namespace DB } } +namespace +{ + +inline std::string_view toDescription(OvercommitResult result) +{ + switch (result) + { + case OvercommitResult::NONE: + return "Memory overcommit isn't used. OvercommitTracker isn't set."; + case OvercommitResult::DISABLED: + return "Memory overcommit isn't used. Waiting time or orvercommit denominator are set to zero."; + case OvercommitResult::MEMORY_FREED: + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, "OvercommitResult::MEMORY_FREED shouldn't be asked for description"); + case OvercommitResult::SELECTED: + return "Query was selected to stop by OvercommitTracker."; + case OvercommitResult::TIMEOUTED: + return "Waiting timeout for memory to be freed is reached."; + case OvercommitResult::NOT_ENOUGH_FREED: + return "Memory overcommit has freed not enough memory."; + } +} + +} + namespace ProfileEvents { extern const Event QueryMemoryLimitExceeded; @@ -189,11 +214,11 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT if (unlikely(current_hard_limit && will_be > current_hard_limit) && memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { - bool need_to_throw = true; + OvercommitResult overcommit_result = OvercommitResult::NONE; if (auto * overcommit_tracker_ptr = overcommit_tracker.load(std::memory_order_relaxed); overcommit_tracker_ptr != nullptr && query_tracker != nullptr) - need_to_throw = overcommit_tracker_ptr->needToStopQuery(query_tracker, size); + overcommit_result = overcommit_tracker_ptr->needToStopQuery(query_tracker, size); - if (need_to_throw) + if (overcommit_result != OvercommitResult::MEMORY_FREED) { /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -201,12 +226,13 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT const auto * description = description_ptr.load(std::memory_order_relaxed); throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, - "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}", + "Memory limit{}{} exceeded: would use {} (attempt to allocate chunk of {} bytes), maximum: {}. OvercommitTracker decision: {}.", description ? " " : "", description ? description : "", formatReadableSizeWithBinarySuffix(will_be), size, - formatReadableSizeWithBinarySuffix(current_hard_limit)); + formatReadableSizeWithBinarySuffix(current_hard_limit), + toDescription(overcommit_result)); } else { diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index dbacc0d81a4..0c03ba58e87 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -2,8 +2,14 @@ #include #include +#include #include +namespace ProfileEvents +{ + extern const Event MemoryOvercommitWaitTimeMicroseconds; +} + using namespace std::chrono_literals; constexpr std::chrono::microseconds ZERO_MICROSEC = 0us; @@ -24,7 +30,7 @@ void OvercommitTracker::setMaxWaitTime(UInt64 wait_time) max_wait_time = wait_time * 1us; } -bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) +OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) { // NOTE: Do not change the order of locks // @@ -36,7 +42,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) std::unique_lock lk(overcommit_m); if (max_wait_time == ZERO_MICROSEC) - return true; + return OvercommitResult::DISABLED; pickQueryToExclude(); assert(cancellation_state != QueryCancellationState::NONE); @@ -50,7 +56,7 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) // picked_tracker to be not null pointer. assert(cancellation_state == QueryCancellationState::SELECTED); cancellation_state = QueryCancellationState::NONE; - return true; + return OvercommitResult::DISABLED; } if (picked_tracker == tracker) { @@ -58,17 +64,20 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) // It may happen even when current state is RUNNING, because // ThreadStatus::~ThreadStatus may call MemoryTracker::alloc. cancellation_state = QueryCancellationState::RUNNING; - return true; + return OvercommitResult::SELECTED; } allow_release = true; required_memory += amount; required_per_thread[tracker] = amount; + auto wait_start_time = std::chrono::system_clock::now(); bool timeout = !cv.wait_for(lk, max_wait_time, [this, tracker]() { return required_per_thread[tracker] == 0 || cancellation_state == QueryCancellationState::NONE; }); + auto wait_end_time = std::chrono::system_clock::now(); + ProfileEvents::increment(ProfileEvents::MemoryOvercommitWaitTimeMicroseconds, (wait_end_time - wait_start_time) / 1us); LOG_DEBUG(getLogger(), "Memory was{} freed within timeout", (timeout ? " not" : "")); required_memory -= amount; @@ -84,7 +93,12 @@ bool OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) // As we don't need to free memory, we can continue execution of the selected query. if (required_memory == 0 && cancellation_state == QueryCancellationState::SELECTED) reset(); - return timeout || still_need != 0; + if (timeout) + return OvercommitResult::TIMEOUTED; + if (still_need != 0) + return OvercommitResult::NOT_ENOUGH_FREED; + else + return OvercommitResult::MEMORY_FREED; } void OvercommitTracker::tryContinueQueryExecutionAfterFree(Int64 amount) diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 37de75f4848..79ed36cd7fa 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -36,6 +36,16 @@ struct OvercommitRatio class MemoryTracker; +enum class OvercommitResult +{ + NONE, + DISABLED, + MEMORY_FREED, + SELECTED, + TIMEOUTED, + NOT_ENOUGH_FREED, +}; + enum class QueryCancellationState { NONE = 0, // Hard limit is not reached, there is no selected query to kill. @@ -54,7 +64,7 @@ struct OvercommitTracker : boost::noncopyable { void setMaxWaitTime(UInt64 wait_time); - bool needToStopQuery(MemoryTracker * tracker, Int64 amount); + OvercommitResult needToStopQuery(MemoryTracker * tracker, Int64 amount); void tryContinueQueryExecutionAfterFree(Int64 amount); diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 7f3b9788c1f..72fefc3e31c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -192,6 +192,7 @@ M(RealTimeMicroseconds, "Total (wall clock) time spent in processing (queries and other tasks) threads (not that this is a sum).") \ M(UserTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in user space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \ M(SystemTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in OS kernel space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \ + M(MemoryOvercommitWaitTimeMicroseconds, "Total time spent in waiting for memory to be freed in OvercommitTracker.") \ M(SoftPageFaults, "") \ M(HardPageFaults, "") \ \ diff --git a/src/Common/tests/gtest_overcommit_tracker.cpp b/src/Common/tests/gtest_overcommit_tracker.cpp index 542af815842..c56ecec669f 100644 --- a/src/Common/tests/gtest_overcommit_tracker.cpp +++ b/src/Common/tests/gtest_overcommit_tracker.cpp @@ -56,7 +56,7 @@ void free_not_continue_test(T & overcommit_tracker) threads.push_back(std::thread( [&, i]() { - if (overcommit_tracker.needToStopQuery(&trackers[i], 100)) + if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } )); @@ -112,7 +112,7 @@ void free_continue_test(T & overcommit_tracker) threads.push_back(std::thread( [&, i]() { - if (overcommit_tracker.needToStopQuery(&trackers[i], 100)) + if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } )); @@ -168,7 +168,7 @@ void free_continue_and_alloc_test(T & overcommit_tracker) threads.push_back(std::thread( [&, i]() { - if (overcommit_tracker.needToStopQuery(&trackers[i], 100)) + if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } )); @@ -181,7 +181,7 @@ void free_continue_and_alloc_test(T & overcommit_tracker) MemoryTracker failed; std::this_thread::sleep_for(1000ms); overcommit_tracker.tryContinueQueryExecutionAfterFree(5000); - stopped_next = overcommit_tracker.needToStopQuery(&failed, 100); + stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED; } ).join(); @@ -228,7 +228,7 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker) threads.push_back(std::thread( [&, i]() { - if (overcommit_tracker.needToStopQuery(&trackers[i], 100)) + if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } )); @@ -241,7 +241,7 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker) MemoryTracker failed; std::this_thread::sleep_for(1000ms); overcommit_tracker.tryContinueQueryExecutionAfterFree(5000); - stopped_next = overcommit_tracker.needToStopQuery(&failed, 100); + stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED; } )); @@ -296,7 +296,7 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker) threads.push_back(std::thread( [&, i]() { - if (overcommit_tracker.needToStopQuery(&trackers[i], 100)) + if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } )); @@ -309,7 +309,7 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker) MemoryTracker failed; std::this_thread::sleep_for(1000ms); overcommit_tracker.tryContinueQueryExecutionAfterFree(5000); - stopped_next = overcommit_tracker.needToStopQuery(&failed, 100); + stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED; } )); @@ -364,7 +364,7 @@ void free_continue_2_test(T & overcommit_tracker) threads.push_back(std::thread( [&, i]() { - if (overcommit_tracker.needToStopQuery(&trackers[i], 100)) + if (overcommit_tracker.needToStopQuery(&trackers[i], 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } )); @@ -415,7 +415,7 @@ void query_stop_not_continue_test(T & overcommit_tracker) auto thread = std::thread( [&]() { - if (overcommit_tracker.needToStopQuery(&another, 100)) + if (overcommit_tracker.needToStopQuery(&another, 100) != OvercommitResult::MEMORY_FREED) ++need_to_stop; } ); From ea60a614d2dd166906efaff26f67e25c571b8a31 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 May 2022 20:33:13 +0200 Subject: [PATCH 004/101] Decrease namespace indent --- src/Functions/Regexps.h | 446 ++++++++++++++++++++-------------------- 1 file changed, 224 insertions(+), 222 deletions(-) diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index dc94b75211c..2611afedc14 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -38,254 +38,256 @@ namespace ErrorCodes namespace Regexps { - using Regexp = OptimizedRegularExpressionSingleThreaded; - using Pool = ObjectPoolMap; +using Regexp = OptimizedRegularExpressionSingleThreaded; +using Pool = ObjectPoolMap; - template - inline Regexp createRegexp(const std::string & pattern, int flags) +template +inline Regexp createRegexp(const std::string & pattern, int flags) +{ + if constexpr (like) + return {likePatternToRegexp(pattern), flags}; + else + return {pattern, flags}; +} + +template +inline int buildRe2Flags() +{ + int flags = OptimizedRegularExpression::RE_DOT_NL; + if constexpr (no_capture) + flags |= OptimizedRegularExpression::RE_NO_CAPTURE; + if constexpr (case_insensitive) + flags |= OptimizedRegularExpression::RE_CASELESS; + return flags; +} + +/** Returns holder of an object from Pool. + * You must hold the ownership while using the object. + * In destructor, it returns the object back to the Pool for further reuse. + */ +template +inline Pool::Pointer get(const std::string & pattern) +{ + /// the Singleton is thread-safe in C++11 + static Pool known_regexps; /// Different variables for different pattern parameters. + + return known_regexps.get(pattern, [&pattern] { - if constexpr (like) - return {likePatternToRegexp(pattern), flags}; - else - return {pattern, flags}; - } + const int flags = buildRe2Flags(); + ProfileEvents::increment(ProfileEvents::RegexpCreated); + return new Regexp{createRegexp(pattern, flags)}; + }); +} - template - inline int buildRe2Flags() - { - int flags = OptimizedRegularExpression::RE_DOT_NL; - if constexpr (no_capture) - flags |= OptimizedRegularExpression::RE_NO_CAPTURE; - if constexpr (case_insensitive) - flags |= OptimizedRegularExpression::RE_CASELESS; - return flags; - } - - /** Returns holder of an object from Pool. - * You must hold the ownership while using the object. - * In destructor, it returns the object back to the Pool for further reuse. - */ - template - inline Pool::Pointer get(const std::string & pattern) - { - /// the Singleton is thread-safe in C++11 - static Pool known_regexps; /// Different variables for different pattern parameters. - - return known_regexps.get(pattern, [&pattern] - { - const int flags = buildRe2Flags(); - ProfileEvents::increment(ProfileEvents::RegexpCreated); - return new Regexp{createRegexp(pattern, flags)}; - }); - } } #if USE_HYPERSCAN namespace MultiRegexps { - template - struct HyperscanDeleter +template +struct HyperscanDeleter +{ + template + void operator()(T * ptr) const { - template - void operator()(T * ptr) const - { - deleter(ptr); - } - }; + deleter(ptr); + } +}; - /// Helper unique pointers to correctly delete the allocated space when hyperscan cannot compile something and we throw an exception. - using CompilerError = std::unique_ptr>; - using ScratchPtr = std::unique_ptr>; - using DataBasePtr = std::unique_ptr>; +/// Helper unique pointers to correctly delete the allocated space when hyperscan cannot compile something and we throw an exception. +using CompilerError = std::unique_ptr>; +using ScratchPtr = std::unique_ptr>; +using DataBasePtr = std::unique_ptr>; - /// Database is thread safe across multiple threads and Scratch is not but we can copy it whenever we use it in the searcher. - class Regexps +/// Database is thread safe across multiple threads and Scratch is not but we can copy it whenever we use it in the searcher. +class Regexps +{ +public: + Regexps(hs_database_t * db_, hs_scratch_t * scratch_) : db{db_}, scratch{scratch_} { } + + hs_database_t * getDB() const { return db.get(); } + hs_scratch_t * getScratch() const { return scratch.get(); } + +private: + DataBasePtr db; + ScratchPtr scratch; +}; + +class RegexpsConstructor +{ +public: + RegexpsConstructor() = default; + + void setConstructor(std::function constructor_) { constructor = std::move(constructor_); } + + Regexps * operator()() { - public: - Regexps(hs_database_t * db_, hs_scratch_t * scratch_) : db{db_}, scratch{scratch_} { } - - hs_database_t * getDB() const { return db.get(); } - hs_scratch_t * getScratch() const { return scratch.get(); } - - private: - DataBasePtr db; - ScratchPtr scratch; - }; - - class RegexpsConstructor - { - public: - RegexpsConstructor() = default; - - void setConstructor(std::function constructor_) { constructor = std::move(constructor_); } - - Regexps * operator()() - { - std::unique_lock lock(mutex); - if (regexp) - return &*regexp; - regexp = constructor(); + std::unique_lock lock(mutex); + if (regexp) return &*regexp; - } + regexp = constructor(); + return &*regexp; + } - private: - std::function constructor; - std::optional regexp; - std::mutex mutex; - }; +private: + std::function constructor; + std::optional regexp; + std::mutex mutex; +}; - struct Pool +struct Pool +{ + /// Mutex for finding in map. + std::mutex mutex; + /// Patterns + possible edit_distance to database and scratch. + std::map, std::optional>, RegexpsConstructor> storage; +}; + +template +inline Regexps constructRegexps(const std::vector & str_patterns, std::optional edit_distance) +{ + (void)edit_distance; + /// Common pointers + std::vector patterns; + std::vector flags; + + /// Pointer for external edit distance compilation + std::vector ext_exprs; + std::vector ext_exprs_ptrs; + + patterns.reserve(str_patterns.size()); + flags.reserve(str_patterns.size()); + + if constexpr (CompileForEditDistance) { - /// Mutex for finding in map. - std::mutex mutex; - /// Patterns + possible edit_distance to database and scratch. - std::map, std::optional>, RegexpsConstructor> storage; - }; + ext_exprs.reserve(str_patterns.size()); + ext_exprs_ptrs.reserve(str_patterns.size()); + } - template - inline Regexps constructRegexps(const std::vector & str_patterns, std::optional edit_distance) + for (const StringRef ref : str_patterns) { - (void)edit_distance; - /// Common pointers - std::vector patterns; - std::vector flags; - - /// Pointer for external edit distance compilation - std::vector ext_exprs; - std::vector ext_exprs_ptrs; - - patterns.reserve(str_patterns.size()); - flags.reserve(str_patterns.size()); - + patterns.push_back(ref.data); + /* Flags below are the pattern matching flags. + * HS_FLAG_DOTALL is a compile flag where matching a . will not exclude newlines. This is a good + * performance practice according to Hyperscan API. https://intel.github.io/hyperscan/dev-reference/performance.html#dot-all-mode + * HS_FLAG_ALLOWEMPTY is a compile flag where empty strings are allowed to match. + * HS_FLAG_UTF8 is a flag where UTF8 literals are matched. + * HS_FLAG_SINGLEMATCH is a compile flag where each pattern match will be returned only once. it is a good performance practice + * as it is said in the Hyperscan documentation. https://intel.github.io/hyperscan/dev-reference/performance.html#single-match-flag + */ + flags.push_back(HS_FLAG_DOTALL | HS_FLAG_SINGLEMATCH | HS_FLAG_ALLOWEMPTY | HS_FLAG_UTF8); if constexpr (CompileForEditDistance) { - ext_exprs.reserve(str_patterns.size()); - ext_exprs_ptrs.reserve(str_patterns.size()); + /// Hyperscan currently does not support UTF8 matching with edit distance. + flags.back() &= ~HS_FLAG_UTF8; + ext_exprs.emplace_back(); + /// HS_EXT_FLAG_EDIT_DISTANCE is a compile flag responsible for Levenstein distance. + ext_exprs.back().flags = HS_EXT_FLAG_EDIT_DISTANCE; + ext_exprs.back().edit_distance = edit_distance.value(); + ext_exprs_ptrs.push_back(&ext_exprs.back()); } - - for (const StringRef ref : str_patterns) - { - patterns.push_back(ref.data); - /* Flags below are the pattern matching flags. - * HS_FLAG_DOTALL is a compile flag where matching a . will not exclude newlines. This is a good - * performance practice according to Hyperscan API. https://intel.github.io/hyperscan/dev-reference/performance.html#dot-all-mode - * HS_FLAG_ALLOWEMPTY is a compile flag where empty strings are allowed to match. - * HS_FLAG_UTF8 is a flag where UTF8 literals are matched. - * HS_FLAG_SINGLEMATCH is a compile flag where each pattern match will be returned only once. it is a good performance practice - * as it is said in the Hyperscan documentation. https://intel.github.io/hyperscan/dev-reference/performance.html#single-match-flag - */ - flags.push_back(HS_FLAG_DOTALL | HS_FLAG_SINGLEMATCH | HS_FLAG_ALLOWEMPTY | HS_FLAG_UTF8); - if constexpr (CompileForEditDistance) - { - /// Hyperscan currently does not support UTF8 matching with edit distance. - flags.back() &= ~HS_FLAG_UTF8; - ext_exprs.emplace_back(); - /// HS_EXT_FLAG_EDIT_DISTANCE is a compile flag responsible for Levenstein distance. - ext_exprs.back().flags = HS_EXT_FLAG_EDIT_DISTANCE; - ext_exprs.back().edit_distance = edit_distance.value(); - ext_exprs_ptrs.push_back(&ext_exprs.back()); - } - } - hs_database_t * db = nullptr; - hs_compile_error_t * compile_error; - - std::unique_ptr ids; - - /// We mark the patterns to provide the callback results. - if constexpr (save_indices) - { - ids.reset(new unsigned int[patterns.size()]); - for (size_t i = 0; i < patterns.size(); ++i) - ids[i] = i + 1; - } - - hs_error_t err; - if constexpr (!CompileForEditDistance) - err = hs_compile_multi( - patterns.data(), - flags.data(), - ids.get(), - patterns.size(), - HS_MODE_BLOCK, - nullptr, - &db, - &compile_error); - else - err = hs_compile_ext_multi( - patterns.data(), - flags.data(), - ids.get(), - ext_exprs_ptrs.data(), - patterns.size(), - HS_MODE_BLOCK, - nullptr, - &db, - &compile_error); - - if (err != HS_SUCCESS) - { - /// CompilerError is a unique_ptr, so correct memory free after the exception is thrown. - CompilerError error(compile_error); - - if (error->expression < 0) - throw Exception(String(error->message), ErrorCodes::LOGICAL_ERROR); - else - throw Exception( - "Pattern '" + str_patterns[error->expression] + "' failed with error '" + String(error->message), - ErrorCodes::BAD_ARGUMENTS); - } - - ProfileEvents::increment(ProfileEvents::RegexpCreated); - - /// We allocate the scratch space only once, then copy it across multiple threads with hs_clone_scratch - /// function which is faster than allocating scratch space each time in each thread. - hs_scratch_t * scratch = nullptr; - err = hs_alloc_scratch(db, &scratch); - - /// If not HS_SUCCESS, it is guaranteed that the memory would not be allocated for scratch. - if (err != HS_SUCCESS) - throw Exception("Could not allocate scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY); - - return Regexps{db, scratch}; } + hs_database_t * db = nullptr; + hs_compile_error_t * compile_error; - /// If CompileForEditDistance is False, edit_distance must be nullopt - /// Also, we use templates here because each instantiation of function - /// template has its own copy of local static variables which must not be the same - /// for different hyperscan compilations. - template - inline Regexps * get(const std::vector & patterns, std::optional edit_distance) + std::unique_ptr ids; + + /// We mark the patterns to provide the callback results. + if constexpr (save_indices) { - /// C++11 has thread-safe function-local static on most modern compilers. - static Pool known_regexps; /// Different variables for different pattern parameters. - - std::vector str_patterns; - str_patterns.reserve(patterns.size()); - for (const StringRef & ref : patterns) - str_patterns.push_back(ref.toString()); - - /// Get the lock for finding database. - std::unique_lock lock(known_regexps.mutex); - - auto it = known_regexps.storage.find({str_patterns, edit_distance}); - - /// If not found, compile and let other threads wait. - if (known_regexps.storage.end() == it) - { - it = known_regexps.storage - .emplace(std::piecewise_construct, std::make_tuple(std::move(str_patterns), edit_distance), std::make_tuple()) - .first; - it->second.setConstructor([&str_patterns = it->first.first, edit_distance]() - { - return constructRegexps(str_patterns, edit_distance); - }); - } - - /// Unlock before possible construction. - lock.unlock(); - return it->second(); + ids.reset(new unsigned int[patterns.size()]); + for (size_t i = 0; i < patterns.size(); ++i) + ids[i] = i + 1; } + + hs_error_t err; + if constexpr (!CompileForEditDistance) + err = hs_compile_multi( + patterns.data(), + flags.data(), + ids.get(), + patterns.size(), + HS_MODE_BLOCK, + nullptr, + &db, + &compile_error); + else + err = hs_compile_ext_multi( + patterns.data(), + flags.data(), + ids.get(), + ext_exprs_ptrs.data(), + patterns.size(), + HS_MODE_BLOCK, + nullptr, + &db, + &compile_error); + + if (err != HS_SUCCESS) + { + /// CompilerError is a unique_ptr, so correct memory free after the exception is thrown. + CompilerError error(compile_error); + + if (error->expression < 0) + throw Exception(String(error->message), ErrorCodes::LOGICAL_ERROR); + else + throw Exception( + "Pattern '" + str_patterns[error->expression] + "' failed with error '" + String(error->message), + ErrorCodes::BAD_ARGUMENTS); + } + + ProfileEvents::increment(ProfileEvents::RegexpCreated); + + /// We allocate the scratch space only once, then copy it across multiple threads with hs_clone_scratch + /// function which is faster than allocating scratch space each time in each thread. + hs_scratch_t * scratch = nullptr; + err = hs_alloc_scratch(db, &scratch); + + /// If not HS_SUCCESS, it is guaranteed that the memory would not be allocated for scratch. + if (err != HS_SUCCESS) + throw Exception("Could not allocate scratch space for hyperscan", ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + return Regexps{db, scratch}; +} + +/// If CompileForEditDistance is False, edit_distance must be nullopt +/// Also, we use templates here because each instantiation of function +/// template has its own copy of local static variables which must not be the same +/// for different hyperscan compilations. +template +inline Regexps * get(const std::vector & patterns, std::optional edit_distance) +{ + /// C++11 has thread-safe function-local static on most modern compilers. + static Pool known_regexps; /// Different variables for different pattern parameters. + + std::vector str_patterns; + str_patterns.reserve(patterns.size()); + for (const StringRef & ref : patterns) + str_patterns.push_back(ref.toString()); + + /// Get the lock for finding database. + std::unique_lock lock(known_regexps.mutex); + + auto it = known_regexps.storage.find({str_patterns, edit_distance}); + + /// If not found, compile and let other threads wait. + if (known_regexps.storage.end() == it) + { + it = known_regexps.storage + .emplace(std::piecewise_construct, std::make_tuple(std::move(str_patterns), edit_distance), std::make_tuple()) + .first; + it->second.setConstructor([&str_patterns = it->first.first, edit_distance]() + { + return constructRegexps(str_patterns, edit_distance); + }); + } + + /// Unlock before possible construction. + lock.unlock(); + return it->second(); +} + } #endif // USE_HYPERSCAN From 49934a3dc865cc8131d94de4592d3bd4f21150c0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 25 May 2022 21:22:45 +0200 Subject: [PATCH 005/101] Cache compiled regexps when evaluating non-const needles Needles in a (non-const) needle column may repeat and this commit allows to skip compilation for known needles. Out of the different design alternatives (see below, if someone is interested), we now maintain - one global pattern cache, - with a fixed size of 42k elements currently, - and use LRU as eviction strategy. ------------------------------------------------------------------------ (sorry for the wall of text, dumping it here not for reading but just for reference) Write-up about considered design alternatives: 1. Keep the current global cache of const needles. For non-const needles, probe the cache but don't store values in it. Pros: need to maintain just a single cache, no problem with cache pollution assuming there are few distinct constant needles Cons: only useful if a non-const needle occurred as already as a const needle --> overall too simplistic 2. Keep the current global cache for const needles. For non-const needles, create a local (e.g. per-query) cache Pros: unlike (1.), non-const needles can be skipped even if they did not occur yet, no pollution of the const pattern cache when there are very many non-const needles (e.g. large / highly distinct needle columns). Cons: caches may explode "horizontally", i.e. we'll end up with the const cache + caches for Q1, Q2, ... QN, this makes it harder to control the overall space consumption, also patterns residing in different caches cannot be reused between queries, another difficulty is that the concept of "query" does not really exist at matching level - there are only column chunks and we'd potentially end up with 1 cache / chunk 3. Queries with const and non-const needles insert into the same global cache. Pros: the advantages of (2.) + allows to reuse compiled patterns accross parallel queries Cons: needs an eviction strategy to control cache size and pollution (and btw. (2.) also needs eviction strategies for the individual caches) 4. Queries with const needle use global cache, queries with non-const needle use a different global cache --> Overall similar to (3) but ignores the (likely) edge case that const and non-const needles overlap. In sum, (3.) seems the simplest and most beneficial approach. Eviction strategies: 0. Don't ever evict --> cache may grow infinitely and eventually make the system unusable (may even pose a DoS risk) 1. Flush the cache after a certain threshold is exceeded --> very simple but may lead to peridic performance drops 2. Use LRU --> more graceful performance degradation at threshold but comes with a (constant) performance overhead to maintain the LRU queue In sum, given that the pattern compilation in RE2 should be quite costly (pattern-to-DFA/NFA), LRU may be acceptable. --- src/Functions/FunctionsStringArray.h | 4 ++-- src/Functions/MatchImpl.h | 30 ++++++++++--------------- src/Functions/Regexps.h | 33 +++++++++++----------------- src/Functions/countMatches.h | 4 ++-- 4 files changed, 29 insertions(+), 42 deletions(-) diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index 2680816670f..6545c3e3549 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -448,7 +448,7 @@ public: class SplitByRegexpImpl { private: - Regexps::Pool::Pointer re; + Regexps::RegexpPtr re; OptimizedRegularExpression::MatchVec matches; Pos pos; @@ -532,7 +532,7 @@ public: class ExtractAllImpl { private: - Regexps::Pool::Pointer re; + Regexps::RegexpPtr re; OptimizedRegularExpression::MatchVec matches; size_t capture; diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 17bda74f8ab..9779eb8d608 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -166,7 +166,7 @@ struct MatchImpl } else { - auto regexp = Regexps::get(needle); + auto regexp = Regexps::get(needle); String required_substring; bool is_trivial; @@ -325,7 +325,7 @@ struct MatchImpl } else { - auto regexp = Regexps::get(needle); + auto regexp = Regexps::get(needle); String required_substring; bool is_trivial; @@ -479,22 +479,19 @@ struct MatchImpl } else { - // each row is expected to contain a different like/re2 pattern - // --> bypass the regexp cache, instead construct the pattern on-the-fly - const int flags = Regexps::buildRe2Flags(); - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle, flags)); + auto regexp = Regexps::get(needle); - regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); + regexp->getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); if (required_substr.empty()) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + if (!regexp->getRE2()) /// An empty regexp. Always matches. { res[i] = !negate; } else { - const bool match = regexp.getRE2()->Match( + const bool match = regexp->getRE2()->Match( {reinterpret_cast(cur_haystack_data), cur_haystack_length}, 0, cur_haystack_length, @@ -524,7 +521,7 @@ struct MatchImpl const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; const size_t end_pos = cur_haystack_length; - const bool match2 = regexp.getRE2()->Match( + const bool match2 = regexp->getRE2()->Match( {reinterpret_cast(cur_haystack_data), cur_haystack_length}, start_pos, end_pos, @@ -593,22 +590,19 @@ struct MatchImpl } else { - // each row is expected to contain a different like/re2 pattern - // --> bypass the regexp cache, instead construct the pattern on-the-fly - const int flags = Regexps::buildRe2Flags(); - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle, flags)); + auto regexp = Regexps::get(needle); - regexp.getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); + regexp->getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); if (required_substr.empty()) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + if (!regexp->getRE2()) /// An empty regexp. Always matches. { res[i] = !negate; } else { - const bool match = regexp.getRE2()->Match( + const bool match = regexp->getRE2()->Match( {reinterpret_cast(cur_haystack_data), cur_haystack_length}, 0, cur_haystack_length, @@ -638,7 +632,7 @@ struct MatchImpl const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; const size_t end_pos = cur_haystack_length; - const bool match2 = regexp.getRE2()->Match( + const bool match2 = regexp->getRE2()->Match( {reinterpret_cast(cur_haystack_data), cur_haystack_length}, start_pos, end_pos, diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index 2611afedc14..be3ce6cdeee 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -39,16 +39,8 @@ namespace ErrorCodes namespace Regexps { using Regexp = OptimizedRegularExpressionSingleThreaded; -using Pool = ObjectPoolMap; - -template -inline Regexp createRegexp(const std::string & pattern, int flags) -{ - if constexpr (like) - return {likePatternToRegexp(pattern), flags}; - else - return {pattern, flags}; -} +using Cache = LRUCache; +using RegexpPtr = Cache::MappedPtr; template inline int buildRe2Flags() @@ -61,22 +53,23 @@ inline int buildRe2Flags() return flags; } -/** Returns holder of an object from Pool. - * You must hold the ownership while using the object. - * In destructor, it returns the object back to the Pool for further reuse. - */ +/// Probes the cache of known compiled regexps for the given string pattern and returns a compiled regexp if +/// found. Otherwise, a new cache entry is created. template -inline Pool::Pointer get(const std::string & pattern) +inline RegexpPtr get(const String & pattern) { - /// the Singleton is thread-safe in C++11 - static Pool known_regexps; /// Different variables for different pattern parameters. + static Cache known_regexps(42'000); - return known_regexps.get(pattern, [&pattern] + auto [regexp_ptr, _] = known_regexps.getOrSet(pattern, [&pattern]() { const int flags = buildRe2Flags(); ProfileEvents::increment(ProfileEvents::RegexpCreated); - return new Regexp{createRegexp(pattern, flags)}; + if constexpr (like) + return std::make_shared(likePatternToRegexp(pattern), flags); + else + return std::make_shared(pattern, flags); }); + return regexp_ptr; } } diff --git a/src/Functions/countMatches.h b/src/Functions/countMatches.h index 6d60ca94c18..1d43b66d867 100644 --- a/src/Functions/countMatches.h +++ b/src/Functions/countMatches.h @@ -55,7 +55,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const ColumnConst * column_pattern = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); - Regexps::Pool::Pointer re = Regexps::get(column_pattern->getValue()); + Regexps::RegexpPtr re = Regexps::get(column_pattern->getValue()); OptimizedRegularExpression::MatchVec matches; const IColumn * column_haystack = arguments[0].column.get(); @@ -95,7 +95,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()"); } - static uint64_t countMatches(StringRef src, Regexps::Pool::Pointer & re, OptimizedRegularExpression::MatchVec & matches) + static uint64_t countMatches(StringRef src, Regexps::RegexpPtr & re, OptimizedRegularExpression::MatchVec & matches) { /// Only one match is required, no need to copy more. static const unsigned matches_limit = 1; From 85c7b55b816674c9e653ba3c9c2ba4671e78effd Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 26 May 2022 16:54:46 +0200 Subject: [PATCH 006/101] Slightly better jepsen tests --- tests/jepsen.clickhouse-keeper/resources/keeper_config.xml | 1 + .../src/jepsen/clickhouse_keeper/db.clj | 6 ------ 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml index 8c574718d88..2ab747fbd71 100644 --- a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml @@ -5,6 +5,7 @@ trace /var/log/clickhouse-keeper/clickhouse-keeper.log /var/log/clickhouse-keeper/clickhouse-keeper.err.log + never diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index 745d88e97f7..22c8bfd348d 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -10,11 +10,6 @@ [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu])) -(defn get-clickhouse-sky - [version] - (c/exec :sky :get :-d common-prefix :-N :Backbone version) - (str common-prefix "/clickhouse")) - (defn get-clickhouse-url [url] (non-precise-cached-wget! url)) @@ -27,7 +22,6 @@ [source] (info "Downloading clickhouse from" source) (cond - (clojure.string/starts-with? source "rbtorrent:") (get-clickhouse-sky source) (clojure.string/starts-with? source "http") (get-clickhouse-url source) (.exists (io/file source)) (get-clickhouse-scp source) :else (throw (Exception. (str "Don't know how to download clickhouse from" source))))) From 25884c68f15967ec5dcd7021db5d795647e0e3a4 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 26 May 2022 20:46:26 -0400 Subject: [PATCH 007/101] http named collection source implemented for dictionary --- src/Dictionaries/HTTPDictionarySource.cpp | 71 +++++++++++++------ .../ExternalDataSourceConfiguration.cpp | 57 +++++++++++++++ .../ExternalDataSourceConfiguration.h | 6 ++ 3 files changed, 111 insertions(+), 23 deletions(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index cf8b60f3681..8f7ca5e7a51 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include "DictionarySourceFactory.h" @@ -228,45 +229,69 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) if (dict_struct.has_expressions) throw Exception(ErrorCodes::LOGICAL_ERROR, "Dictionary source of type `http` does not support attribute expressions"); - auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); - - const auto & settings_config_prefix = config_prefix + ".http"; - const auto & credentials_prefix = settings_config_prefix + ".credentials"; - + auto settings_config_prefix = config_prefix + ".http"; Poco::Net::HTTPBasicCredentials credentials; - - if (config.has(credentials_prefix)) - { - credentials.setUsername(config.getString(credentials_prefix + ".user", "")); - credentials.setPassword(config.getString(credentials_prefix + ".password", "")); - } - - const auto & headers_prefix = settings_config_prefix + ".headers"; ReadWriteBufferFromHTTP::HTTPHeaderEntries header_entries; + String url; + String format; - if (config.has(headers_prefix)) + auto named_collection = created_from_ddl + ? getURLBasedDataSourceConfiguration(config, settings_config_prefix, global_context) + : std::nullopt; + if (named_collection) { - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(headers_prefix, config_keys); + url = named_collection->configuration.url; + format = named_collection->configuration.format; - header_entries.reserve(config_keys.size()); - for (const auto & key : config_keys) + credentials.setUsername(named_collection->configuration.user); + credentials.setPassword(named_collection->configuration.password); + + header_entries.reserve(named_collection->configuration.headers.size()); + for (const auto & header : named_collection->configuration.headers) + header_entries.emplace_back(std::make_tuple(header.first, header.second.get())); + } + else + { + const auto & credentials_prefix = settings_config_prefix + ".credentials"; + + if (config.has(credentials_prefix)) { - const auto header_key = config.getString(headers_prefix + "." + key + ".name", ""); - const auto header_value = config.getString(headers_prefix + "." + key + ".value", ""); - header_entries.emplace_back(std::make_tuple(header_key, header_value)); + credentials.setUsername(config.getString(credentials_prefix + ".user", "")); + credentials.setPassword(config.getString(credentials_prefix + ".password", "")); } + + const auto & headers_prefix = settings_config_prefix + ".headers"; + + + if (config.has(headers_prefix)) + { + Poco::Util::AbstractConfiguration::Keys config_keys; + config.keys(headers_prefix, config_keys); + + header_entries.reserve(config_keys.size()); + for (const auto & key : config_keys) + { + const auto header_key = config.getString(headers_prefix + "." + key + ".name", ""); + const auto header_value = config.getString(headers_prefix + "." + key + ".value", ""); + header_entries.emplace_back(std::make_tuple(header_key, header_value)); + } + } + + url = config.getString(settings_config_prefix + ".url", ""); + format =config.getString(settings_config_prefix + ".format", ""); } auto configuration = HTTPDictionarySource::Configuration { - .url = config.getString(settings_config_prefix + ".url", ""), - .format =config.getString(settings_config_prefix + ".format", ""), + .url = url, + .format = format, .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), .header_entries = std::move(header_entries) //-V1030 }; + auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + return std::make_unique(dict_struct, configuration, credentials, sample_block, context, created_from_ddl); }; factory.registerSource("http", create_table_source); diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index abd20e6e5fd..55eff117d5e 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -248,6 +248,63 @@ std::optional getExternalDataSourceConfiguration( return std::nullopt; } +std::optional getURLBasedDataSourceConfiguration( + const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context) +{ + URLBasedDataSourceConfiguration configuration; + auto collection_name = dict_config.getString(dict_config_prefix + ".name", ""); + if (!collection_name.empty()) + { + const auto & config = context->getConfigRef(); + const auto & collection_prefix = fmt::format("named_collections.{}", collection_name); + + if (!config.has(collection_prefix)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "There is no collection named `{}` in config", collection_name); + + configuration.url = + dict_config.getString(dict_config_prefix + ".url", config.getString(collection_prefix + ".url", "")); + configuration.format = + dict_config.getString(dict_config_prefix + ".format", config.getString(collection_prefix + ".format", "")); + configuration.compression_method = + dict_config.getString(dict_config_prefix + ".compression", config.getString(collection_prefix + ".compression_method", "")); + configuration.structure = + dict_config.getString(dict_config_prefix + ".structure", config.getString(collection_prefix + ".structure", "")); + configuration.user = + dict_config.getString(dict_config_prefix + ".credentials.user", config.getString(collection_prefix + ".credentials.user", "")); + configuration.password = + dict_config.getString(dict_config_prefix + ".credentials.password", config.getString(collection_prefix + ".credentials.password", "")); + + String headers_prefix; + const Poco::Util::AbstractConfiguration *headers_config = nullptr; + if (dict_config.has(dict_config_prefix + ".headers")) + { + headers_prefix = dict_config_prefix + ".headers"; + headers_config = &dict_config; + } + else + { + headers_prefix = collection_prefix + ".headers"; + headers_config = &config; + } + + if (headers_config) + { + Poco::Util::AbstractConfiguration::Keys header_keys; + headers_config->keys(headers_prefix, header_keys); + headers_prefix += "."; + for (const auto & header : header_keys) + { + const auto header_prefix = headers_prefix + header; + configuration.headers.emplace_back( + std::make_pair(headers_config->getString(header_prefix + ".name"), headers_config->getString(header_prefix + ".value"))); + } + } + + return URLBasedDataSourceConfig{ .configuration = configuration }; + } + + return std::nullopt; +} ExternalDataSourcesByPriority getExternalDataSourceConfigurationByPriority( const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context, HasConfigKeyFunc has_config_key) diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index dfac101e22d..19301c360f0 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -103,6 +103,9 @@ struct URLBasedDataSourceConfiguration String compression_method = "auto"; String structure = "auto"; + String user; + String password; + std::vector> headers; String http_method; @@ -129,6 +132,9 @@ struct URLBasedDataSourceConfig std::optional getURLBasedDataSourceConfiguration(const ASTs & args, ContextPtr context); +std::optional getURLBasedDataSourceConfiguration( + const Poco::Util::AbstractConfiguration & dict_config, const String & dict_config_prefix, ContextPtr context); + template bool getExternalDataSourceConfiguration(const ASTs & args, BaseSettings & settings, ContextPtr context); From 54d6f981222819ea9c915c151bf711ae8c07a1ba Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 27 May 2022 04:50:36 +0000 Subject: [PATCH 008/101] flush and shutdown temporary table before drop --- src/Interpreters/DatabaseCatalog.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 267564eb84c..8e41887ca44 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -119,7 +119,11 @@ TemporaryTableHolder & TemporaryTableHolder::operator=(TemporaryTableHolder && r TemporaryTableHolder::~TemporaryTableHolder() { if (id != UUIDHelpers::Nil) + { + auto table = getTable(); + table->flushAndShutdown(); temporary_tables->dropTable(getContext(), "_tmp_" + toString(id)); + } } StorageID TemporaryTableHolder::getGlobalTableID() const From e1772c571c12574f97d9e5252a5b86bc934491f3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 27 May 2022 07:10:20 +0000 Subject: [PATCH 009/101] Compress clickhouse logs --- .../src/jepsen/clickhouse_keeper/db.clj | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index 22c8bfd348d..0df704f0d4a 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -135,8 +135,13 @@ (do (info node "Coordination files exists, going to compress") (c/cd data-dir - (c/exec :tar :czf "coordination.tar.gz" "coordination"))))) - (let [common-logs [stderr-file (str logs-dir "/clickhouse-keeper.log") (str data-dir "/coordination.tar.gz")] + (c/exec :tar :czf "coordination.tar.gz" "coordination")))) + (if (cu/exists? (str logs-dir "/clickhouse-keeper.log")) + (do + (info node "Clickhouse logs exist, going to compress") + (c/cd logs-dir + (c/exec :gzip "clickhouse-keeper.log"))) (info node "Logs are missing"))) + (let [common-logs [stderr-file (str logs-dir "/clickhouse-keeper.log.gz") (str data-dir "/coordination.tar.gz")] gdb-log (str logs-dir "/gdb.log")] (if (cu/exists? (str logs-dir "/gdb.log")) (conj common-logs gdb-log) From ca67e67a7432582ee5ff5837d31432ab0e585d56 Mon Sep 17 00:00:00 2001 From: zhanglistar Date: Fri, 27 May 2022 15:52:04 +0800 Subject: [PATCH 010/101] Fix a typo --- src/Interpreters/ActionsDAG.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2fc9b51674f..eb073ee8752 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -39,7 +39,7 @@ void ActionsDAG::Node::toTree(JSONBuilder::JSONMap & map) const map.add("Result Type", result_type->getName()); if (!result_name.empty()) - map.add("Result Type", magic_enum::enum_name(type)); + map.add("Result Name", result_name); if (column) map.add("Column", column->getName()); From ff228d63e8eece4cdd504f000de1dd2260954941 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 27 May 2022 10:14:13 +0200 Subject: [PATCH 011/101] Fix typo --- docs/en/development/tests.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/tests.md b/docs/en/development/tests.md index f9c7ae37157..be361bd1e3f 100644 --- a/docs/en/development/tests.md +++ b/docs/en/development/tests.md @@ -81,7 +81,7 @@ $ ./src/unit_tests_dbms --gtest_filter=LocalAddress* ## Performance Tests {#performance-tests} -Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Tests are located at `tests/performance`. Each test is represented by `.xml` file with description of test case. Tests are run with `docker/tests/performance-comparison` tool . See the readme file for invocation. +Performance tests allow to measure and compare performance of some isolated part of ClickHouse on synthetic queries. Tests are located at `tests/performance`. Each test is represented by `.xml` file with description of test case. Tests are run with `docker/test/performance-comparison` tool . See the readme file for invocation. Each test run one or multiple queries (possibly with combinations of parameters) in a loop. From 7ccf4f4db73fc5f28efeb36427a2eae0af05dcc3 Mon Sep 17 00:00:00 2001 From: PigInCloud <44889745+yjant@users.noreply.github.com> Date: Fri, 27 May 2022 18:30:20 +0800 Subject: [PATCH 012/101] Update insert-into.md I translated some untranslated text into Chinese --- .../sql-reference/statements/insert-into.md | 68 ++++++++++++++++++- 1 file changed, 67 insertions(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index 928107fa2b2..4f958e31b18 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -71,7 +71,7 @@ INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set INSERT INTO [db.]table [(c1, c2, c3)] FORMAT Values (v11, v12, v13), (v21, v22, v23), ... ``` -ClickHouse会清除数据前所有的空白字符与一行摘要信息(如果需要的话)。所以在进行查询时,我们建议您将数据放入到输入输出格式名称后的新的一行中去(如果数据是以空白字符开始的,这将非常重要)。 +ClickHouse会清除数据前所有的空白字符与一个换行符(如果有换行符的话)。所以在进行查询时,我们建议您将数据放入到输入输出格式名称后的新的一行中去(如果数据是以空白字符开始的,这将非常重要)。 示例: @@ -83,6 +83,10 @@ INSERT INTO t FORMAT TabSeparated 在使用命令行客户端或HTTP客户端时,你可以将具体的查询语句与数据分开发送。更多具体信息,请参考«[客户端](../../interfaces/index.md#interfaces)»部分。 +### 限制 {#constraints} + +如果表中有一些[限制](../../sql-reference/statements/create/table.md#constraints),,数据插入时会逐行进行数据校验,如果这里面包含了不符合限制条件的数据,服务将会抛出包含限制信息的异常,这个语句也会被停止执行。 + ### 使用`SELECT`的结果写入 {#insert_query_insert-select} ``` sql @@ -96,6 +100,66 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... 系统不支持的其他用于修改数据的查询:`UPDATE`, `DELETE`, `REPLACE`, `MERGE`, `UPSERT`, `INSERT UPDATE`。 但是,您可以使用 `ALTER TABLE ... DROP PARTITION`查询来删除一些旧的数据。 +如果 `SELECT` 查询中包含了 [input()](../../sql-reference/table-functions/input.md) 函数,那么 `FORMAT` 必须出现在查询语句的最后。 + +如果某一列限制了值不能是NULL,那么插入NULL的时候就会插入这个列类型的默认数据,可以通过设置 [insert_null_as_default](../../operations/settings/settings.md#insert_null_as_default) 插入NULL。 + +### 从文件向表中插入数据 {#inserting-data-from-a-file} + +``` sql +INSERT INTO [db.]table [(c1, c2, c3)] FROM INFILE file_name [COMPRESSION type] FORMAT format_name +``` +使用上面的语句可以从客户端的文件上读取数据并插入表中,`file_name` 和 `type` 都是 `String` 类型,输入文件的[格式](../../interfaces/formats.md) 一定要在 `FORMAT` 语句中设置。 + +支持读取压缩文件。默认会去读文件的拓展名作为文件的压缩方式,或者也可以在 `COMPRESSION` 语句中指明,支持的文件压缩格式如下:`'none'`, `'gzip'`, `'deflate'`, `'br'`, `'xz'`, `'zstd'`, `'lz4'`, `'bz2'`。 + +这个功能在 [command-line client](../../interfaces/cli.md) 和 [clickhouse-local](../../operations/utilities/clickhouse-local.md) 是可用的。 + +**样例** + +```bash +echo 1,A > input.csv ; echo 2,B >> input.csv +clickhouse-client --query="CREATE TABLE table_from_file (id UInt32, text String) ENGINE=MergeTree() ORDER BY id;" +clickhouse-client --query="INSERT INTO table_from_file FROM INFILE 'input.csv' FORMAT CSV;" +clickhouse-client --query="SELECT * FROM table_from_file FORMAT PrettyCompact;" +``` + +结果: + +```text +┌─id─┬─text─┐ +│ 1 │ A │ +│ 2 │ B │ +└────┴──────┘ +``` + +### 插入表函数 {#inserting-into-table-function} + +数据可以通过 [table functions](../../sql-reference/table-functions/index.md) 方法插入。 +``` sql +INSERT INTO [TABLE] FUNCTION table_func ... +``` + +**例如** + +可以这样使用[remote](../../sql-reference/table-functions/index.md#remote) 表函数: + +``` sql +CREATE TABLE simple_table (id UInt32, text String) ENGINE=MergeTree() ORDER BY id; +INSERT INTO TABLE FUNCTION remote('localhost', default.simple_table) + VALUES (100, 'inserted via remote()'); +SELECT * FROM simple_table; +``` + +结果: + +``` text +┌──id─┬─text──────────────────┐ +│ 100 │ inserted via remote() │ +└─────┴───────────────────────┘ +``` + + ### 性能的注意事项 {#xing-neng-de-zhu-yi-shi-xiang} 在进行`INSERT`时将会对写入的数据进行一些处理,按照主键排序,按照月份对数据进行分区等。所以如果在您的写入数据中包含多个月份的混合数据时,将会显著的降低`INSERT`的性能。为了避免这种情况: @@ -108,4 +172,6 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... - 数据总是被实时的写入。 - 写入的数据已经按照时间排序。 +也可以异步的、小规模的插入数据,这些数据会被合并成多个批次,然后安全地写入到表中,通过设置[async_insert](../../operations/settings/settings.md#async-insert),可以使用异步插入的方式,请注意,异步插入的方式只支持HTTP协议,并且不支持数据去重。 + [来源文章](https://clickhouse.com/docs/en/query_language/insert_into/) From 60b9d81773d5a56486c5685ec3880d6caf207d1f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 27 May 2022 16:30:29 +0000 Subject: [PATCH 013/101] Remove global_memory_usage_overcommit_max_wait_microseconds --- .../settings.md | 10 ----- docs/en/operations/settings/settings.md | 2 +- programs/server/Server.cpp | 2 - src/Common/MemoryTracker.cpp | 8 ++++ src/Common/MemoryTracker.h | 10 +++++ src/Common/OvercommitTracker.cpp | 11 ++---- src/Common/OvercommitTracker.h | 4 -- src/Common/tests/gtest_overcommit_tracker.cpp | 38 ++++++++++++------- src/Core/Settings.h | 2 +- src/Interpreters/ProcessList.cpp | 3 +- .../configs/global_overcommit_tracker.xml | 1 - .../test_global_overcommit_tracker/test.py | 12 ++++-- 12 files changed, 57 insertions(+), 46 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index fd5c2a187b5..f235fba84f7 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1745,13 +1745,3 @@ Possible values: - Positive integer. Default value: `10000`. - -## global_memory_usage_overcommit_max_wait_microseconds {#global_memory_usage_overcommit_max_wait_microseconds} - -Sets maximum waiting time for global overcommit tracker. - -Possible values: - -- Positive integer. - -Default value: `200`. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 76fbc5f239d..9367d70507f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4279,7 +4279,7 @@ Maximum time thread will wait for memory to be freed in the case of memory overc If the timeout is reached and memory is not freed, an exception is thrown. Read more about [memory overcommit](memory-overcommit.md). -Default value: `200`. +Default value: `5000000`. ## memory_overcommit_ratio_denominator_for_user diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 18ab96983eb..2c6ffccd39d 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1095,8 +1095,6 @@ int Server::main(const std::vector & /*args*/) total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); - UInt64 max_overcommit_wait_time = config->getUInt64("global_memory_usage_overcommit_max_wait_microseconds", 5'000'000); - global_overcommit_tracker->setMaxWaitTime(max_overcommit_wait_time); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); // FIXME logging-related things need synchronization -- see the 'Logger * log' saved diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index b5a27543b4e..51f4c83dc23 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -82,6 +82,8 @@ namespace ProfileEvents extern const Event QueryMemoryLimitExceeded; } +using namespace std::chrono_literals; + static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); @@ -363,6 +365,12 @@ OvercommitRatio MemoryTracker::getOvercommitRatio(Int64 limit) } +void MemoryTracker::setOvercommitWaitingTime(UInt64 wait_time) +{ + max_wait_time.store(wait_time * 1us, std::memory_order_relaxed); +} + + void MemoryTracker::resetCounters() { amount.store(0, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 73af2ab8857..58bd3a460bd 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -73,6 +74,8 @@ private: /// This description will be used as prefix into log messages (if isn't nullptr) std::atomic description_ptr = nullptr; + std::atomic max_wait_time; + std::atomic overcommit_tracker = nullptr; bool updatePeak(Int64 will_be, bool log_memory_usage); @@ -186,6 +189,13 @@ public: OvercommitRatio getOvercommitRatio(); OvercommitRatio getOvercommitRatio(Int64 limit); + std::chrono::microseconds getOvercommitWaitingTime() + { + return max_wait_time.load(std::memory_order_relaxed); + } + + void setOvercommitWaitingTime(UInt64 wait_time); + void setOvercommitTracker(OvercommitTracker * tracker) noexcept { overcommit_tracker.store(tracker, std::memory_order_relaxed); diff --git a/src/Common/OvercommitTracker.cpp b/src/Common/OvercommitTracker.cpp index 0c03ba58e87..3cef72eb8b4 100644 --- a/src/Common/OvercommitTracker.cpp +++ b/src/Common/OvercommitTracker.cpp @@ -15,8 +15,7 @@ using namespace std::chrono_literals; constexpr std::chrono::microseconds ZERO_MICROSEC = 0us; OvercommitTracker::OvercommitTracker(std::mutex & global_mutex_) - : max_wait_time(ZERO_MICROSEC) - , picked_tracker(nullptr) + : picked_tracker(nullptr) , cancellation_state(QueryCancellationState::NONE) , global_mutex(global_mutex_) , freed_memory(0) @@ -24,12 +23,6 @@ OvercommitTracker::OvercommitTracker(std::mutex & global_mutex_) , allow_release(true) {} -void OvercommitTracker::setMaxWaitTime(UInt64 wait_time) -{ - std::lock_guard guard(overcommit_m); - max_wait_time = wait_time * 1us; -} - OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int64 amount) { // NOTE: Do not change the order of locks @@ -41,6 +34,8 @@ OvercommitResult OvercommitTracker::needToStopQuery(MemoryTracker * tracker, Int std::unique_lock global_lock(global_mutex); std::unique_lock lk(overcommit_m); + auto max_wait_time = tracker->getOvercommitWaitingTime(); + if (max_wait_time == ZERO_MICROSEC) return OvercommitResult::DISABLED; diff --git a/src/Common/OvercommitTracker.h b/src/Common/OvercommitTracker.h index 79ed36cd7fa..80aaed68e37 100644 --- a/src/Common/OvercommitTracker.h +++ b/src/Common/OvercommitTracker.h @@ -62,8 +62,6 @@ enum class QueryCancellationState // is killed to free memory. struct OvercommitTracker : boost::noncopyable { - void setMaxWaitTime(UInt64 wait_time); - OvercommitResult needToStopQuery(MemoryTracker * tracker, Int64 amount); void tryContinueQueryExecutionAfterFree(Int64 amount); @@ -82,8 +80,6 @@ protected: std::mutex overcommit_m; std::condition_variable cv; - std::chrono::microseconds max_wait_time; - // Specifies memory tracker of the chosen to stop query. // If soft limit is not set, all the queries which reach hard limit must stop. // This case is represented as picked tracker pointer is set to nullptr and diff --git a/src/Common/tests/gtest_overcommit_tracker.cpp b/src/Common/tests/gtest_overcommit_tracker.cpp index c56ecec669f..d832a73ffd9 100644 --- a/src/Common/tests/gtest_overcommit_tracker.cpp +++ b/src/Common/tests/gtest_overcommit_tracker.cpp @@ -40,15 +40,17 @@ static constexpr UInt64 WAIT_TIME = 4'000'000; template void free_not_continue_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - static constexpr size_t THREADS = 5; std::vector trackers(THREADS); + for (auto & tracker : trackers) + tracker.setOvercommitWaitingTime(WAIT_TIME); + std::atomic need_to_stop = 0; std::vector threads; threads.reserve(THREADS); MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); for (size_t i = 0; i < THREADS; ++i) @@ -96,15 +98,16 @@ TEST(OvercommitTracker, GlobalFreeNotContinue) template void free_continue_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - static constexpr size_t THREADS = 5; std::vector trackers(THREADS); + for (auto & tracker : trackers) + tracker.setOvercommitWaitingTime(WAIT_TIME); std::atomic need_to_stop = 0; std::vector threads; threads.reserve(THREADS); MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); for (size_t i = 0; i < THREADS; ++i) @@ -152,15 +155,16 @@ TEST(OvercommitTracker, GlobalFreeContinue) template void free_continue_and_alloc_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - static constexpr size_t THREADS = 5; std::vector trackers(THREADS); + for (auto & tracker : trackers) + tracker.setOvercommitWaitingTime(WAIT_TIME); std::atomic need_to_stop = 0; std::vector threads; threads.reserve(THREADS); MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); for (size_t i = 0; i < THREADS; ++i) @@ -179,6 +183,7 @@ void free_continue_and_alloc_test(T & overcommit_tracker) [&]() { MemoryTracker failed; + failed.setOvercommitWaitingTime(WAIT_TIME); std::this_thread::sleep_for(1000ms); overcommit_tracker.tryContinueQueryExecutionAfterFree(5000); stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED; @@ -212,15 +217,16 @@ TEST(OvercommitTracker, GlobalFreeContinueAndAlloc) template void free_continue_and_alloc_2_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - static constexpr size_t THREADS = 5; std::vector trackers(THREADS); + for (auto & tracker : trackers) + tracker.setOvercommitWaitingTime(WAIT_TIME); std::atomic need_to_stop = 0; std::vector threads; threads.reserve(THREADS); MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); for (size_t i = 0; i < THREADS; ++i) @@ -239,6 +245,7 @@ void free_continue_and_alloc_2_test(T & overcommit_tracker) [&]() { MemoryTracker failed; + failed.setOvercommitWaitingTime(WAIT_TIME); std::this_thread::sleep_for(1000ms); overcommit_tracker.tryContinueQueryExecutionAfterFree(5000); stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED; @@ -280,15 +287,16 @@ TEST(OvercommitTracker, GlobalFreeContinueAndAlloc2) template void free_continue_and_alloc_3_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - static constexpr size_t THREADS = 5; std::vector trackers(THREADS); + for (auto & tracker : trackers) + tracker.setOvercommitWaitingTime(WAIT_TIME); std::atomic need_to_stop = 0; std::vector threads; threads.reserve(THREADS); MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); for (size_t i = 0; i < THREADS; ++i) @@ -307,6 +315,7 @@ void free_continue_and_alloc_3_test(T & overcommit_tracker) [&]() { MemoryTracker failed; + failed.setOvercommitWaitingTime(WAIT_TIME); std::this_thread::sleep_for(1000ms); overcommit_tracker.tryContinueQueryExecutionAfterFree(5000); stopped_next = overcommit_tracker.needToStopQuery(&failed, 100) != OvercommitResult::MEMORY_FREED; @@ -348,15 +357,16 @@ TEST(OvercommitTracker, GlobalFreeContinueAndAlloc3) template void free_continue_2_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - static constexpr size_t THREADS = 5; std::vector trackers(THREADS); + for (auto & tracker : trackers) + tracker.setOvercommitWaitingTime(WAIT_TIME); std::atomic need_to_stop = 0; std::vector threads; threads.reserve(THREADS); MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); for (size_t i = 0; i < THREADS; ++i) @@ -404,14 +414,14 @@ TEST(OvercommitTracker, GlobalFreeContinue2) template void query_stop_not_continue_test(T & overcommit_tracker) { - overcommit_tracker.setMaxWaitTime(WAIT_TIME); - std::atomic need_to_stop = 0; MemoryTracker picked; + picked.setOvercommitWaitingTime(WAIT_TIME); overcommit_tracker.setCandidate(&picked); MemoryTracker another; + another.setOvercommitWaitingTime(WAIT_TIME); auto thread = std::thread( [&]() { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 29427c673ac..9111e1d80da 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -371,7 +371,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ \ - M(UInt64, memory_usage_overcommit_max_wait_microseconds, 200, "Maximum time thread will wait for memory to be freed in the case of memory overcommit on user level. If timeout is reached and memory is not freed, exception is thrown.", 0) \ + M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit on user level. If timeout is reached and memory is not freed, exception is thrown.", 0) \ \ M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ M(UInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \ diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 6c101143234..6b8894414ee 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -225,6 +225,8 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as if (settings.memory_tracker_fault_probability) thread_group->memory_tracker.setFaultProbability(settings.memory_tracker_fault_probability); + thread_group->memory_tracker.setOvercommitWaitingTime(settings.memory_usage_overcommit_max_wait_microseconds); + /// NOTE: Do not set the limit for thread-level memory tracker since it could show unreal values /// since allocation and deallocation could happen in different threads } @@ -244,7 +246,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_memory_tracker.setOrRaiseHardLimit(settings.max_memory_usage_for_user); user_process_list.user_memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); - user_process_list.user_overcommit_tracker.setMaxWaitTime(settings.memory_usage_overcommit_max_wait_microseconds); if (!user_process_list.user_throttler) { diff --git a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml index 590759bd15d..a05d8865a6b 100644 --- a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml +++ b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml @@ -1,4 +1,3 @@ 50000000 - 500 \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py index d3d56e82f38..093549249ce 100644 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -18,8 +18,8 @@ def start_cluster(): cluster.shutdown() -TEST_QUERY_A = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=1" -TEST_QUERY_B = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=2" +TEST_QUERY_A = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=1, memory_usage_overcommit_max_wait_microseconds=500" +TEST_QUERY_B = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=2, memory_usage_overcommit_max_wait_microseconds=500" def test_overcommited_is_killed(): @@ -46,8 +46,12 @@ def test_overcommited_is_killed(): finished = True assert ( - overcommited_killed and finished - ), "no overcommited task was killed or all tasks are killed" + overcommited_killed + ), "no overcommited task was killed" + + assert ( + finished + ), "all tasks are killed" node.query("DROP USER IF EXISTS A") node.query("DROP USER IF EXISTS B") From 41ef0044f068f02d9fac278ec07dad6826825687 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Fri, 27 May 2022 13:43:34 -0400 Subject: [PATCH 014/101] endpoint is added --- src/Dictionaries/HTTPDictionarySource.cpp | 13 ++++++++++++- src/Storages/ExternalDataSourceConfiguration.cpp | 2 ++ src/Storages/ExternalDataSourceConfiguration.h | 1 + 3 files changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 8f7ca5e7a51..17592a8d9da 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -233,6 +233,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) Poco::Net::HTTPBasicCredentials credentials; ReadWriteBufferFromHTTP::HTTPHeaderEntries header_entries; String url; + String endpoint; String format; auto named_collection = created_from_ddl @@ -241,6 +242,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) if (named_collection) { url = named_collection->configuration.url; + endpoint = named_collection->configuration.endpoint; format = named_collection->configuration.format; credentials.setUsername(named_collection->configuration.user); @@ -278,12 +280,21 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) } url = config.getString(settings_config_prefix + ".url", ""); + endpoint = config.getString(settings_config_prefix + ".endpoint", ""); format =config.getString(settings_config_prefix + ".format", ""); } + if (url.ends_with('/')) + { + if (endpoint.starts_with('/')) + url.pop_back(); + } + else if (!endpoint.empty() && !endpoint.starts_with('/')) + url.push_back('/'); + auto configuration = HTTPDictionarySource::Configuration { - .url = url, + .url = url + endpoint, .format = format, .update_field = config.getString(settings_config_prefix + ".update_field", ""), .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), diff --git a/src/Storages/ExternalDataSourceConfiguration.cpp b/src/Storages/ExternalDataSourceConfiguration.cpp index 55eff117d5e..f916ac8c2af 100644 --- a/src/Storages/ExternalDataSourceConfiguration.cpp +++ b/src/Storages/ExternalDataSourceConfiguration.cpp @@ -263,6 +263,8 @@ std::optional getURLBasedDataSourceConfiguration( configuration.url = dict_config.getString(dict_config_prefix + ".url", config.getString(collection_prefix + ".url", "")); + configuration.endpoint = + dict_config.getString(dict_config_prefix + ".endpoint", config.getString(collection_prefix + ".endpoint", "")); configuration.format = dict_config.getString(dict_config_prefix + ".format", config.getString(collection_prefix + ".format", "")); configuration.compression_method = diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 19301c360f0..4ed46e1b26c 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -99,6 +99,7 @@ getExternalDataSourceConfigurationByPriority(const Poco::Util::AbstractConfigura struct URLBasedDataSourceConfiguration { String url; + String endpoint; String format = "auto"; String compression_method = "auto"; String structure = "auto"; From db2fe33926af47aad3155407a5443768a307dd43 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Fri, 27 May 2022 20:43:59 +0200 Subject: [PATCH 015/101] Update test.py --- tests/integration/test_global_overcommit_tracker/test.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py index 093549249ce..871f9ca983e 100644 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -45,13 +45,8 @@ def test_overcommited_is_killed(): if err == "": finished = True - assert ( - overcommited_killed - ), "no overcommited task was killed" - - assert ( - finished - ), "all tasks are killed" + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" node.query("DROP USER IF EXISTS A") node.query("DROP USER IF EXISTS B") From 3ff32fe81dcdb86d555917e9ed5f7437996e4579 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 28 May 2022 12:30:05 -0400 Subject: [PATCH 016/101] test is added --- .../integration/test_storage_dict/__init__.py | 0 .../test_storage_dict/configs/conf.xml | 16 ++++++++ tests/integration/test_storage_dict/test.py | 40 +++++++++++++++++++ 3 files changed, 56 insertions(+) create mode 100644 tests/integration/test_storage_dict/__init__.py create mode 100644 tests/integration/test_storage_dict/configs/conf.xml create mode 100644 tests/integration/test_storage_dict/test.py diff --git a/tests/integration/test_storage_dict/__init__.py b/tests/integration/test_storage_dict/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_dict/configs/conf.xml b/tests/integration/test_storage_dict/configs/conf.xml new file mode 100644 index 00000000000..c2ecb518884 --- /dev/null +++ b/tests/integration/test_storage_dict/configs/conf.xml @@ -0,0 +1,16 @@ + + + + + http://nginx:80/test_dict + PUT + TSV + k String, v String + + + http://nginx:80/ + /test_dict + TabSeparated + + + diff --git a/tests/integration/test_storage_dict/test.py b/tests/integration/test_storage_dict/test.py new file mode 100644 index 00000000000..df224f08968 --- /dev/null +++ b/tests/integration/test_storage_dict/test.py @@ -0,0 +1,40 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +uuids = [] + + +@pytest.fixture(scope="module") +def cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "node1", main_configs=["configs/conf.xml"], with_nginx=True + ) + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + + +def test_storage_dict(cluster): + node1 = cluster.instances["node1"] + + node1.query( + f"insert into table function url(urldb) values ('foo', 'bar')" + ) + result = node1.query( + f"select * from url(urldb)" + ) + assert result.strip() == "foo\tbar" + + node1.query( + f"create dictionary dict (k String, v String) primary key k source(http(name urldict)) layout(complex_key_hashed()) lifetime(min 0 max 100)" + ) + result = node1.query( + f"select * from dict" + ) + assert result.strip() == "foo\tbar" From 4f07c684da00ead69d24f0aa04b68f64b39b72db Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 28 May 2022 12:45:53 -0400 Subject: [PATCH 017/101] style fix --- tests/integration/test_storage_dict/test.py | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_storage_dict/test.py b/tests/integration/test_storage_dict/test.py index df224f08968..a5270a42114 100644 --- a/tests/integration/test_storage_dict/test.py +++ b/tests/integration/test_storage_dict/test.py @@ -23,18 +23,12 @@ def cluster(): def test_storage_dict(cluster): node1 = cluster.instances["node1"] - node1.query( - f"insert into table function url(urldb) values ('foo', 'bar')" - ) - result = node1.query( - f"select * from url(urldb)" - ) + node1.query(f"insert into table function url(urldb) values ('foo', 'bar')") + result = node1.query(f"select * from url(urldb)") assert result.strip() == "foo\tbar" node1.query( f"create dictionary dict (k String, v String) primary key k source(http(name urldict)) layout(complex_key_hashed()) lifetime(min 0 max 100)" ) - result = node1.query( - f"select * from dict" - ) + result = node1.query(f"select * from dict") assert result.strip() == "foo\tbar" From e33d986cf1a842e66d9376fb9bb8bfc6fa83934c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 19:20:32 +0200 Subject: [PATCH 018/101] Revert "Merge pull request #34924 from azat/tests-overlaps" This reverts commit f13e436d40db2a15892653cc8baff61d09b73c7a, reversing changes made to aa3c05e9d49bcc8f2bcc6d79242742de6b16e51a. --- tests/clickhouse-test | 31 ----- tests/queries/0_stateless/00746_sql_fuzzy.sh | 3 +- .../00816_long_concurrent_alter_column.sh | 63 ++++++---- tests/queries/0_stateless/00900_orc_load.sh | 4 +- .../00909_kill_not_initialized_query.sh | 2 +- .../00941_system_columns_race_condition.sh | 51 ++++---- .../00991_system_parts_race_condition_long.sh | 69 ++++++----- ...tem_parts_race_condition_zookeeper_long.sh | 87 ++++++-------- ...tem_parts_race_condition_drop_zookeeper.sh | 110 +++++++++-------- .../01001_rename_merge_race_condition.sh | 31 ++--- ...lter_nullable_adaptive_granularity_long.sh | 41 ++++--- .../01003_kill_query_race_condition.sh | 41 ++++--- .../0_stateless/01004_rename_deadlock.sh | 45 ++++--- .../0_stateless/01005_rwr_shard_deadlock.sh | 31 +++-- .../0_stateless/01007_r1r2_w_r2r1_deadlock.sh | 43 ++++--- .../01013_sync_replica_timeout_zookeeper.sh | 2 +- ...rrent_recreate_reattach_and_show_tables.sh | 89 +++++++++----- ...8_ddl_dictionaries_concurrent_requrests.sh | 102 ++++++++-------- .../01019_alter_materialized_view_atomic.sh | 13 +- ...ent_move_partition_from_table_zookeeper.sh | 50 +++++--- .../01054_cache_dictionary_bunch_update.sh | 28 +++-- ...cache_dictionary_datarace_exception_ptr.sh | 22 ++-- ...076_parallel_alter_replicated_zookeeper.sh | 64 +++++----- ...arallel_alter_add_drop_column_zookeeper.sh | 67 ++++++----- ...9_parallel_alter_detach_table_zookeeper.sh | 71 +++++------ ...79_parallel_alter_modify_zookeeper_long.sh | 73 ++++++------ .../01085_max_distributed_connections.sh | 2 +- .../01103_optimize_drop_race_zookeeper.sh | 57 ++++----- ...tart_replicas_rename_deadlock_zookeeper.sh | 45 +++---- .../0_stateless/01150_ddl_guard_rwr.sh | 35 +++--- .../0_stateless/01154_move_partition_long.sh | 112 ++++++++++-------- .../01164_detach_attach_partition_race.sh | 28 +++-- .../01175_distributed_ddl_output_mode_long.sh | 2 +- .../0_stateless/01249_flush_interactive.sh | 2 - ..._recreate_reattach_and_show_tables_long.sh | 87 +++++++++----- ...1_aggregate_state_exception_memory_leak.sh | 11 +- ...2_aggregate_state_exception_memory_leak.sh | 8 +- .../01305_replica_create_drop_zookeeper.sh | 15 ++- ...20_create_sync_race_condition_zookeeper.sh | 8 +- .../01412_cache_dictionary_race.sh | 46 +++---- .../01444_create_table_drop_database_race.sh | 18 +-- ...01454_storagememory_data_race_challenge.sh | 18 +-- .../01542_dictionary_load_exception_race.sh | 30 ++--- .../01593_concurrent_alter_mutations_kill.sh | 36 +++--- ...alter_mutations_kill_many_replicas_long.sh | 42 ++++--- .../01602_max_distributed_connections.sh | 11 +- .../0_stateless/01632_tinylog_read_write.sh | 35 +++--- .../01671_ddl_hang_timeout_long.sh | 26 ++-- .../01732_race_condition_storage_join_long.sh | 35 ++++-- ...nt_ttl_and_normal_merges_zookeeper_long.sh | 40 ++++--- .../02015_async_inserts_stress_long.sh | 20 ++-- .../02015_shard_crash_clang_12_build.sh | 28 +++-- .../0_stateless/02030_rocksdb_race_long.sh | 27 +++-- ...44_url_glob_parallel_connection_refused.sh | 9 +- .../02122_4letter_words_stress_zookeeper.sh | 30 ++--- .../02124_buffer_with_type_map_long.sh | 12 +- tests/queries/shell_config.sh | 38 +----- 57 files changed, 1148 insertions(+), 998 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c4ad314ff9e..a8cab282b0d 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -243,37 +243,6 @@ def get_transactions_list(args): return f"Cannot get list of transactions: {e}" -def get_processlist_after_test(args): - log_comment = args.testcase_basename - database = args.testcase_database - if args.replicated_database: - return clickhouse_execute_json( - args, - f""" - SELECT materialize((hostName(), tcpPort())) as host, * - FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) - WHERE - query NOT LIKE '%system.processes%' AND - {'NOT is_all_data_sent AND' if args.suppport_system_processes_is_all_data_sent else ''} - Settings['log_comment'] = '{log_comment}' AND - current_database = '{database}' - """, - ) - else: - return clickhouse_execute_json( - args, - f""" - SELECT * - FROM system.processes - WHERE - query NOT LIKE '%system.processes%' AND - {'NOT is_all_data_sent AND' if args.suppport_system_processes_is_all_data_sent else ''} - Settings['log_comment'] = '{log_comment}' AND - current_database = '{database}' - """, - ) - - # collect server stacktraces using gdb def get_stacktraces_from_gdb(server_pid): try: diff --git a/tests/queries/0_stateless/00746_sql_fuzzy.sh b/tests/queries/0_stateless/00746_sql_fuzzy.sh index 72e80956f59..b534b1820ba 100755 --- a/tests/queries/0_stateless/00746_sql_fuzzy.sh +++ b/tests/queries/0_stateless/00746_sql_fuzzy.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -15,7 +14,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > "$ # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy for SQL_FUZZY_RUN in $(seq "${SQL_FUZZY_RUNS:=5}"); do - env SQL_FUZZY_RUN="$SQL_FUZZY_RUN" perl "$CURDIR"/00746_sql_fuzzy.pl | clickhouse_client_timeout 60 $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 + env SQL_FUZZY_RUN="$SQL_FUZZY_RUN" perl "$CURDIR"/00746_sql_fuzzy.pl | timeout 60 $CLICKHOUSE_CLIENT --format Null --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 if [[ $($CLICKHOUSE_CLIENT -q "SELECT 'Still alive'") != 'Still alive' ]]; then break fi diff --git a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh index 8e04ce7ff1e..19d9b006cd7 100755 --- a/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh +++ b/tests/queries/0_stateless/00816_long_concurrent_alter_column.sh @@ -7,51 +7,64 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -nm -q " - DROP TABLE IF EXISTS concurrent_alter_column; - CREATE TABLE concurrent_alter_column (ts DATETIME) ENGINE = MergeTree PARTITION BY toStartOfDay(ts) ORDER BY tuple(); -" +echo "DROP TABLE IF EXISTS concurrent_alter_column" | ${CLICKHOUSE_CLIENT} +echo "CREATE TABLE concurrent_alter_column (ts DATETIME) ENGINE = MergeTree PARTITION BY toStartOfDay(ts) ORDER BY tuple()" | ${CLICKHOUSE_CLIENT} + function thread1() { - for i in {1..500}; do - echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;" - done | ${CLICKHOUSE_CLIENT} -n + while true; do + for i in {1..500}; do echo "ALTER TABLE concurrent_alter_column ADD COLUMN c$i DOUBLE;"; done | ${CLICKHOUSE_CLIENT} -n --query_id=alter_00816_1 + done } function thread2() { - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_column DROP COLUMN d" + while true; do + echo "ALTER TABLE concurrent_alter_column ADD COLUMN d DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2; + sleep "$(echo 0.0$RANDOM)"; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN d" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_2; + done } function thread3() { - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_column DROP COLUMN e" + while true; do + echo "ALTER TABLE concurrent_alter_column ADD COLUMN e DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3; + sleep "$(echo 0.0$RANDOM)"; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN e" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_3; + done } function thread4() { - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_column DROP COLUMN f" + while true; do + echo "ALTER TABLE concurrent_alter_column ADD COLUMN f DOUBLE" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4; + sleep "$(echo 0.0$RANDOM)"; + echo "ALTER TABLE concurrent_alter_column DROP COLUMN f" | ${CLICKHOUSE_CLIENT} --query_id=alter_00816_4; + done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & wait -$CLICKHOUSE_CLIENT -q "DROP TABLE concurrent_alter_column NO DELAY" +echo "DROP TABLE concurrent_alter_column NO DELAY" | ${CLICKHOUSE_CLIENT} # NO DELAY has effect only for Atomic database + +# Wait for alters and check for deadlocks (in case of deadlock this loop will not finish) +while true; do + echo "SELECT * FROM system.processes WHERE query_id LIKE 'alter\\_00816\\_%'" | ${CLICKHOUSE_CLIENT} | grep -q -F 'alter' || break + sleep 1; +done + echo 'did not crash' diff --git a/tests/queries/0_stateless/00900_orc_load.sh b/tests/queries/0_stateless/00900_orc_load.sh index 3779092c511..b3f2c39e5d2 100755 --- a/tests/queries/0_stateless/00900_orc_load.sh +++ b/tests/queries/0_stateless/00900_orc_load.sh @@ -5,7 +5,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -DATA_FILE=$CUR_DIR/data_orc/test_$CLICKHOUSE_TEST_UNIQUE_NAME.orc +DATA_FILE=$CUR_DIR/data_orc/test.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (int Int32, smallint Int8, bigint Int64, float Float32, double Float64, date Date, y String, datetime64 DateTime64(3)) ENGINE = Memory" @@ -14,7 +14,7 @@ ${CLICKHOUSE_CLIENT} --query="select * from orc_load FORMAT ORC" > $DATA_FILE ${CLICKHOUSE_CLIENT} --query="truncate table orc_load" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" -clickhouse_client_timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE ${CLICKHOUSE_CLIENT} --query="select * from orc_load" ${CLICKHOUSE_CLIENT} --query="drop table orc_load" rm -rf "$DATA_FILE" diff --git a/tests/queries/0_stateless/00909_kill_not_initialized_query.sh b/tests/queries/0_stateless/00909_kill_not_initialized_query.sh index 816bab4c491..531652a33e7 100755 --- a/tests/queries/0_stateless/00909_kill_not_initialized_query.sh +++ b/tests/queries/0_stateless/00909_kill_not_initialized_query.sh @@ -35,7 +35,7 @@ $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE query='$query_to_kill' ASYNC" &>/dev/nul sleep 1 # Kill $query_for_pending SYNC. This query is not blocker, so it should be killed fast. -clickhouse_client_timeout 20 ${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null +timeout 20 ${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE query='$query_for_pending' SYNC" &>/dev/null # Both queries have to be killed, doesn't matter with SYNC or ASYNC kill for _ in {1..15} diff --git a/tests/queries/0_stateless/00941_system_columns_race_condition.sh b/tests/queries/0_stateless/00941_system_columns_race_condition.sh index 24f159c7d24..69dfb30cd2c 100755 --- a/tests/queries/0_stateless/00941_system_columns_race_condition.sh +++ b/tests/queries/0_stateless/00941_system_columns_race_condition.sh @@ -9,45 +9,40 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -nm -q " - DROP TABLE IF EXISTS alter_table; - CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = MergeTree ORDER BY a; -" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table" +$CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = MergeTree ORDER BY a" function thread1() { # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - $CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns UNION ALL SELECT name FROM system.columns FORMAT Null" + while true; do $CLICKHOUSE_CLIENT --query "SELECT name FROM system.columns UNION ALL SELECT name FROM system.columns FORMAT Null"; done } function thread2() { - $CLICKHOUSE_CLIENT -n --query " - ALTER TABLE alter_table ADD COLUMN h String; - ALTER TABLE alter_table MODIFY COLUMN h UInt64; - ALTER TABLE alter_table DROP COLUMN h; - " + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done } -export -f thread1 -export -f thread2 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread1 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & -clickhouse_client_loop_timeout 15 thread2 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread1 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & +timeout 15 bash -c thread2 2> /dev/null & wait diff --git a/tests/queries/0_stateless/00991_system_parts_race_condition_long.sh b/tests/queries/0_stateless/00991_system_parts_race_condition_long.sh index 731e96b055e..8243c6bde62 100755 --- a/tests/queries/0_stateless/00991_system_parts_race_condition_long.sh +++ b/tests/queries/0_stateless/00991_system_parts_race_condition_long.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, long +# Tags: race # This test is disabled because it triggers internal assert in Thread Sanitizer. # Thread Sanitizer does not support for more than 64 mutexes to be locked in a single thread. @@ -11,68 +11,67 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -nm -q " - DROP TABLE IF EXISTS alter_table; - CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = MergeTree ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1; -" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table" +$CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = MergeTree ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1" function thread1() { # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null" + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } function thread2() { - $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String '0'; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;" + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String '0'; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done } function thread3() { - $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)" + while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done } function thread4() { - $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table FINAL" + while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table FINAL"; done } function thread5() { - $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table DELETE WHERE rand() % 2 = 1" + while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table DELETE WHERE rand() % 2 = 1"; done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 -export -f thread5 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & wait diff --git a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh index 3cd61c9972a..8dbd10fc27b 100755 --- a/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh +++ b/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper_long.sh @@ -9,95 +9,76 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -n -q " DROP TABLE IF EXISTS alter_table0; DROP TABLE IF EXISTS alter_table1; - CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') - ORDER BY a - PARTITION BY b % 10 - SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; - - CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') - ORDER BY a - PARTITION BY b % 10 - SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; + CREATE TABLE alter_table0 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0; + CREATE TABLE alter_table1 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 1, cleanup_delay_period_random_add = 0 " function thread1() { # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null" + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done } function thread2() { - $CLICKHOUSE_CLIENT -nm --query " - ALTER TABLE alter_table0 ADD COLUMN h String DEFAULT '0'; - ALTER TABLE alter_table0 MODIFY COLUMN h UInt64; - ALTER TABLE alter_table0 DROP COLUMN h; - " + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table0 ADD COLUMN h String DEFAULT '0'; ALTER TABLE alter_table0 MODIFY COLUMN h UInt64; ALTER TABLE alter_table0 DROP COLUMN h;"; done } function thread3() { - $CLICKHOUSE_CLIENT -q " - INSERT INTO alter_table0 - SELECT - rand(1), rand(2), 1 / rand(3), toString(rand(4)), - [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), - (rand(8), rand(9)) - FROM numbers(100000)" + while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table0 SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done } function thread4() { - $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table0 FINAL" + while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table0 FINAL"; done } function thread5() { - $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288" + while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table0 DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288"; done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 -export -f thread5 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & wait check_replication_consistency "alter_table" "count(), sum(a), sum(b), round(sum(c))" $CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table0;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & $CLICKHOUSE_CLIENT -n -q "DROP TABLE alter_table1;" 2> >(grep -F -v 'is already started to be removing by another replica right now') & - wait diff --git a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh index 3fe7b0443eb..d5d43d3c293 100755 --- a/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh +++ b/tests/queries/0_stateless/00993_system_parts_race_condition_drop_zookeeper.sh @@ -12,87 +12,93 @@ set -e function thread1() { # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null" + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; + done } function thread2() { - REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table_$REPLICA ADD COLUMN h String '0'; ALTER TABLE alter_table_$REPLICA MODIFY COLUMN h UInt64; ALTER TABLE alter_table_$REPLICA DROP COLUMN h;" + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table_$REPLICA ADD COLUMN h String '0'; ALTER TABLE alter_table_$REPLICA MODIFY COLUMN h UInt64; ALTER TABLE alter_table_$REPLICA DROP COLUMN h;"; + done } function thread3() { - REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table_$REPLICA SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)" + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table_$REPLICA SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; + done } function thread4() { - REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table_$REPLICA FINAL" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table_$REPLICA FINAL"; + sleep 0.$RANDOM; + done } function thread5() { - REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table_$REPLICA DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table_$REPLICA DELETE WHERE cityHash64(a,b,c,d,e,g) % 1048576 < 524288"; + sleep 0.$RANDOM; + done } function thread6() { - REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -mn -q " - DROP TABLE IF EXISTS alter_table_$REPLICA; - - CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') - ORDER BY a - PARTITION BY b % 10 - SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; - " - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS alter_table_$REPLICA; + CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;"; + sleep 0.$RANDOM; + done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 -export -f thread5 -export -f thread6 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; +export -f thread6; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2>&1 | grep "was not completely removed from ZooKeeper" & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2>&1 | grep "was not completely removed from ZooKeeper" & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2>&1 | grep "was not completely removed from ZooKeeper" & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2>&1 | grep "was not completely removed from ZooKeeper" & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2>&1 | grep "was not completely removed from ZooKeeper" & wait diff --git a/tests/queries/0_stateless/01001_rename_merge_race_condition.sh b/tests/queries/0_stateless/01001_rename_merge_race_condition.sh index 9b5f92628f7..253d06c038c 100755 --- a/tests/queries/0_stateless/01001_rename_merge_race_condition.sh +++ b/tests/queries/0_stateless/01001_rename_merge_race_condition.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: race, long +# Tags: race CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -7,32 +7,35 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test1" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test2" -$CLICKHOUSE_CLIENT --query "CREATE TABLE test1 (x UInt64) ENGINE = Memory" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test1"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test2"; +$CLICKHOUSE_CLIENT --query "CREATE TABLE test1 (x UInt64) ENGINE = Memory"; function thread1() { - seq 1 1000 | { - sed -r -e 's/.+/RENAME TABLE test1 TO test2; RENAME TABLE test2 TO test1;/' - } | $CLICKHOUSE_CLIENT -n + while true; do + seq 1 1000 | sed -r -e 's/.+/RENAME TABLE test1 TO test2; RENAME TABLE test2 TO test1;/' | $CLICKHOUSE_CLIENT -n + done } function thread2() { - $CLICKHOUSE_CLIENT --query "SELECT * FROM merge('$CLICKHOUSE_DATABASE', '^test[12]$')" + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM merge('$CLICKHOUSE_DATABASE', '^test[12]$')" + done } -export -f thread1 -export -f thread2 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & wait -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test1" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test2" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test1"; +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test2"; diff --git a/tests/queries/0_stateless/01002_alter_nullable_adaptive_granularity_long.sh b/tests/queries/0_stateless/01002_alter_nullable_adaptive_granularity_long.sh index ee38cf4eb7e..b29a42a7356 100755 --- a/tests/queries/0_stateless/01002_alter_nullable_adaptive_granularity_long.sh +++ b/tests/queries/0_stateless/01002_alter_nullable_adaptive_granularity_long.sh @@ -12,39 +12,48 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE test (x UInt8, s String MATERIALIZED to function thread1() { - $CLICKHOUSE_CLIENT --query "INSERT INTO test SELECT rand() FROM numbers(1000)" + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO test SELECT rand() FROM numbers(1000)"; + done } function thread2() { - $CLICKHOUSE_CLIENT -n --query "ALTER TABLE test MODIFY COLUMN x Nullable(UInt8);" - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT -n --query "ALTER TABLE test MODIFY COLUMN x UInt8;" - sleep 0.0$RANDOM + while true; do + $CLICKHOUSE_CLIENT -n --query "ALTER TABLE test MODIFY COLUMN x Nullable(UInt8);"; + sleep 0.0$RANDOM + $CLICKHOUSE_CLIENT -n --query "ALTER TABLE test MODIFY COLUMN x UInt8;"; + sleep 0.0$RANDOM + done } function thread3() { - $CLICKHOUSE_CLIENT -n --query "SELECT count() FROM test FORMAT Null" + while true; do + $CLICKHOUSE_CLIENT -n --query "SELECT count() FROM test FORMAT Null"; + done } function thread4() { - $CLICKHOUSE_CLIENT -n --query "OPTIMIZE TABLE test FINAL" - sleep 0.1$RANDOM + while true; do + $CLICKHOUSE_CLIENT -n --query "OPTIMIZE TABLE test FINAL"; + sleep 0.1$RANDOM + done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01003_kill_query_race_condition.sh b/tests/queries/0_stateless/01003_kill_query_race_condition.sh index f98897b1544..b71a5106383 100755 --- a/tests/queries/0_stateless/01003_kill_query_race_condition.sh +++ b/tests/queries/0_stateless/01003_kill_query_race_condition.sh @@ -9,37 +9,44 @@ set -e function thread1() { - $CLICKHOUSE_CLIENT --query_id=hello_01003 --query "SELECT count() FROM numbers(1000000000)" --format Null; + while true; do + $CLICKHOUSE_CLIENT --query_id=hello_01003 --query "SELECT count() FROM numbers(1000000000)" --format Null; + done } function thread2() { - $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id = 'hello_01003'" --format Null - sleep 0.$RANDOM + while true; do + $CLICKHOUSE_CLIENT --query "KILL QUERY WHERE query_id = 'hello_01003'" --format Null; + sleep 0.$RANDOM + done } function thread3() { - $CLICKHOUSE_CLIENT --query "SHOW PROCESSLIST" --format Null - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.processes" --format Null + while true; do + $CLICKHOUSE_CLIENT --query "SHOW PROCESSLIST" --format Null; + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.processes" --format Null; + done } -export -f thread1 -export -f thread2 -export -f thread3 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01004_rename_deadlock.sh b/tests/queries/0_stateless/01004_rename_deadlock.sh index 4a3f23883c0..f0adf136e94 100755 --- a/tests/queries/0_stateless/01004_rename_deadlock.sh +++ b/tests/queries/0_stateless/01004_rename_deadlock.sh @@ -14,41 +14,48 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE test2 (x UInt8) ENGINE = MergeTree ORDE function thread1() { - $CLICKHOUSE_CLIENT --query "RENAME TABLE test1 TO test_tmp, test2 TO test1, test_tmp TO test2" + while true; do + $CLICKHOUSE_CLIENT --query "RENAME TABLE test1 TO test_tmp, test2 TO test1, test_tmp TO test2" + done } function thread2() { - $CLICKHOUSE_CLIENT --query "SELECT * FROM test1 UNION ALL SELECT * FROM test2" --format Null + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM test1 UNION ALL SELECT * FROM test2" --format Null + done } function thread3() { - # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables" --format Null + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.tables" --format Null + done } -export -f thread1 -export -f thread2 -export -f thread3 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & wait sleep 1 diff --git a/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh b/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh index 948032f19e7..ef352606b69 100755 --- a/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh +++ b/tests/queries/0_stateless/01005_rwr_shard_deadlock.sh @@ -12,31 +12,36 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE test1 (x UInt8) ENGINE = MergeTree ORDE function thread1() { - $CLICKHOUSE_CLIENT --query "ALTER TABLE test1 MODIFY COLUMN x Nullable(UInt8)" - $CLICKHOUSE_CLIENT --query "ALTER TABLE test1 MODIFY COLUMN x UInt8" + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE test1 MODIFY COLUMN x Nullable(UInt8)" + $CLICKHOUSE_CLIENT --query "ALTER TABLE test1 MODIFY COLUMN x UInt8" + done } function thread2() { - $CLICKHOUSE_CLIENT --query "SELECT x FROM test1 WHERE x IN (SELECT x FROM remote('127.0.0.2', '$CLICKHOUSE_DATABASE', test1))" --format Null + while true; do + $CLICKHOUSE_CLIENT --query "SELECT x FROM test1 WHERE x IN (SELECT x FROM remote('127.0.0.2', '$CLICKHOUSE_DATABASE', test1))" --format Null + done } -export -f thread1 -export -f thread2 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh index 7fa5f5456a2..9f4b2241732 100755 --- a/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh +++ b/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: deadlock, long +# Tags: deadlock CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -16,39 +16,48 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE b (x UInt8) ENGINE = MergeTree ORDER BY function thread1() { - # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n + done } function thread2() { - # NOTE: database = $CLICKHOUSE_DATABASE is unwanted - seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n + while true; do + # NOTE: database = $CLICKHOUSE_DATABASE is unwanted + seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n + done } function thread3() { - $CLICKHOUSE_CLIENT --query "ALTER TABLE a MODIFY COLUMN x Nullable(UInt8)" - $CLICKHOUSE_CLIENT --query "ALTER TABLE a MODIFY COLUMN x UInt8" + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE a MODIFY COLUMN x Nullable(UInt8)" + $CLICKHOUSE_CLIENT --query "ALTER TABLE a MODIFY COLUMN x UInt8" + done } function thread4() { - $CLICKHOUSE_CLIENT --query "ALTER TABLE b MODIFY COLUMN x Nullable(UInt8)" - $CLICKHOUSE_CLIENT --query "ALTER TABLE b MODIFY COLUMN x UInt8" + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE b MODIFY COLUMN x Nullable(UInt8)" + $CLICKHOUSE_CLIENT --query "ALTER TABLE b MODIFY COLUMN x UInt8" + done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh index 290708ccde4..55bbfb3ff11 100755 --- a/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh +++ b/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh @@ -19,7 +19,7 @@ ${CLICKHOUSE_CLIENT} -n -q " INSERT INTO $R1 VALUES (1) " -clickhouse_client_timeout 10s ${CLICKHOUSE_CLIENT} --receive_timeout 1 -n -q " +timeout 10s ${CLICKHOUSE_CLIENT} -n -q " SET receive_timeout=1; SYSTEM SYNC REPLICA $R2 " 2>&1 | grep -F -q "Code: 159. DB::Exception" && echo 'OK' || echo 'Failed!' diff --git a/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh b/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh index d1953dbfd0f..0a6888a5c69 100755 --- a/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh +++ b/tests/queries/0_stateless/01014_lazy_database_concurrent_recreate_reattach_and_show_tables.sh @@ -7,67 +7,95 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) export CURR_DATABASE="test_lazy_01014_concurrent_${CLICKHOUSE_DATABASE}" + function recreate_lazy_func1() { - $CLICKHOUSE_CLIENT -nm -q " - DETACH TABLE $CURR_DATABASE.log; - ATTACH TABLE $CURR_DATABASE.log; - " + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.log (a UInt64, b UInt64) ENGINE = Log; + "; + + while true; do + $CLICKHOUSE_CLIENT -q " + DETACH TABLE $CURR_DATABASE.log; + "; + + $CLICKHOUSE_CLIENT -q " + ATTACH TABLE $CURR_DATABASE.log; + "; + done } function recreate_lazy_func2() { - $CLICKHOUSE_CLIENT -nm -q " - CREATE TABLE $CURR_DATABASE.tlog (a UInt64, b UInt64) ENGINE = TinyLog; - DROP TABLE $CURR_DATABASE.tlog; - " + while true; do + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.tlog (a UInt64, b UInt64) ENGINE = TinyLog; + "; + + $CLICKHOUSE_CLIENT -q " + DROP TABLE $CURR_DATABASE.tlog; + "; + done } function recreate_lazy_func3() { - $CLICKHOUSE_CLIENT -nm -q " - ATTACH TABLE $CURR_DATABASE.slog; - DETACH TABLE $CURR_DATABASE.slog; - " + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.slog (a UInt64, b UInt64) ENGINE = StripeLog; + "; + + while true; do + $CLICKHOUSE_CLIENT -q " + ATTACH TABLE $CURR_DATABASE.slog; + "; + + $CLICKHOUSE_CLIENT -q " + DETACH TABLE $CURR_DATABASE.slog; + "; + done } function recreate_lazy_func4() { - $CLICKHOUSE_CLIENT -nm -q " - CREATE TABLE $CURR_DATABASE.tlog2 (a UInt64, b UInt64) ENGINE = TinyLog; - DROP TABLE $CURR_DATABASE.tlog2; - " + while true; do + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.tlog2 (a UInt64, b UInt64) ENGINE = TinyLog; + "; + + $CLICKHOUSE_CLIENT -q " + DROP TABLE $CURR_DATABASE.tlog2; + "; + done } function show_tables_func() { - $CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE database = '$CURR_DATABASE' FORMAT Null" + while true; do + $CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE database = '$CURR_DATABASE' FORMAT Null"; + done } -export -f recreate_lazy_func1 -export -f recreate_lazy_func2 -export -f recreate_lazy_func3 -export -f recreate_lazy_func4 -export -f show_tables_func +export -f recreate_lazy_func1; +export -f recreate_lazy_func2; +export -f recreate_lazy_func3; +export -f recreate_lazy_func4; +export -f show_tables_func; ${CLICKHOUSE_CLIENT} -n -q " DROP DATABASE IF EXISTS $CURR_DATABASE; CREATE DATABASE $CURR_DATABASE ENGINE = Lazy(1); - - CREATE TABLE $CURR_DATABASE.log (a UInt64, b UInt64) ENGINE = Log; - CREATE TABLE $CURR_DATABASE.slog (a UInt64, b UInt64) ENGINE = StripeLog; " TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT show_tables_func 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func1 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func2 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func3 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func4 2> /dev/null & +timeout $TIMEOUT bash -c show_tables_func 2> /dev/null & wait sleep 1 @@ -80,3 +108,4 @@ ${CLICKHOUSE_CLIENT} -q "ATTACH TABLE $CURR_DATABASE.tlog2;" 2>/dev/null ${CLICKHOUSE_CLIENT} -q "DROP DATABASE $CURR_DATABASE" echo "Test OK" + diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh b/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh index e98fb57c2a4..872b0a7c1a1 100755 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_concurrent_requrests.sh @@ -7,7 +7,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) set -e -$CLICKHOUSE_CLIENT -mn -q " +$CLICKHOUSE_CLIENT -n -q " DROP DATABASE IF EXISTS database_for_dict; DROP TABLE IF EXISTS table_for_dict1; DROP TABLE IF EXISTS table_for_dict2; @@ -20,104 +20,96 @@ $CLICKHOUSE_CLIENT -mn -q " CREATE DATABASE database_for_dict; - CREATE DICTIONARY database_for_dict.dict1 (key_column UInt64, value_column String) - PRIMARY KEY key_column - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) - LIFETIME(MIN 1 MAX 5) - LAYOUT(FLAT()); + CREATE DICTIONARY database_for_dict.dict1 (key_column UInt64, value_column String) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict1' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) LIFETIME(MIN 1 MAX 5) LAYOUT(FLAT()); - CREATE DICTIONARY database_for_dict.dict2 (key_column UInt64, value_column String) - PRIMARY KEY key_column - SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) - LIFETIME(MIN 1 MAX 5) - LAYOUT(CACHE(SIZE_IN_CELLS 150)); + CREATE DICTIONARY database_for_dict.dict2 (key_column UInt64, value_column String) PRIMARY KEY key_column SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' TABLE 'table_for_dict2' PASSWORD '' DB '$CLICKHOUSE_DATABASE')) LIFETIME(MIN 1 MAX 5) LAYOUT(CACHE(SIZE_IN_CELLS 150)); " function thread1() { - $CLICKHOUSE_CLIENT --query "SELECT * FROM system.dictionaries FORMAT Null" + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.dictionaries FORMAT Null"; done } function thread2() { - $CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict1" ||: + while true; do CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict1" ||: ; done } function thread3() { - $CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict2" ||: + while true; do CLICKHOUSE_CLIENT --query "ATTACH DICTIONARY database_for_dict.dict2" ||:; done } function thread4() { - $CLICKHOUSE_CLIENT -n -q " + while true; do $CLICKHOUSE_CLIENT -n -q " SELECT * FROM database_for_dict.dict1 FORMAT Null; SELECT * FROM database_for_dict.dict2 FORMAT Null; - " ||: + " ||: ; done } function thread5() { - $CLICKHOUSE_CLIENT -n -q " + while true; do $CLICKHOUSE_CLIENT -n -q " SELECT dictGetString('database_for_dict.dict1', 'value_column', toUInt64(number)) from numbers(1000) FROM FORMAT Null; SELECT dictGetString('database_for_dict.dict2', 'value_column', toUInt64(number)) from numbers(1000) FROM FORMAT Null; - " ||: + " ||: ; done } function thread6() { - $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict1" + while true; do $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict1"; done } function thread7() { - $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict2" + while true; do $CLICKHOUSE_CLIENT -q "DETACH DICTIONARY database_for_dict.dict2"; done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 -export -f thread5 -export -f thread6 -export -f thread7 +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; +export -f thread6; +export -f thread7; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread7 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread7 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread7 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread6 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread7 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & +timeout $TIMEOUT bash -c thread7 2> /dev/null & wait $CLICKHOUSE_CLIENT -q "SELECT 'Still alive'" diff --git a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh index aa3e1b612dc..54a7e940377 100755 --- a/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh +++ b/tests/queries/0_stateless/01019_alter_materialized_view_atomic.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long set -e @@ -30,15 +29,19 @@ EOF function alter_thread() { + trap 'exit' INT + ALTERS[0]="ALTER TABLE mv MODIFY QUERY SELECT v FROM src;" ALTERS[1]="ALTER TABLE mv MODIFY QUERY SELECT v * 2 as v FROM src;" - $CLICKHOUSE_CLIENT --allow_experimental_alter_materialized_view_structure=1 -q "${ALTERS[$RANDOM % 2]}" - sleep 0.$RANDOM + while true; do + $CLICKHOUSE_CLIENT --allow_experimental_alter_materialized_view_structure=1 -q "${ALTERS[$RANDOM % 2]}" + sleep "$(echo 0.$RANDOM)"; + done } -export -f alter_thread -clickhouse_client_loop_timeout 10 alter_thread & +export -f alter_thread; +timeout 10 bash -c alter_thread & for _ in {1..100}; do # Retry (hopefully retriable (deadlock avoided)) errors. diff --git a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh index 19c97ae96ec..4c0afc4c439 100755 --- a/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh +++ b/tests/queries/0_stateless/01035_concurrent_move_partition_from_table_zookeeper.sh @@ -15,46 +15,66 @@ $CLICKHOUSE_CLIENT --query="CREATE TABLE dst (p UInt64, k String) ENGINE = Repli function thread1() { - $CLICKHOUSE_CLIENT --query="ALTER TABLE src MOVE PARTITION 1 TO TABLE dst" + while true; + do + $CLICKHOUSE_CLIENT --query="ALTER TABLE src MOVE PARTITION 1 TO TABLE dst;" --query_id=query1 + done } function thread2() { - $CLICKHOUSE_CLIENT --query="INSERT INTO src SELECT number % 2, toString(number) FROM system.numbers LIMIT 100000" + while true; + do + $CLICKHOUSE_CLIENT --query="INSERT INTO src SELECT number % 2, toString(number) FROM system.numbers LIMIT 100000" --query_id=query2 + done } function thread3() { - $CLICKHOUSE_CLIENT --query="SELECT * FROM src" > /dev/null + while true; + do + $CLICKHOUSE_CLIENT --query="SELECT * FROM src" --query_id=query3 1> /dev/null + done } function thread4() { - $CLICKHOUSE_CLIENT --query="SELECT * FROM dst" > /dev/null + while true; + do + $CLICKHOUSE_CLIENT --query="SELECT * FROM dst" --query_id=query4 1> /dev/null + done } function thread5() { - $CLICKHOUSE_CLIENT --query="ALTER TABLE src MOVE PARTITION 1 TO TABLE dst" + while true; + do + $CLICKHOUSE_CLIENT --query="ALTER TABLE src MOVE PARTITION 1 TO TABLE dst;" --query_id=query5 + done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 -export -f thread5 +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & wait echo "DROP TABLE src NO DELAY" | ${CLICKHOUSE_CLIENT} echo "DROP TABLE dst NO DELAY" | ${CLICKHOUSE_CLIENT} +sleep 5 + +# Check for deadlocks +echo "SELECT * FROM system.processes WHERE query_id LIKE 'query%'" | ${CLICKHOUSE_CLIENT} echo 'did not crash' diff --git a/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh b/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh index 6df9cc3e258..04b1f8b65ce 100755 --- a/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh +++ b/tests/queries/0_stateless/01054_cache_dictionary_bunch_update.sh @@ -18,44 +18,56 @@ $CLICKHOUSE_CLIENT --query="insert into test_01054.ints values (3, 3, 3, 3, 3, 3 function thread1() { + for _ in {1..100} + do RAND_NUMBER_THREAD1=$($CLICKHOUSE_CLIENT --query="SELECT rand() % 100;") $CLICKHOUSE_CLIENT --query="select dictGet('one_cell_cache_ints', 'i8', toUInt64($RAND_NUMBER_THREAD1));" + done } function thread2() { + for _ in {1..100} + do RAND_NUMBER_THREAD2=$($CLICKHOUSE_CLIENT --query="SELECT rand() % 100;") $CLICKHOUSE_CLIENT --query="select dictGet('one_cell_cache_ints', 'i8', toUInt64($RAND_NUMBER_THREAD2));" + done } function thread3() { + for _ in {1..100} + do RAND_NUMBER_THREAD3=$($CLICKHOUSE_CLIENT --query="SELECT rand() % 100;") $CLICKHOUSE_CLIENT --query="select dictGet('one_cell_cache_ints', 'i8', toUInt64($RAND_NUMBER_THREAD3));" + done } function thread4() { + for _ in {1..100} + do RAND_NUMBER_THREAD4=$($CLICKHOUSE_CLIENT --query="SELECT rand() % 100;") $CLICKHOUSE_CLIENT --query="select dictGet('one_cell_cache_ints', 'i8', toUInt64($RAND_NUMBER_THREAD4));" + done } -export -f thread1 -export -f thread2 -export -f thread3 -export -f thread4 +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; TIMEOUT=10 # shellcheck disable=SC2188 -clickhouse_client_loop_timeout $TIMEOUT thread1 > /dev/null 2>&1 & -clickhouse_client_loop_timeout $TIMEOUT thread2 > /dev/null 2>&1 & -clickhouse_client_loop_timeout $TIMEOUT thread3 > /dev/null 2>&1 & -clickhouse_client_loop_timeout $TIMEOUT thread4 > /dev/null 2>&1 & +timeout $TIMEOUT bash -c thread1 > /dev/null 2>&1 & +timeout $TIMEOUT bash -c thread2 > /dev/null 2>&1 & +timeout $TIMEOUT bash -c thread3 > /dev/null 2>&1 & +timeout $TIMEOUT bash -c thread4 > /dev/null 2>&1 & wait diff --git a/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh b/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh index a1a38370554..17068dcbdf9 100755 --- a/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh +++ b/tests/queries/0_stateless/01076_cache_dictionary_datarace_exception_ptr.sh @@ -38,24 +38,30 @@ LAYOUT(CACHE(SIZE_IN_CELLS 10)); function thread1() { - # This query will be ended with exception, because source dictionary has UUID as a key type. - $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('dictdb_01076.dict_datarace', 'value', toUInt64(1));" + for _ in {1..50} + do + # This query will be ended with exception, because source dictionary has UUID as a key type. + $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('dictdb_01076.dict_datarace', 'value', toUInt64(1));" + done } function thread2() { - # This query will be ended with exception, because source dictionary has UUID as a key type. - $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('dictdb_01076.dict_datarace', 'value', toUInt64(2));" + for _ in {1..50} + do + # This query will be ended with exception, because source dictionary has UUID as a key type. + $CLICKHOUSE_CLIENT --query="SELECT dictGetFloat64('dictdb_01076.dict_datarace', 'value', toUInt64(2));" + done } -export -f thread1 -export -f thread2 +export -f thread1; +export -f thread2; TIMEOUT=5 -clickhouse_client_loop_timeout $TIMEOUT thread1 > /dev/null 2>&1 & -clickhouse_client_loop_timeout $TIMEOUT thread2 > /dev/null 2>&1 & +timeout $TIMEOUT bash -c thread1 > /dev/null 2>&1 & +timeout $TIMEOUT bash -c thread2 > /dev/null 2>&1 & wait diff --git a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh index 6c9acd6d0cd..bbc16121cb6 100755 --- a/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh +++ b/tests/queries/0_stateless/01076_parallel_alter_replicated_zookeeper.sh @@ -21,12 +21,7 @@ for i in $(seq $REPLICAS); do done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT -nm --query " - CREATE TABLE concurrent_mutate_mt_$i (key UInt64, value1 UInt64, value2 String) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_mutate_mt', '$i') - ORDER BY key - SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000,temporary_directories_lifetime=10,cleanup_delay_period=3,cleanup_delay_period_random_add=0; - " + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_mt_$i (key UInt64, value1 UInt64, value2 String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_mutate_mt', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000,temporary_directories_lifetime=10,cleanup_delay_period=3,cleanup_delay_period_random_add=0" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_mt_1 SELECT number, number + 10, toString(number) from numbers(10)" @@ -45,52 +40,59 @@ INITIAL_SUM=$($CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_mut # Run mutation on random replica function correct_alter_thread() { - REPLICA=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_mt_$REPLICA UPDATE value1 = value1 + 1 WHERE 1" - sleep 1 + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_mt_$REPLICA UPDATE value1 = value1 + 1 WHERE 1"; + sleep 1 + done } # This thread add some data to table. function insert_thread() { + VALUES=(7 8 9) - REPLICA=$(($RANDOM % 5 + 1)) - VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]} - $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_mt_$REPLICA VALUES($RANDOM, $VALUE, toString($VALUE))" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]} + $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_mt_$REPLICA VALUES($RANDOM, $VALUE, toString($VALUE))" + sleep 0.$RANDOM + done } function detach_attach_thread() { - REPLICA=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_mutate_mt_$REPLICA" - sleep 0.$RANDOM - sleep 0.$RANDOM - sleep 0.$RANDOM - $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_mutate_mt_$REPLICA" + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_mutate_mt_$REPLICA" + sleep 0.$RANDOM + sleep 0.$RANDOM + sleep 0.$RANDOM + $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_mutate_mt_$REPLICA" + done } echo "Starting alters" -export -f correct_alter_thread -export -f insert_thread -export -f detach_attach_thread +export -f correct_alter_thread; +export -f insert_thread; +export -f detach_attach_thread; # We assign a lot of mutations so timeout shouldn't be too big TIMEOUT=15 -clickhouse_client_loop_timeout $TIMEOUT detach_attach_thread 2> /dev/null & +timeout $TIMEOUT bash -c detach_attach_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT correct_alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh index 77278ad0bf9..06d6ef6a94b 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_add_drop_column_zookeeper.sh @@ -15,12 +15,7 @@ done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT -nm --query " - CREATE TABLE concurrent_alter_add_drop_$i (key UInt64, value0 UInt8) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_add_drop_column', '$i') - ORDER BY key - SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000; - " + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_add_drop_$i (key UInt64, value0 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_add_drop_column', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_1 SELECT number, number + 10 from numbers(100000)" @@ -32,54 +27,58 @@ done function alter_thread() { - REPLICA=$(($RANDOM % 3 + 1)) - ADD=$(($RANDOM % 5 + 1)) - # additionaly we don't wait anything for more heavy concurrency - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_$REPLICA ADD COLUMN value$ADD UInt32 DEFAULT 42 SETTINGS replication_alter_partitions_sync=0" - DROP=$(($RANDOM % 5 + 1)) - # additionaly we don't wait anything for more heavy concurrency - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_$REPLICA DROP COLUMN value$DROP SETTINGS replication_alter_partitions_sync=0" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + ADD=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_$REPLICA ADD COLUMN value$ADD UInt32 DEFAULT 42 SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + DROP=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_add_drop_$REPLICA DROP COLUMN value$DROP SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + sleep 0.$RANDOM + done } function optimize_thread() { - REPLICA=$(($RANDOM % 3 + 1)) - $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_alter_add_drop_$REPLICA FINAL SETTINGS replication_alter_partitions_sync=0" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_alter_add_drop_$REPLICA FINAL SETTINGS replication_alter_partitions_sync=0"; + sleep 0.$RANDOM + done } function insert_thread() { - REPLICA=$(($RANDOM % 3 + 1)) - $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_$REPLICA VALUES($RANDOM, 7)" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_add_drop_$REPLICA VALUES($RANDOM, 7)" + sleep 0.$RANDOM + done } echo "Starting alters" -export -f alter_thread -export -f optimize_thread -export -f insert_thread +export -f alter_thread; +export -f optimize_thread; +export -f insert_thread; TIMEOUT=30 # Sometimes we detach and attach tables -clickhouse_client_loop_timeout $TIMEOUT alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh index 20a5f9af108..1f316b4b389 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_detach_table_zookeeper.sh @@ -12,24 +12,11 @@ for i in $(seq $REPLICAS); do done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT -nm --query " - CREATE TABLE concurrent_alter_detach_$i (key UInt64, value1 UInt8, value2 UInt8) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_detach', '$i') - ORDER BY key - SETTINGS - max_replicated_mutations_in_queue=1000, - number_of_free_entries_in_pool_to_execute_mutation=0, - max_replicated_merges_in_queue=1000, - temporary_directories_lifetime=10, - cleanup_delay_period=3, - cleanup_delay_period_random_add=0; - " + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_detach_$i (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_detach', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000,temporary_directories_lifetime=10,cleanup_delay_period=3,cleanup_delay_period_random_add=0" done -$CLICKHOUSE_CLIENT -nm --query " - INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10); - INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10, 40); -" +$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10)" +$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_1 SELECT number, number + 10, number from numbers(10, 40)" for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA concurrent_alter_detach_$i" @@ -44,11 +31,12 @@ INITIAL_SUM=$($CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alt function correct_alter_thread() { TYPES=(Float64 String UInt8 UInt32) - REPLICA=$(($RANDOM % 3 + 1)) - TYPE=${TYPES[$RANDOM % ${#TYPES[@]} ]} - # additionaly we don't wait anything for more heavy concurrency - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_detach_$REPLICA MODIFY COLUMN value1 $TYPE SETTINGS replication_alter_partitions_sync=0" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + TYPE=${TYPES[$RANDOM % ${#TYPES[@]} ]} + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_detach_$REPLICA MODIFY COLUMN value1 $TYPE SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + sleep 0.$RANDOM + done } # This thread add some data to table. After we finish we can check, that @@ -56,38 +44,43 @@ function correct_alter_thread() # insert queries will fail sometime because of wrong types. function insert_thread() { + VALUES=(7.0 7 '7') - REPLICA=$(($RANDOM % 3 + 1)) - VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]} - $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_$REPLICA VALUES($RANDOM, $VALUE, $VALUE)" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]} + $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_detach_$REPLICA VALUES($RANDOM, $VALUE, $VALUE)" + sleep 0.$RANDOM + done } function detach_attach_thread() { - REPLICA=$(($RANDOM % 3 + 1)) - $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_alter_detach_$REPLICA" - sleep 0.$RANDOM - $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_detach_$REPLICA" + while true; do + REPLICA=$(($RANDOM % 3 + 1)) + $CLICKHOUSE_CLIENT --query "DETACH TABLE concurrent_alter_detach_$REPLICA" + sleep 0.$RANDOM + $CLICKHOUSE_CLIENT --query "ATTACH TABLE concurrent_alter_detach_$REPLICA" + done } echo "Starting alters" -export -f correct_alter_thread -export -f insert_thread -export -f detach_attach_thread +export -f correct_alter_thread; +export -f insert_thread; +export -f detach_attach_thread; TIMEOUT=15 # Sometimes we detach and attach tables -clickhouse_client_loop_timeout $TIMEOUT detach_attach_thread 2> /dev/null & +timeout $TIMEOUT bash -c detach_attach_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT correct_alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh index aef23c460d8..ba8d89aad3c 100755 --- a/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh +++ b/tests/queries/0_stateless/01079_parallel_alter_modify_zookeeper_long.sh @@ -14,11 +14,7 @@ for i in $(seq $REPLICAS); do done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT -nm --query " - CREATE TABLE concurrent_alter_mt_$i (key UInt64, value1 UInt64, value2 Int32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_mt', '$i') - ORDER BY key - SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_alter_mt_$i (key UInt64, value1 UInt64, value2 Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_alter_mt', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_mt_1 SELECT number, number + 10, number from numbers(10)" @@ -40,10 +36,12 @@ INITIAL_SUM=$($CLICKHOUSE_CLIENT --query "SELECT SUM(value1) FROM concurrent_alt function correct_alter_thread() { TYPES=(Float64 String UInt8 UInt32) - REPLICA=$(($RANDOM % 5 + 1)) - TYPE=${TYPES[$RANDOM % ${#TYPES[@]} ]} - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_mt_$REPLICA MODIFY COLUMN value1 $TYPE SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + TYPE=${TYPES[$RANDOM % ${#TYPES[@]} ]} + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_alter_mt_$REPLICA MODIFY COLUMN value1 $TYPE SETTINGS replication_alter_partitions_sync=0"; # additionaly we don't wait anything for more heavy concurrency + sleep 0.$RANDOM + done } # This thread add some data to table. After we finish we can check, that @@ -51,49 +49,56 @@ function correct_alter_thread() # insert queries will fail sometime because of wrong types. function insert_thread() { + VALUES=(7.0 7 '7') - REPLICA=$(($RANDOM % 5 + 1)) - VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]} - $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_mt_$REPLICA VALUES($RANDOM, $VALUE, $VALUE)" - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + VALUE=${VALUES[$RANDOM % ${#VALUES[@]} ]} + $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_alter_mt_$REPLICA VALUES($RANDOM, $VALUE, $VALUE)" + sleep 0.$RANDOM + done } # Some select load, to be sure, that our selects work in concurrent execution with alters function select_thread() { - REPLICA=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) FROM concurrent_alter_mt_$REPLICA" 1>/dev/null - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "SELECT SUM(toUInt64(value1)) FROM concurrent_alter_mt_$REPLICA" 1>/dev/null + sleep 0.$RANDOM + done } echo "Starting alters" -export -f correct_alter_thread -export -f insert_thread -export -f select_thread +export -f correct_alter_thread; +export -f insert_thread; +export -f select_thread; TIMEOUT=30 # Selects should run successfully -clickhouse_client_loop_timeout $TIMEOUT select_thread & -clickhouse_client_loop_timeout $TIMEOUT select_thread & -clickhouse_client_loop_timeout $TIMEOUT select_thread & +timeout $TIMEOUT bash -c select_thread & +timeout $TIMEOUT bash -c select_thread & +timeout $TIMEOUT bash -c select_thread & -clickhouse_client_loop_timeout $TIMEOUT correct_alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT correct_alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT correct_alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c correct_alter_thread 2> /dev/null & + + +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01085_max_distributed_connections.sh b/tests/queries/0_stateless/01085_max_distributed_connections.sh index 8a74935c05d..34862289d1e 100755 --- a/tests/queries/0_stateless/01085_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01085_max_distributed_connections.sh @@ -18,6 +18,6 @@ while [[ $i -lt $retries ]]; do # 10 less then 20 seconds (20 streams), but long enough to cover possible load peaks # "$@" left to pass manual options (like --experimental_use_processors 0) during manual testing - clickhouse_client_timeout 10s ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" && break + timeout 10s ${CLICKHOUSE_CLIENT} "${opts[@]}" "$@" && break ((++i)) done diff --git a/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh b/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh index d73ce1aeef4..95f8dfc0377 100755 --- a/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh +++ b/tests/queries/0_stateless/01103_optimize_drop_race_zookeeper.sh @@ -9,53 +9,54 @@ set -e function thread1() { - $CLICKHOUSE_CLIENT -q "INSERT INTO concurrent_optimize_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(10000)" + while true; do + $CLICKHOUSE_CLIENT -q "INSERT INTO concurrent_optimize_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(10000)"; + done } function thread2() { - $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE concurrent_optimize_table FINAL" - sleep 0.$RANDOM + while true; do + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE concurrent_optimize_table FINAL"; + sleep 0.$RANDOM; + done } function thread3() { - $CLICKHOUSE_CLIENT -mn -q " - DROP TABLE IF EXISTS concurrent_optimize_table; - CREATE TABLE concurrent_optimize_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_optimize_table', '1') - ORDER BY a - PARTITION BY b % 10 - SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;"; - sleep 0.$RANDOM - sleep 0.$RANDOM - sleep 0.$RANDOM + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS concurrent_optimize_table; + CREATE TABLE concurrent_optimize_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_optimize_table', '1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0;"; + sleep 0.$RANDOM; + sleep 0.$RANDOM; + sleep 0.$RANDOM; + done } -export -f thread1 -export -f thread2 -export -f thread3 +export -f thread1; +export -f thread2; +export -f thread3; TIMEOUT=15 -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh index a51e786b058..e6b145aa57c 100755 --- a/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh +++ b/tests/queries/0_stateless/01108_restart_replicas_rename_deadlock_zookeeper.sh @@ -14,41 +14,46 @@ done function rename_thread_1() { - $CLICKHOUSE_CLIENT -q "RENAME TABLE replica_01108_1 TO replica_01108_1_tmp, - replica_01108_2 TO replica_01108_2_tmp, - replica_01108_3 TO replica_01108_3_tmp, - replica_01108_4 TO replica_01108_4_tmp" - sleep 0.$RANDOM + while true; do + $CLICKHOUSE_CLIENT -q "RENAME TABLE replica_01108_1 TO replica_01108_1_tmp, + replica_01108_2 TO replica_01108_2_tmp, + replica_01108_3 TO replica_01108_3_tmp, + replica_01108_4 TO replica_01108_4_tmp"; + sleep 0.$RANDOM; + done } function rename_thread_2() { - $CLICKHOUSE_CLIENT -q "RENAME TABLE replica_01108_1_tmp TO replica_01108_2, - replica_01108_2_tmp TO replica_01108_3, - replica_01108_3_tmp TO replica_01108_4, - replica_01108_4_tmp TO replica_01108_1" - sleep 0.$RANDOM + while true; do + $CLICKHOUSE_CLIENT -q "RENAME TABLE replica_01108_1_tmp TO replica_01108_2, + replica_01108_2_tmp TO replica_01108_3, + replica_01108_3_tmp TO replica_01108_4, + replica_01108_4_tmp TO replica_01108_1"; + sleep 0.$RANDOM; + done } function restart_replicas_loop() { - for i in $(seq 4); do - $CLICKHOUSE_CLIENT -q "SYSTEM RESTART REPLICA replica_01108_${i}" - $CLICKHOUSE_CLIENT -q "SYSTEM RESTART REPLICA replica_01108_${i}_tmp" + while true; do + for i in $(seq 4); do + $CLICKHOUSE_CLIENT -q "SYSTEM RESTART REPLICA replica_01108_${i}"; + $CLICKHOUSE_CLIENT -q "SYSTEM RESTART REPLICA replica_01108_${i}_tmp"; + done + sleep 0.$RANDOM; done - sleep 0.$RANDOM } -export -f rename_thread_1 -export -f rename_thread_2 +export -f rename_thread_1; +export -f rename_thread_2; export -f restart_replicas_loop TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT rename_thread_1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT rename_thread_2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT restart_replicas_loop 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT restart_replicas_loop 2> /dev/null & +timeout $TIMEOUT bash -c rename_thread_1 2> /dev/null & +timeout $TIMEOUT bash -c rename_thread_2 2> /dev/null & +timeout $TIMEOUT bash -c restart_replicas_loop 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01150_ddl_guard_rwr.sh b/tests/queries/0_stateless/01150_ddl_guard_rwr.sh index 175df7fd9b9..50e6f91b49b 100755 --- a/tests/queries/0_stateless/01150_ddl_guard_rwr.sh +++ b/tests/queries/0_stateless/01150_ddl_guard_rwr.sh @@ -13,30 +13,31 @@ $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01150" $CLICKHOUSE_CLIENT --query "CREATE TABLE test_01150.t1 (x UInt64, s Array(Nullable(String))) ENGINE = Memory" $CLICKHOUSE_CLIENT --query "CREATE TABLE test_01150.t2 (x UInt64, s Array(Nullable(String))) ENGINE = Memory" -function thread_detach_attach() -{ - $CLICKHOUSE_CLIENT --query "DETACH DATABASE test_01150" 2>&1 | grep -v -F -e 'Received exception from server' -e 'Code: 219' -e '(query: ' - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "ATTACH DATABASE test_01150" 2>&1 | grep -v -F -e 'Received exception from server' -e 'Code: 82' -e '(query: ' - sleep 0.0$RANDOM +function thread_detach_attach { + while true; do + $CLICKHOUSE_CLIENT --query "DETACH DATABASE test_01150" 2>&1 | grep -v -F -e 'Received exception from server' -e 'Code: 219' -e '(query: ' + sleep 0.0$RANDOM + $CLICKHOUSE_CLIENT --query "ATTACH DATABASE test_01150" 2>&1 | grep -v -F -e 'Received exception from server' -e 'Code: 82' -e '(query: ' + sleep 0.0$RANDOM + done } -function thread_rename() -{ - $CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t1 TO test_01150.t2_tmp, test_01150.t2 TO test_01150.t1, test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F -e 'Received exception from server' -e '(query: ' | grep -v -P 'Code: (81|60|57|521)' - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t2 TO test_01150.t1, test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F -e 'Received exception from server' -e '(query: ' | grep -v -P 'Code: (81|60|57|521)' - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F -e 'Received exception from server' -e '(query: ' | grep -v -P 'Code: (81|60|57|521)' - sleep 0.0$RANDOM +function thread_rename { + while true; do + $CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t1 TO test_01150.t2_tmp, test_01150.t2 TO test_01150.t1, test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F -e 'Received exception from server' -e '(query: ' | grep -v -P 'Code: (81|60|57|521)' + sleep 0.0$RANDOM + $CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t2 TO test_01150.t1, test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F -e 'Received exception from server' -e '(query: ' | grep -v -P 'Code: (81|60|57|521)' + sleep 0.0$RANDOM + $CLICKHOUSE_CLIENT --query "RENAME TABLE test_01150.t2_tmp TO test_01150.t2" 2>&1 | grep -v -F -e 'Received exception from server' -e '(query: ' | grep -v -P 'Code: (81|60|57|521)' + sleep 0.0$RANDOM + done } export -f thread_detach_attach export -f thread_rename -clickhouse_client_loop_timeout 20 thread_detach_attach & -clickhouse_client_loop_timeout 20 thread_rename & - +timeout 20 bash -c "thread_detach_attach" & +timeout 20 bash -c 'thread_rename' & wait sleep 1 diff --git a/tests/queries/0_stateless/01154_move_partition_long.sh b/tests/queries/0_stateless/01154_move_partition_long.sh index 4a0ed7808da..7cefac28e22 100755 --- a/tests/queries/0_stateless/01154_move_partition_long.sh +++ b/tests/queries/0_stateless/01154_move_partition_long.sh @@ -26,85 +26,99 @@ wait #function create_drop_thread() #{ -# REPLICA=$(($RANDOM % 16)) -# $CLICKHOUSE_CLIENT -q "DROP TABLE src_$REPLICA;" -# arr=("$@") -# engine=${arr[$RANDOM % ${#arr[@]}]} -# $CLICKHOUSE_CLIENT -q "CREATE TABLE src_$REPLICA (p UInt64, k UInt64, v UInt64) ENGINE=$engine PARTITION BY p % 10 ORDER BY k" -# sleep 0.$RANDOM +# while true; do +# REPLICA=$(($RANDOM % 16)) +# $CLICKHOUSE_CLIENT -q "DROP TABLE src_$REPLICA;" +# arr=("$@") +# engine=${arr[$RANDOM % ${#arr[@]}]} +# $CLICKHOUSE_CLIENT -q "CREATE TABLE src_$REPLICA (p UInt64, k UInt64, v UInt64) ENGINE=$engine PARTITION BY p % 10 ORDER BY k" +# sleep 0.$RANDOM; +# done #} function insert_thread() { - REPLICA=$(($RANDOM % 16)) - LIMIT=$(($RANDOM % 100)) - $CLICKHOUSE_CLIENT -q "INSERT INTO $1_$REPLICA SELECT * FROM generateRandom('p UInt64, k UInt64, v UInt64') LIMIT $LIMIT" 2>/dev/null + while true; do + REPLICA=$(($RANDOM % 16)) + LIMIT=$(($RANDOM % 100)) + $CLICKHOUSE_CLIENT -q "INSERT INTO $1_$REPLICA SELECT * FROM generateRandom('p UInt64, k UInt64, v UInt64') LIMIT $LIMIT" 2>/dev/null + done } function move_partition_src_dst_thread() { - FROM_REPLICA=$(($RANDOM % 16)) - TO_REPLICA=$(($RANDOM % 16)) - PARTITION=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "ALTER TABLE src_$FROM_REPLICA MOVE PARTITION $PARTITION TO TABLE dst_$TO_REPLICA" 2>/dev/null - sleep 0.$RANDOM + while true; do + FROM_REPLICA=$(($RANDOM % 16)) + TO_REPLICA=$(($RANDOM % 16)) + PARTITION=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "ALTER TABLE src_$FROM_REPLICA MOVE PARTITION $PARTITION TO TABLE dst_$TO_REPLICA" 2>/dev/null + sleep 0.$RANDOM; + done } function replace_partition_src_src_thread() { - FROM_REPLICA=$(($RANDOM % 16)) - TO_REPLICA=$(($RANDOM % 16)) - PARTITION=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "ALTER TABLE src_$TO_REPLICA REPLACE PARTITION $PARTITION FROM src_$FROM_REPLICA" 2>/dev/null - sleep 0.$RANDOM + while true; do + FROM_REPLICA=$(($RANDOM % 16)) + TO_REPLICA=$(($RANDOM % 16)) + PARTITION=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "ALTER TABLE src_$TO_REPLICA REPLACE PARTITION $PARTITION FROM src_$FROM_REPLICA" 2>/dev/null + sleep 0.$RANDOM; + done } function drop_partition_thread() { - REPLICA=$(($RANDOM % 16)) - PARTITION=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT -q "ALTER TABLE dst_$REPLICA DROP PARTITION $PARTITION" 2>/dev/null - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 16)) + PARTITION=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "ALTER TABLE dst_$REPLICA DROP PARTITION $PARTITION" 2>/dev/null + sleep 0.$RANDOM; + done } function optimize_thread() { - REPLICA=$(($RANDOM % 16)) - TABLE="src" - if (( RANDOM % 2 )); then - TABLE="dst" - fi - $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE ${TABLE}_$REPLICA" 2>/dev/null - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 16)) + TABLE="src" + if (( RANDOM % 2 )); then + TABLE="dst" + fi + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE ${TABLE}_$REPLICA" 2>/dev/null + sleep 0.$RANDOM; + done } function drop_part_thread() { - REPLICA=$(($RANDOM % 16)) - part=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.parts WHERE active AND database='$CLICKHOUSE_DATABASE' and table='dst_$REPLICA' ORDER BY rand() LIMIT 1") - $CLICKHOUSE_CLIENT -q "ALTER TABLE dst_$REPLICA DROP PART '$part'" 2>/dev/null - sleep 0.$RANDOM + while true; do + REPLICA=$(($RANDOM % 16)) + part=$($CLICKHOUSE_CLIENT -q "SELECT name FROM system.parts WHERE active AND database='$CLICKHOUSE_DATABASE' and table='dst_$REPLICA' ORDER BY rand() LIMIT 1") + $CLICKHOUSE_CLIENT -q "ALTER TABLE dst_$REPLICA DROP PART '$part'" 2>/dev/null + sleep 0.$RANDOM; + done } #export -f create_drop_thread; -export -f insert_thread -export -f move_partition_src_dst_thread -export -f replace_partition_src_src_thread -export -f drop_partition_thread -export -f optimize_thread -export -f drop_part_thread +export -f insert_thread; +export -f move_partition_src_dst_thread; +export -f replace_partition_src_src_thread; +export -f drop_partition_thread; +export -f optimize_thread; +export -f drop_part_thread; TIMEOUT=60 -#clickhouse_client_loop_timeout $TIMEOUT "create_drop_thread ${engines[@]}" & -clickhouse_client_loop_timeout $TIMEOUT insert_thread src & -clickhouse_client_loop_timeout $TIMEOUT insert_thread src & -clickhouse_client_loop_timeout $TIMEOUT insert_thread dst & -clickhouse_client_loop_timeout $TIMEOUT move_partition_src_dst_thread & -clickhouse_client_loop_timeout $TIMEOUT replace_partition_src_src_thread & -clickhouse_client_loop_timeout $TIMEOUT drop_partition_thread & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread & -clickhouse_client_loop_timeout $TIMEOUT drop_part_thread & +#timeout $TIMEOUT bash -c "create_drop_thread ${engines[@]}" & +timeout $TIMEOUT bash -c 'insert_thread src' & +timeout $TIMEOUT bash -c 'insert_thread src' & +timeout $TIMEOUT bash -c 'insert_thread dst' & +timeout $TIMEOUT bash -c move_partition_src_dst_thread & +timeout $TIMEOUT bash -c replace_partition_src_src_thread & +timeout $TIMEOUT bash -c drop_partition_thread & +timeout $TIMEOUT bash -c optimize_thread & +timeout $TIMEOUT bash -c drop_part_thread & wait check_replication_consistency "dst_" "count(), sum(p), sum(k), sum(v)" diff --git a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh index 27658650b91..a7c87351ff4 100755 --- a/tests/queries/0_stateless/01164_detach_attach_partition_race.sh +++ b/tests/queries/0_stateless/01164_detach_attach_partition_race.sh @@ -12,30 +12,36 @@ $CLICKHOUSE_CLIENT -q "insert into mt values (3)" function thread_insert() { - $CLICKHOUSE_CLIENT -q "insert into mt values (rand())"; + while true; do + $CLICKHOUSE_CLIENT -q "insert into mt values (rand())"; + done } function thread_detach_attach() { - $CLICKHOUSE_CLIENT -q "alter table mt detach partition id 'all'"; - $CLICKHOUSE_CLIENT -q "alter table mt attach partition id 'all'"; + while true; do + $CLICKHOUSE_CLIENT -q "alter table mt detach partition id 'all'"; + $CLICKHOUSE_CLIENT -q "alter table mt attach partition id 'all'"; + done } function thread_drop_detached() { - $CLICKHOUSE_CLIENT --allow_drop_detached -q "alter table mt drop detached partition id 'all'"; + while true; do + $CLICKHOUSE_CLIENT --allow_drop_detached -q "alter table mt drop detached partition id 'all'"; + done } -export -f thread_insert -export -f thread_detach_attach -export -f thread_drop_detached +export -f thread_insert; +export -f thread_detach_attach; +export -f thread_drop_detached; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread_insert & -clickhouse_client_loop_timeout $TIMEOUT thread_detach_attach 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread_detach_attach 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT thread_drop_detached 2> /dev/null & +timeout $TIMEOUT bash -c thread_insert & +timeout $TIMEOUT bash -c thread_detach_attach 2> /dev/null & +timeout $TIMEOUT bash -c thread_detach_attach 2> /dev/null & +timeout $TIMEOUT bash -c thread_drop_detached 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh index e632841bd01..e4a23055ae6 100755 --- a/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh +++ b/tests/queries/0_stateless/01175_distributed_ddl_output_mode_long.sh @@ -36,7 +36,7 @@ function run_until_out_contains() RAND_COMMENT="01175_DDL_$RANDOM" LOG_COMMENT="${CLICKHOUSE_LOG_COMMENT}_$RAND_COMMENT" -CLICKHOUSE_CLIENT_WITH_SETTINGS=${CLICKHOUSE_CLIENT/--log_comment ${CLICKHOUSE_LOG_COMMENT}/--log_comment ${LOG_COMMENT}} +CLICKHOUSE_CLIENT_WITH_SETTINGS=${CLICKHOUSE_CLIENT/--log_comment=\'${CLICKHOUSE_LOG_COMMENT}\'/--log_comment=\'${LOG_COMMENT}\'} CLICKHOUSE_CLIENT_WITH_SETTINGS+=" --output_format_parallel_formatting=0 " CLICKHOUSE_CLIENT_WITH_SETTINGS+=" --distributed_ddl_entry_format_version=2 " diff --git a/tests/queries/0_stateless/01249_flush_interactive.sh b/tests/queries/0_stateless/01249_flush_interactive.sh index 2ab85e2fb6c..89167002ed5 100755 --- a/tests/queries/0_stateless/01249_flush_interactive.sh +++ b/tests/queries/0_stateless/01249_flush_interactive.sh @@ -28,5 +28,3 @@ while true; do [[ $(test) == $(echo -ne "0\n1\n2\n3\n4\n---\n0\n1\n2\n3\n4\n---\n") ]] && break sleep 1 done - -clickhouse_test_wait_queries 60 diff --git a/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh b/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh index 1235e013ff7..3c11dc5f772 100755 --- a/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh +++ b/tests/queries/0_stateless/01294_lazy_database_concurrent_recreate_reattach_and_show_tables_long.sh @@ -10,67 +10,94 @@ export CURR_DATABASE="test_lazy_01294_concurrent_${CLICKHOUSE_DATABASE}" function recreate_lazy_func1() { - $CLICKHOUSE_CLIENT -nm -q " - DETACH TABLE $CURR_DATABASE.log; - ATTACH TABLE $CURR_DATABASE.log; + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.log (a UInt64, b UInt64) ENGINE = Log; "; + + while true; do + $CLICKHOUSE_CLIENT -q " + DETACH TABLE $CURR_DATABASE.log; + "; + + $CLICKHOUSE_CLIENT -q " + ATTACH TABLE $CURR_DATABASE.log; + "; + done } function recreate_lazy_func2() { - $CLICKHOUSE_CLIENT -nm -q " - CREATE TABLE $CURR_DATABASE.tlog (a UInt64, b UInt64) ENGINE = TinyLog; - DROP TABLE $CURR_DATABASE.tlog; - " + while true; do + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.tlog (a UInt64, b UInt64) ENGINE = TinyLog; + "; + + $CLICKHOUSE_CLIENT -q " + DROP TABLE $CURR_DATABASE.tlog; + "; + done } function recreate_lazy_func3() { - $CLICKHOUSE_CLIENT -nm -q " - ATTACH TABLE $CURR_DATABASE.slog; - DETACH TABLE $CURR_DATABASE.slog; - " + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.slog (a UInt64, b UInt64) ENGINE = StripeLog; + "; + + while true; do + $CLICKHOUSE_CLIENT -q " + ATTACH TABLE $CURR_DATABASE.slog; + "; + + $CLICKHOUSE_CLIENT -q " + DETACH TABLE $CURR_DATABASE.slog; + "; + done } function recreate_lazy_func4() { - $CLICKHOUSE_CLIENT -nm -q " - CREATE TABLE $CURR_DATABASE.tlog2 (a UInt64, b UInt64) ENGINE = TinyLog; - DROP TABLE $CURR_DATABASE.tlog2; - " + while true; do + $CLICKHOUSE_CLIENT -q " + CREATE TABLE $CURR_DATABASE.tlog2 (a UInt64, b UInt64) ENGINE = TinyLog; + "; + + $CLICKHOUSE_CLIENT -q " + DROP TABLE $CURR_DATABASE.tlog2; + "; + done } function test_func() { - for table in log tlog slog tlog2; do - $CLICKHOUSE_CLIENT -q "SYSTEM STOP TTL MERGES $CURR_DATABASE.$table" >& /dev/null + while true; do + for table in log tlog slog tlog2; do + $CLICKHOUSE_CLIENT -q "SYSTEM STOP TTL MERGES $CURR_DATABASE.$table" >& /dev/null + done done } -export -f recreate_lazy_func1 -export -f recreate_lazy_func2 -export -f recreate_lazy_func3 -export -f recreate_lazy_func4 -export -f test_func +export -f recreate_lazy_func1; +export -f recreate_lazy_func2; +export -f recreate_lazy_func3; +export -f recreate_lazy_func4; +export -f test_func; ${CLICKHOUSE_CLIENT} -n -q " DROP DATABASE IF EXISTS $CURR_DATABASE; CREATE DATABASE $CURR_DATABASE ENGINE = Lazy(1); - - CREATE TABLE $CURR_DATABASE.log (a UInt64, b UInt64) ENGINE = Log; - CREATE TABLE $CURR_DATABASE.slog (a UInt64, b UInt64) ENGINE = StripeLog; " TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func1 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func2 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func3 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT recreate_lazy_func4 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT test_func 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func1 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func2 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func3 2> /dev/null & +timeout $TIMEOUT bash -c recreate_lazy_func4 2> /dev/null & +timeout $TIMEOUT bash -c test_func 2> /dev/null & wait sleep 1 diff --git a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh index ed7e5937bd0..010c66b9bb1 100755 --- a/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01301_aggregate_state_exception_memory_leak.sh @@ -5,6 +5,15 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +function test() +{ + for _ in {1..1000}; do + $CLICKHOUSE_CLIENT --max_memory_usage 1G <<< "SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10"; + done +} + +export -f test; + # If the memory leak exists, it will lead to OOM fairly quickly. -clickhouse_client_loop_timeout 30 $CLICKHOUSE_CLIENT --max_memory_usage 1G -q 'SELECT uniqExactState(number) FROM system.numbers_mt GROUP BY number % 10' 2>&1 | grep -o -F 'Memory limit (for query) exceeded' | uniq +timeout 30 bash -c test 2>&1 | grep -o -F 'Memory limit (for query) exceeded' | uniq echo 'Ok' diff --git a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh index 52b635fe5df..7e10ab475d4 100755 --- a/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh +++ b/tests/queries/0_stateless/01302_aggregate_state_exception_memory_leak.sh @@ -7,10 +7,12 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function test() { - $CLICKHOUSE_CLIENT --query "SELECT groupArrayIfState(('Hello, world' AS s) || s || s || s || s || s || s || s || s || s, NOT throwIf(number > 10000000, 'Ok')) FROM system.numbers_mt GROUP BY number % 10"; + for _ in {1..250}; do + $CLICKHOUSE_CLIENT --query "SELECT groupArrayIfState(('Hello, world' AS s) || s || s || s || s || s || s || s || s || s, NOT throwIf(number > 10000000, 'Ok')) FROM system.numbers_mt GROUP BY number % 10"; + done } -export -f test +export -f test; # If the memory leak exists, it will lead to OOM fairly quickly. -clickhouse_client_loop_timeout 30 test 2>&1 | grep -o -F 'Ok' | uniq +timeout 30 bash -c test 2>&1 | grep -o -F 'Ok' | uniq diff --git a/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh b/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh index 9b4159fa002..1d2d4516b9c 100755 --- a/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh +++ b/tests/queries/0_stateless/01305_replica_create_drop_zookeeper.sh @@ -9,17 +9,20 @@ set -e function thread() { - $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1 SYNC; - CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | - grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now| were removed by another replica|Removing leftovers from table|Another replica was suddenly created|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time|^\(query: ' + while true; do + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS test_table_$1 SYNC; + CREATE TABLE test_table_$1 (a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/alter_table', 'r_$1') ORDER BY tuple();" 2>&1 | + grep -vP '(^$)|(^Received exception from server)|(^\d+\. )|because the last replica of the table was dropped right now|is already started to be removing by another replica right now| were removed by another replica|Removing leftovers from table|Another replica was suddenly created|was created by another server at the same moment|was suddenly removed|some other replicas were created at the same time|^\(query: ' + done } -export -f thread +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread 1 & -clickhouse_client_loop_timeout $TIMEOUT thread 2 & +timeout $TIMEOUT bash -c 'thread 1' & +timeout $TIMEOUT bash -c 'thread 2' & wait diff --git a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh index d538cafd79d..758ec4825e0 100755 --- a/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh +++ b/tests/queries/0_stateless/01320_create_sync_race_condition_zookeeper.sh @@ -12,19 +12,19 @@ $CLICKHOUSE_CLIENT --query "CREATE DATABASE test_01320 ENGINE=Ordinary" # Diff function thread1() { - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;" + while true; do $CLICKHOUSE_CLIENT -n --query "CREATE TABLE test_01320.r (x UInt64) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table', 'r') ORDER BY x; DROP TABLE test_01320.r;"; done } function thread2() { - $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA test_01320.r" 2>/dev/null + while true; do $CLICKHOUSE_CLIENT --query "SYSTEM SYNC REPLICA test_01320.r" 2>/dev/null; done } export -f thread1 export -f thread2 -clickhouse_client_loop_timeout 10 thread1 & -clickhouse_client_loop_timeout 10 thread2 & +timeout 10 bash -c thread1 & +timeout 10 bash -c thread2 & wait diff --git a/tests/queries/0_stateless/01412_cache_dictionary_race.sh b/tests/queries/0_stateless/01412_cache_dictionary_race.sh index 7d189518a58..165a461193d 100755 --- a/tests/queries/0_stateless/01412_cache_dictionary_race.sh +++ b/tests/queries/0_stateless/01412_cache_dictionary_race.sh @@ -26,41 +26,45 @@ LAYOUT(CACHE(SIZE_IN_CELLS 3)); function dict_get_thread() { - $CLICKHOUSE_CLIENT --query "SELECT dictGetString('ordinary_db.dict1', 'third_column', toUInt64(rand() % 1000)) from numbers(2)" &>/dev/null + while true; do + $CLICKHOUSE_CLIENT --query "SELECT dictGetString('ordinary_db.dict1', 'third_column', toUInt64(rand() % 1000)) from numbers(2)" &>/dev/null + done } function drop_create_table_thread() { - $CLICKHOUSE_CLIENT -n --query "CREATE TABLE ordinary_db.table_for_dict_real ( - key_column UInt64, - second_column UInt8, - third_column String - ) - ENGINE MergeTree() ORDER BY tuple(); - INSERT INTO ordinary_db.table_for_dict_real SELECT number, number, toString(number) from numbers(2); - CREATE VIEW ordinary_db.view_for_dict AS SELECT key_column, second_column, third_column from ordinary_db.table_for_dict_real WHERE sleepEachRow(1) == 0; + while true; do + $CLICKHOUSE_CLIENT -n --query "CREATE TABLE ordinary_db.table_for_dict_real ( + key_column UInt64, + second_column UInt8, + third_column String + ) + ENGINE MergeTree() ORDER BY tuple(); + INSERT INTO ordinary_db.table_for_dict_real SELECT number, number, toString(number) from numbers(2); + CREATE VIEW ordinary_db.view_for_dict AS SELECT key_column, second_column, third_column from ordinary_db.table_for_dict_real WHERE sleepEachRow(1) == 0; " - sleep 10 + sleep 10 - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ordinary_db.table_for_dict_real" - sleep 10 + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS ordinary_db.table_for_dict_real" + sleep 10 + done } -export -f dict_get_thread -export -f drop_create_table_thread +export -f dict_get_thread; +export -f drop_create_table_thread; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT bash -c drop_create_table_thread 2> /dev/null & +timeout $TIMEOUT bash -c drop_create_table_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01444_create_table_drop_database_race.sh b/tests/queries/0_stateless/01444_create_table_drop_database_race.sh index 51989685e05..eb231e71525 100755 --- a/tests/queries/0_stateless/01444_create_table_drop_database_race.sh +++ b/tests/queries/0_stateless/01444_create_table_drop_database_race.sh @@ -11,14 +11,18 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) function thread1() { - # ${CLICKHOUSE_CLIENT} --query="SHOW TABLES FROM test_01444" - ${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS test_01444" 2>&1| grep -F "Code: " | grep -Fv "Code: 219" - ${CLICKHOUSE_CLIENT} --query="CREATE DATABASE IF NOT EXISTS test_01444" + while true; do +# ${CLICKHOUSE_CLIENT} --query="SHOW TABLES FROM test_01444" + ${CLICKHOUSE_CLIENT} --query="DROP DATABASE IF EXISTS test_01444" 2>&1| grep -F "Code: " | grep -Fv "Code: 219" + ${CLICKHOUSE_CLIENT} --query="CREATE DATABASE IF NOT EXISTS test_01444" + done } function thread2() { - ${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS test_01444.t$RANDOM (x UInt8) ENGINE = MergeTree ORDER BY tuple()" 2>/dev/null + while true; do + ${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS test_01444.t$RANDOM (x UInt8) ENGINE = MergeTree ORDER BY tuple()" 2>/dev/null + done } export -f thread1 @@ -26,9 +30,9 @@ export -f thread2 TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT thread1 & -clickhouse_client_loop_timeout $TIMEOUT thread2 & -clickhouse_client_loop_timeout $TIMEOUT thread2 & +timeout $TIMEOUT bash -c thread1 & +timeout $TIMEOUT bash -c thread2 & +timeout $TIMEOUT bash -c thread2 & wait diff --git a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh index 27fa74f332c..d83343b3cb3 100755 --- a/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh +++ b/tests/queries/0_stateless/01454_storagememory_data_race_challenge.sh @@ -10,13 +10,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS mem" $CLICKHOUSE_CLIENT -q "CREATE TABLE mem (x UInt64) engine = Memory" -function f() -{ +function f { + for _ in $(seq 1 300); do $CLICKHOUSE_CLIENT -q "SELECT count() FROM (SELECT * FROM mem SETTINGS max_threads=2) FORMAT Null;" + done } -function g() -{ +function g { + for _ in $(seq 1 100); do $CLICKHOUSE_CLIENT -n -q " INSERT INTO mem SELECT number FROM numbers(1000000); INSERT INTO mem SELECT number FROM numbers(1000000); @@ -29,13 +30,14 @@ function g() INSERT INTO mem VALUES (1); TRUNCATE TABLE mem; " + done } -export -f f -export -f g +export -f f; +export -f g; -clickhouse_client_loop_timeout 30 f > /dev/null & -clickhouse_client_loop_timeout 30 g > /dev/null & +timeout 30 bash -c f > /dev/null & +timeout 30 bash -c g > /dev/null & wait $CLICKHOUSE_CLIENT -q "DROP TABLE mem" diff --git a/tests/queries/0_stateless/01542_dictionary_load_exception_race.sh b/tests/queries/0_stateless/01542_dictionary_load_exception_race.sh index 39ef530f6e9..981beb785a5 100755 --- a/tests/queries/0_stateless/01542_dictionary_load_exception_race.sh +++ b/tests/queries/0_stateless/01542_dictionary_load_exception_race.sh @@ -18,25 +18,27 @@ $CLICKHOUSE_CLIENT --query "CREATE DICTIONARY ordinary_db.dict1 ( key_column UIn function dict_get_thread() { - $CLICKHOUSE_CLIENT --query "SELECT dictGetString('ordinary_db.dict1', 'third_column', toUInt64(rand() % 1000)) from numbers(2)" &>/dev/null + while true; do + $CLICKHOUSE_CLIENT --query "SELECT dictGetString('ordinary_db.dict1', 'third_column', toUInt64(rand() % 1000)) from numbers(2)" &>/dev/null + done } -export -f dict_get_thread +export -f dict_get_thread; TIMEOUT=10 -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & +timeout $TIMEOUT bash -c dict_get_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh index 30e1eff87da..acaa2cfcd25 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh @@ -13,32 +13,36 @@ $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_kill SELECT number, to function alter_thread { - TYPE=$($CLICKHOUSE_CLIENT --query "SELECT type FROM system.columns WHERE table='concurrent_mutate_kill' and database='${CLICKHOUSE_DATABASE}' and name='value'") - if [ "$TYPE" == "String" ]; then - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_kill MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2" - else - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_kill MODIFY COLUMN value String SETTINGS replication_alter_partitions_sync=2" - fi + while true; do + TYPE=$($CLICKHOUSE_CLIENT --query "SELECT type FROM system.columns WHERE table='concurrent_mutate_kill' and database='${CLICKHOUSE_DATABASE}' and name='value'") + if [ "$TYPE" == "String" ]; then + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_kill MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2" + else + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_mutate_kill MODIFY COLUMN value String SETTINGS replication_alter_partitions_sync=2" + fi + done } function kill_mutation_thread { - # find any mutation and kill it - mutation_id=$($CLICKHOUSE_CLIENT --query "SELECT mutation_id FROM system.mutations WHERE is_done=0 and database='${CLICKHOUSE_DATABASE}' and table='concurrent_mutate_kill' LIMIT 1") - if [ ! -z "$mutation_id" ]; then - $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='$mutation_id' and table='concurrent_mutate_kill' and database='${CLICKHOUSE_DATABASE}'" 1> /dev/null - sleep 1 - fi + while true; do + # find any mutation and kill it + mutation_id=$($CLICKHOUSE_CLIENT --query "SELECT mutation_id FROM system.mutations WHERE is_done=0 and database='${CLICKHOUSE_DATABASE}' and table='concurrent_mutate_kill' LIMIT 1") + if [ ! -z "$mutation_id" ]; then + $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='$mutation_id' and table='concurrent_mutate_kill' and database='${CLICKHOUSE_DATABASE}'" 1> /dev/null + sleep 1 + fi + done } -export -f alter_thread -export -f kill_mutation_thread +export -f alter_thread; +export -f kill_mutation_thread; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT kill_mutation_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c kill_mutation_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh index 3ee321371bf..f8f3ccd6dd6 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh @@ -27,34 +27,38 @@ for i in $(seq $REPLICAS); do $CLICKHOUSE_CLIENT --query "SELECT sum(toUInt64(value)) FROM concurrent_kill_$i" done -function alter_thread() +function alter_thread { - REPLICA=$(($RANDOM % 5 + 1)) - TYPE=$($CLICKHOUSE_CLIENT --query "SELECT type FROM system.columns WHERE table='concurrent_kill_$REPLICA' and database='${CLICKHOUSE_DATABASE}' and name='value'") - if [ "$TYPE" == "String" ]; then - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_kill_$REPLICA MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2" - else - $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_kill_$REPLICA MODIFY COLUMN value String SETTINGS replication_alter_partitions_sync=2" - fi + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + TYPE=$($CLICKHOUSE_CLIENT --query "SELECT type FROM system.columns WHERE table='concurrent_kill_$REPLICA' and database='${CLICKHOUSE_DATABASE}' and name='value'") + if [ "$TYPE" == "String" ]; then + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_kill_$REPLICA MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2" + else + $CLICKHOUSE_CLIENT --query "ALTER TABLE concurrent_kill_$REPLICA MODIFY COLUMN value String SETTINGS replication_alter_partitions_sync=2" + fi + done } -function kill_mutation_thread() +function kill_mutation_thread { - # find any mutation and kill it - mutation_id=$($CLICKHOUSE_CLIENT --query "SELECT mutation_id FROM system.mutations WHERE is_done = 0 and table like 'concurrent_kill_%' and database='${CLICKHOUSE_DATABASE}' LIMIT 1") - if [ ! -z "$mutation_id" ]; then - $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='$mutation_id' and table like 'concurrent_kill_%' and database='${CLICKHOUSE_DATABASE}'" 1> /dev/null - sleep 1 - fi + while true; do + # find any mutation and kill it + mutation_id=$($CLICKHOUSE_CLIENT --query "SELECT mutation_id FROM system.mutations WHERE is_done = 0 and table like 'concurrent_kill_%' and database='${CLICKHOUSE_DATABASE}' LIMIT 1") + if [ ! -z "$mutation_id" ]; then + $CLICKHOUSE_CLIENT --query "KILL MUTATION WHERE mutation_id='$mutation_id' and table like 'concurrent_kill_%' and database='${CLICKHOUSE_DATABASE}'" 1> /dev/null + sleep 1 + fi + done } -export -f alter_thread -export -f kill_mutation_thread +export -f alter_thread; +export -f kill_mutation_thread; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT alter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT kill_mutation_thread 2> /dev/null & +timeout $TIMEOUT bash -c alter_thread 2> /dev/null & +timeout $TIMEOUT bash -c kill_mutation_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/01602_max_distributed_connections.sh b/tests/queries/0_stateless/01602_max_distributed_connections.sh index 0869d2d3a68..ed835a8768f 100755 --- a/tests/queries/0_stateless/01602_max_distributed_connections.sh +++ b/tests/queries/0_stateless/01602_max_distributed_connections.sh @@ -15,14 +15,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) i=0 retries=30 while [[ $i -lt $retries ]]; do - clickhouse_client_timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " + timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && break ((++i)) done i=0 retries=30 while [[ $i -lt $retries ]]; do - clickhouse_client_timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " + timeout 10 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 10 --query " SELECT sleep(1.5) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && break ((++i)) done @@ -30,13 +30,10 @@ done # If max_distributed_connections is low and async_socket_for_remote is disabled, # the concurrency of distributed queries will be also low. -clickhouse_client_timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " +timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=0 && echo 'Fail' -clickhouse_client_timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " +timeout 1 ${CLICKHOUSE_CLIENT} --max_threads 1 --max_distributed_connections 1 --async_socket_for_remote 0 --query " SELECT sleep(0.15) FROM remote('127.{1..10}', system.one) FORMAT Null" --prefer_localhost_replica=1 && echo 'Fail' -# FIXME -clickhouse_test_wait_queries 5 - echo 'Ok' diff --git a/tests/queries/0_stateless/01632_tinylog_read_write.sh b/tests/queries/0_stateless/01632_tinylog_read_write.sh index 42211168d45..3f41bcc5924 100755 --- a/tests/queries/0_stateless/01632_tinylog_read_write.sh +++ b/tests/queries/0_stateless/01632_tinylog_read_write.sh @@ -9,32 +9,35 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT --multiquery --query "DROP TABLE IF EXISTS test; CREATE TABLE IF NOT EXISTS test (x UInt64, s Array(Nullable(String))) ENGINE = TinyLog;" -function thread_select() -{ - $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" - sleep 0.0$RANDOM +function thread_select { + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM test FORMAT Null" + sleep 0.0$RANDOM + done } -function thread_insert() -{ - $CLICKHOUSE_CLIENT --query "INSERT INTO test VALUES (1, ['Hello'])" - sleep 0.0$RANDOM +function thread_insert { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO test VALUES (1, ['Hello'])" + sleep 0.0$RANDOM + done } export -f thread_select export -f thread_insert + # Do randomized queries and expect nothing extraordinary happens. -clickhouse_client_loop_timeout 10 thread_select & -clickhouse_client_loop_timeout 10 thread_select & -clickhouse_client_loop_timeout 10 thread_select & -clickhouse_client_loop_timeout 10 thread_select & +timeout 10 bash -c 'thread_select' & +timeout 10 bash -c 'thread_select' & +timeout 10 bash -c 'thread_select' & +timeout 10 bash -c 'thread_select' & -clickhouse_client_loop_timeout 10 thread_insert & -clickhouse_client_loop_timeout 10 thread_insert & -clickhouse_client_loop_timeout 10 thread_insert & -clickhouse_client_loop_timeout 10 thread_insert & +timeout 10 bash -c 'thread_insert' & +timeout 10 bash -c 'thread_insert' & +timeout 10 bash -c 'thread_insert' & +timeout 10 bash -c 'thread_insert' & wait echo "Done" diff --git a/tests/queries/0_stateless/01671_ddl_hang_timeout_long.sh b/tests/queries/0_stateless/01671_ddl_hang_timeout_long.sh index a06a9ec7057..93c4451524c 100755 --- a/tests/queries/0_stateless/01671_ddl_hang_timeout_long.sh +++ b/tests/queries/0_stateless/01671_ddl_hang_timeout_long.sh @@ -5,24 +5,26 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -function thread_create_drop_table() -{ - REPLICA=$(($RANDOM % 10)) - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_01671', 'r_$REPLICA') order by x" 2>/dev/null - sleep 0.0$RANDOM - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t1" +function thread_create_drop_table { + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS t1 (x UInt64, s Array(Nullable(String))) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_01671', 'r_$REPLICA') order by x" 2>/dev/null + sleep 0.0$RANDOM + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t1" + done } -function thread_alter_table() -{ - $CLICKHOUSE_CLIENT --query "ALTER TABLE $CLICKHOUSE_DATABASE.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" >/dev/null 2>&1 - sleep 0.0$RANDOM +function thread_alter_table { + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE $CLICKHOUSE_DATABASE.t1 on cluster test_shard_localhost ADD COLUMN newcol UInt32" >/dev/null 2>&1 + sleep 0.0$RANDOM + done } export -f thread_create_drop_table export -f thread_alter_table -clickhouse_client_loop_timeout 20 thread_create_drop_table & -clickhouse_client_loop_timeout 20 thread_alter_table & +timeout 20 bash -c "thread_create_drop_table" & +timeout 20 bash -c 'thread_alter_table' & wait sleep 1 diff --git a/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh b/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh index 1cd243675c6..5bb10220f7f 100755 --- a/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh +++ b/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh @@ -11,34 +11,47 @@ set -o errexit set -o pipefail echo " - DROP TABLE IF EXISTS storage_join_race; - CREATE TABLE storage_join_race (x UInt64, y UInt64) Engine = Join(ALL, FULL, x); + DROP TABLE IF EXISTS storage_join_race; + CREATE TABLE storage_join_race (x UInt64, y UInt64) Engine = Join(ALL, FULL, x); " | $CLICKHOUSE_CLIENT -n function read_thread_big() { - $CLICKHOUSE_CLIENT -n -q "SELECT * FROM ( SELECT number AS x FROM numbers(100000) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null" + while true; do + echo " + SELECT * FROM ( SELECT number AS x FROM numbers(100000) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done } function read_thread_small() { - $CLICKHOUSE_CLIENT -n -q "SELECT * FROM ( SELECT number AS x FROM numbers(10) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null" + while true; do + echo " + SELECT * FROM ( SELECT number AS x FROM numbers(10) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done } function read_thread_select() { - $CLICKHOUSE_CLIENT -n -q "SELECT * FROM storage_join_race FORMAT Null" + while true; do + echo " + SELECT * FROM storage_join_race FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done } -export -f read_thread_big -export -f read_thread_small -export -f read_thread_select +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f read_thread_big; +export -f read_thread_small; +export -f read_thread_select; TIMEOUT=20 -clickhouse_client_loop_timeout $TIMEOUT read_thread_big 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT read_thread_small 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT read_thread_select 2> /dev/null & +timeout $TIMEOUT bash -c read_thread_big 2> /dev/null & +timeout $TIMEOUT bash -c read_thread_small 2> /dev/null & +timeout $TIMEOUT bash -c read_thread_select 2> /dev/null & echo " INSERT INTO storage_join_race SELECT number AS x, number AS y FROM numbers (10000000); diff --git a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh index a29d0661621..a3682a3a74b 100755 --- a/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh +++ b/tests/queries/0_stateless/01921_concurrent_ttl_and_normal_merges_zookeeper_long.sh @@ -29,34 +29,38 @@ done function optimize_thread { - REPLICA=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_table$REPLICA FINAl" + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE ttl_table$REPLICA FINAl" + done } function insert_thread { - REPLICA=$(($RANDOM % 5 + 1)) - $CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)" - $CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)" - $CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)" + while true; do + REPLICA=$(($RANDOM % 5 + 1)) + $CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)" + $CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)" + $CLICKHOUSE_CLIENT --optimize_on_insert=0 --query "INSERT INTO ttl_table$REPLICA SELECT now() + rand() % 5 - rand() % 3 FROM numbers(5)" + done } -export -f insert_thread -export -f optimize_thread +export -f insert_thread; +export -f optimize_thread; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT insert_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c insert_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & +timeout $TIMEOUT bash -c optimize_thread 2> /dev/null & wait for i in $(seq 1 $NUM_REPLICAS); do diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index c6f40d35855..81d70f79574 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -37,18 +37,24 @@ function insert3() function select1() { - ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" + while true; do + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM async_inserts FORMAT Null" + done } function select2() { - ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" + while true; do + ${CLICKHOUSE_CLIENT} -q "SELECT * FROM system.asynchronous_inserts FORMAT Null" + done } function truncate1() { - sleep 0.1 - ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE async_inserts" + while true; do + sleep 0.1 + ${CLICKHOUSE_CLIENT} -q "TRUNCATE TABLE async_inserts" + done } ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts" @@ -69,9 +75,9 @@ for _ in {1..5}; do timeout $TIMEOUT bash -c insert3 & done -clickhouse_client_loop_timeout $TIMEOUT select1 & -clickhouse_client_loop_timeout $TIMEOUT select2 & -clickhouse_client_loop_timeout $TIMEOUT truncate1 & +timeout $TIMEOUT bash -c select1 & +timeout $TIMEOUT bash -c select2 & +timeout $TIMEOUT bash -c truncate1 & wait echo "OK" diff --git a/tests/queries/0_stateless/02015_shard_crash_clang_12_build.sh b/tests/queries/0_stateless/02015_shard_crash_clang_12_build.sh index 062b8512bff..57a263abac5 100755 --- a/tests/queries/0_stateless/02015_shard_crash_clang_12_build.sh +++ b/tests/queries/0_stateless/02015_shard_crash_clang_12_build.sh @@ -19,24 +19,26 @@ $CLICKHOUSE_CLIENT --insert_distributed_sync=0 --network_compression_method='zst function select_thread() { - $CLICKHOUSE_CLIENT --insert_distributed_sync=0 --network_compression_method='zstd' --query "SELECT count() FROM local" >/dev/null - $CLICKHOUSE_CLIENT --insert_distributed_sync=0 --network_compression_method='zstd' --query "SELECT count() FROM distributed" >/dev/null + while true; do + $CLICKHOUSE_CLIENT --insert_distributed_sync=0 --network_compression_method='zstd' --query "SELECT count() FROM local" >/dev/null + $CLICKHOUSE_CLIENT --insert_distributed_sync=0 --network_compression_method='zstd' --query "SELECT count() FROM distributed" >/dev/null + done } -export -f select_thread +export -f select_thread; TIMEOUT=30 -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & +timeout $TIMEOUT bash -c select_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/02030_rocksdb_race_long.sh b/tests/queries/0_stateless/02030_rocksdb_race_long.sh index ec20007892a..88c30852c86 100755 --- a/tests/queries/0_stateless/02030_rocksdb_race_long.sh +++ b/tests/queries/0_stateless/02030_rocksdb_race_long.sh @@ -11,29 +11,38 @@ set -o errexit set -o pipefail echo " - DROP TABLE IF EXISTS rocksdb_race; - CREATE TABLE rocksdb_race (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); + DROP TABLE IF EXISTS rocksdb_race; + CREATE TABLE rocksdb_race (key String, value UInt32) Engine=EmbeddedRocksDB PRIMARY KEY(key); INSERT INTO rocksdb_race SELECT '1_' || toString(number), number FROM numbers(100000); " | $CLICKHOUSE_CLIENT -n function read_stat_thread() { - $CLICKHOUSE_CLIENT -n -q 'SELECT * FROM system.rocksdb FORMAT Null' + while true; do + echo " + SELECT * FROM system.rocksdb FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done } function truncate_thread() { - sleep 3s; - $CLICKHOUSE_CLIENT -n -q 'TRUNCATE TABLE rocksdb_race' + while true; do + sleep 3s; + echo " + TRUNCATE TABLE rocksdb_race; + " | $CLICKHOUSE_CLIENT -n + done } -export -f read_stat_thread -export -f truncate_thread +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f read_stat_thread; +export -f truncate_thread; TIMEOUT=20 -clickhouse_client_loop_timeout $TIMEOUT read_stat_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT truncate_thread 2> /dev/null & +timeout $TIMEOUT bash -c read_stat_thread 2> /dev/null & +timeout $TIMEOUT bash -c truncate_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh index 4b7d15c01c3..7e8579f7cbe 100755 --- a/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh +++ b/tests/queries/0_stateless/02044_url_glob_parallel_connection_refused.sh @@ -7,15 +7,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) i=0 retries=5 -client_opts=( - --http_max_tries 1 - --max_execution_time 3 - --max_threads 10 - --query "SELECT * FROM url('http://128.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" - --format Null -) # Connecting to wrong address and checking for race condition while [[ $i -lt $retries ]]; do - clickhouse_client_timeout 4s ${CLICKHOUSE_CLIENT} "${client_opts[@]}" 2>/dev/null + timeout 5s ${CLICKHOUSE_CLIENT} --max_threads 10 --query "SELECT * FROM url('http://128.0.0.{1..10}:${CLICKHOUSE_PORT_HTTP}/?query=SELECT+sleep(1)', TSV, 'x UInt8')" --format Null 2>/dev/null ((++i)) done diff --git a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh index 84a1befe421..2deaf788ecf 100755 --- a/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh +++ b/tests/queries/0_stateless/02122_4letter_words_stress_zookeeper.sh @@ -17,26 +17,28 @@ function four_letter_thread() function create_drop_thread() { - num=$(($RANDOM % 10 + 1)) - $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table$num (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table$num', '0') ORDER BY key" - sleep 0.$RANDOM - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" + while true; do + num=$(($RANDOM % 10 + 1)) + $CLICKHOUSE_CLIENT --query "CREATE TABLE test_table$num (key UInt64, value1 UInt8, value2 UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_table$num', '0') ORDER BY key" + sleep 0.$RANDOM + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS test_table$num" + done } -export -f four_letter_thread -export -f create_drop_thread +export -f four_letter_thread; +export -f create_drop_thread; TIMEOUT=15 -timeout $TIMEOUT four_letter_thread 2> /dev/null & -timeout $TIMEOUT four_letter_thread 2> /dev/null & -timeout $TIMEOUT four_letter_thread 2> /dev/null & -timeout $TIMEOUT four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & +timeout $TIMEOUT bash -c four_letter_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT create_drop_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT create_drop_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT create_drop_thread 2> /dev/null & -clickhouse_client_loop_timeout $TIMEOUT create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & +timeout $TIMEOUT bash -c create_drop_thread 2> /dev/null & wait diff --git a/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh b/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh index bc43f460289..1b2197ef943 100755 --- a/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh +++ b/tests/queries/0_stateless/02124_buffer_with_type_map_long.sh @@ -9,12 +9,16 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_buffer_map(m1 Map(String, UInt64), m2 Map( function insert1 { - $CLICKHOUSE_CLIENT -q "INSERT INTO t_buffer_map SELECT (range(10), range(10)), (range(10), range(10)) from numbers(100)" + while true; do + $CLICKHOUSE_CLIENT -q "INSERT INTO t_buffer_map SELECT (range(10), range(10)), (range(10), range(10)) from numbers(100)" + done } function select1 { - $CLICKHOUSE_CLIENT -q "SELECT * FROM t_buffer_map" 2> /dev/null > /dev/null + while true; do + $CLICKHOUSE_CLIENT -q "SELECT * FROM t_buffer_map" 2> /dev/null > /dev/null + done } TIMEOUT=10 @@ -22,8 +26,8 @@ TIMEOUT=10 export -f insert1 export -f select1 -clickhouse_client_loop_timeout $TIMEOUT insert1 & -clickhouse_client_loop_timeout $TIMEOUT select1 & +timeout $TIMEOUT bash -c insert1 & +timeout $TIMEOUT bash -c select1 & wait diff --git a/tests/queries/shell_config.sh b/tests/queries/shell_config.sh index 7699e803f76..ce5947d95ed 100644 --- a/tests/queries/shell_config.sh +++ b/tests/queries/shell_config.sh @@ -19,9 +19,9 @@ export CLICKHOUSE_TEST_UNIQUE_NAME="${CLICKHOUSE_TEST_NAME}_${CLICKHOUSE_DATABAS [ -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} " -[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_CLIENT_OPT0+=" --log_comment $(printf '%q' ${CLICKHOUSE_LOG_COMMENT}) " +[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_CLIENT_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' " [ -v CLICKHOUSE_DATABASE ] && CLICKHOUSE_BENCHMARK_OPT0+=" --database=${CLICKHOUSE_DATABASE} " -[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment $(printf '%q' ${CLICKHOUSE_LOG_COMMENT}) " +[ -v CLICKHOUSE_LOG_COMMENT ] && CLICKHOUSE_BENCHMARK_OPT0+=" --log_comment='${CLICKHOUSE_LOG_COMMENT}' " export CLICKHOUSE_BINARY=${CLICKHOUSE_BINARY:="clickhouse"} # client @@ -129,37 +129,3 @@ function clickhouse_client_removed_host_parameter() # bash regex magic is arcane, but version dependant and weak; sed or awk are not really portable. $(echo "$CLICKHOUSE_CLIENT" | python3 -c "import sys, re; print(re.sub('--host(\s+|=)[^\s]+', '', sys.stdin.read()))") "$@" } - -function clickhouse_client_timeout() -{ - local timeout=$1 && shift - timeout -s INT "$timeout" "$@" -} -# Helper function to stop the clickhouse-client after SIGINT properly. -function clickhouse_client_loop_timeout() -{ - local timeout=$1 && shift - - local cmd - cmd="$(printf '%q ' "$@")" - - timeout -s INT "$timeout" bash -c "trap 'STOP_THE_LOOP=1' INT; while true; do [ ! -v STOP_THE_LOOP ] || break; $cmd; done" -} -# wait for queries to be finished -function clickhouse_test_wait_queries() -{ - local timeout=${1:-"600"} && shift - local query_id="wait-$CLICKHOUSE_TEST_UNIQUE_NAME" - local query="SELECT count() FROM system.processes WHERE current_database = '$CLICKHOUSE_DATABASE' AND query_id != '$query_id'" - local i=0 - (( timeout*=2 )) - while [[ "$(${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&query_id=$query_id" --data-binary "$query")" != "0" ]]; do - sleep 0.5 - - (( ++i )) - if [[ $i -gt $timeout ]]; then - echo "clickhouse_test_wait_queries: timeout exceeded" - exit 1 - fi - done -} From 1ae919a18aa37fdea552c3c589dfccbf1fd83263 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 19:23:58 +0200 Subject: [PATCH 019/101] Revert "Merge pull request #35865 from azat/clickhouse-test-left-queries" This reverts commit 18d094d79d239bbb8d14ea509b6dc7bf2a74e0fb, reversing changes made to 224f4dc620da606a7b5b52f855de607e43f0cc58. --- docker/test/stateless/run.sh | 2 - tests/clickhouse-test | 16 ---- .../01085_max_distributed_connections_http.sh | 2 - .../01502_log_tinylog_deadlock_race.sh | 78 ++++++++++--------- .../0_stateless/01675_data_type_coroutine.sh | 6 -- .../01675_data_type_coroutine_2.sh | 6 -- .../01731_async_task_queue_wait.sh | 2 - .../0_stateless/02104_overcommit_memory.sh | 23 +++--- .../02151_http_s_structure_set_eof.sh | 7 -- 9 files changed, 55 insertions(+), 87 deletions(-) diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index e2ad91220d3..50632630f1f 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -92,8 +92,6 @@ function run_tests() if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ADDITIONAL_OPTIONS+=('--replicated-database') - # Cannot be used with replicated database, due to distributed_ddl_output_mode=none - ADDITIONAL_OPTIONS+=('--no-left-queries-check') ADDITIONAL_OPTIONS+=('--jobs') ADDITIONAL_OPTIONS+=('2') else diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a8cab282b0d..eb97118aecc 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -375,7 +375,6 @@ class FailureReason(enum.Enum): RESULT_DIFF = "result differs with reference: " TOO_LONG = "Test runs too long (> 60s). Make it faster." INTERNAL_QUERY_FAIL = "Internal query (CREATE/DROP DATABASE) failed:" - LEFT_QUERIES = "Queries left in background after the test finished:" # SKIPPED reasons DISABLED = "disabled" @@ -828,21 +827,6 @@ class TestCase: description, ) - left_queries_check = args.no_left_queries_check is False - if self.tags and "no-left-queries-check" in self.tags: - left_queries_check = False - if left_queries_check: - processlist = get_processlist_after_test(self.testcase_args) - if processlist: - description += f"\n{json.dumps(processlist, indent=4)}\n" - return TestResult( - self.name, - TestStatus.FAIL, - FailureReason.LEFT_QUERIES, - total_time, - description, - ) - if os.path.exists(self.stdout_file): os.remove(self.stdout_file) if os.path.exists(self.stderr_file): diff --git a/tests/queries/0_stateless/01085_max_distributed_connections_http.sh b/tests/queries/0_stateless/01085_max_distributed_connections_http.sh index 6e840f44930..0e40918257d 100755 --- a/tests/queries/0_stateless/01085_max_distributed_connections_http.sh +++ b/tests/queries/0_stateless/01085_max_distributed_connections_http.sh @@ -15,5 +15,3 @@ while [[ $i -lt $retries ]]; do timeout 1.8s ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&max_distributed_connections=2&max_threads=1" -d "$query" && break ((++i)) done - -clickhouse_test_wait_queries 60 diff --git a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh index 142b83f1e0b..b659d550fa4 100755 --- a/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_log_tinylog_deadlock_race.sh @@ -10,40 +10,46 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -function thread_create() -{ - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" - sleep 0.0$RANDOM +function thread_create { + while true; do + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" + sleep 0.0$RANDOM + done } -function thread_drop() -{ - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" - sleep 0.0$RANDOM +function thread_drop { + while true; do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" + sleep 0.0$RANDOM + done } -function thread_rename() -{ - $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM +function thread_rename { + while true; do + $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done } -function thread_select() -{ - $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM +function thread_select { + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done } -function thread_insert() -{ - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: '| grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM +function thread_insert { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: '| grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done } -function thread_insert_select() -{ - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM +function thread_insert_select { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done } export -f thread_create @@ -59,18 +65,18 @@ export -f thread_insert_select function test_with_engine { echo "Testing $1" - clickhouse_client_loop_timeout 10 thread_create t1 $1 & - clickhouse_client_loop_timeout 10 thread_create t2 $1 & - clickhouse_client_loop_timeout 10 thread_drop t1 & - clickhouse_client_loop_timeout 10 thread_drop t2 & - clickhouse_client_loop_timeout 10 thread_rename t1 t2 & - clickhouse_client_loop_timeout 10 thread_rename t2 t1 & - clickhouse_client_loop_timeout 10 thread_select t1 & - clickhouse_client_loop_timeout 10 thread_select t2 & - clickhouse_client_loop_timeout 10 thread_insert t1 5 & - clickhouse_client_loop_timeout 10 thread_insert t2 10 & - clickhouse_client_loop_timeout 10 thread_insert_select t1 t2 & - clickhouse_client_loop_timeout 10 thread_insert_select t2 t1 & + timeout 10 bash -c "thread_create t1 $1" & + timeout 10 bash -c "thread_create t2 $1" & + timeout 10 bash -c 'thread_drop t1' & + timeout 10 bash -c 'thread_drop t2' & + timeout 10 bash -c 'thread_rename t1 t2' & + timeout 10 bash -c 'thread_rename t2 t1' & + timeout 10 bash -c 'thread_select t1' & + timeout 10 bash -c 'thread_select t2' & + timeout 10 bash -c 'thread_insert t1 5' & + timeout 10 bash -c 'thread_insert t2 10' & + timeout 10 bash -c 'thread_insert_select t1 t2' & + timeout 10 bash -c 'thread_insert_select t2 t1' & wait echo "Done $1" diff --git a/tests/queries/0_stateless/01675_data_type_coroutine.sh b/tests/queries/0_stateless/01675_data_type_coroutine.sh index 687ff6ac473..4106d0d7f73 100755 --- a/tests/queries/0_stateless/01675_data_type_coroutine.sh +++ b/tests/queries/0_stateless/01675_data_type_coroutine.sh @@ -17,9 +17,3 @@ while [[ $counter -lt $retries ]]; do done echo 'Ok' - -# wait queries, since there is 'Maximum parse depth' error on the client -# and in this case it simply reset the connection and don't read everything -# from server, so there is no guarantee that the query is stopped when the -# client returns -clickhouse_test_wait_queries 60 diff --git a/tests/queries/0_stateless/01675_data_type_coroutine_2.sh b/tests/queries/0_stateless/01675_data_type_coroutine_2.sh index 9c4ed81e345..501b9d4ab12 100755 --- a/tests/queries/0_stateless/01675_data_type_coroutine_2.sh +++ b/tests/queries/0_stateless/01675_data_type_coroutine_2.sh @@ -17,9 +17,3 @@ done #echo "I = ${I}" echo 'Ok' - -# wait queries, since there is 'Maximum parse depth' error on the client -# and in this case it simply reset the connection and don't read everything -# from server, so there is no guarantee that the query is stopped when the -# client returns -clickhouse_test_wait_queries 60 diff --git a/tests/queries/0_stateless/01731_async_task_queue_wait.sh b/tests/queries/0_stateless/01731_async_task_queue_wait.sh index 6fdd676f336..e0babf3c6ff 100755 --- a/tests/queries/0_stateless/01731_async_task_queue_wait.sh +++ b/tests/queries/0_stateless/01731_async_task_queue_wait.sh @@ -8,5 +8,3 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # cancellation with async_socket_for_remote=1 (that ignores # max_distributed_connections) timeout --signal=SIGINT 1 ${CLICKHOUSE_CLIENT} --max_distributed_connections=1 --max_block_size=2 --interactive_delay=900000 -q "select number + sleep(0.3) as x from remote('127.{2,3}', system.numbers) settings max_block_size = 2" 2>&1 | grep "Empty task was returned from async task queue" || true - -clickhouse_test_wait_queries 60 diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index f2016dbc0c1..303c7cb44b6 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -5,19 +5,22 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u02104' -$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u02104 IDENTIFIED WITH no_password' -$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u02104' +$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u1 IDENTIFIED WITH no_password' +$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u1' function overcommited() { - $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator=1,memory_usage_overcommit_max_wait_microseconds=500' 2>&1 \ - | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "OVERCOMMITED WITH USER LIMIT IS KILLED" + while true; do + $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator=1,memory_usage_overcommit_max_wait_microseconds=500' 2>&1 \ + | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "OVERCOMMITED WITH USER LIMIT IS KILLED" + done } function expect_execution() { - $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,memory_overcommit_ratio_denominator=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null + while true; do + $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,memory_overcommit_ratio_denominator=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null + done } export -f overcommited @@ -27,9 +30,9 @@ function user_test() { for _ in {1..10}; do - clickhouse_client_loop_timeout 10 overcommited & - clickhouse_client_loop_timeout 10 expect_execution & - done + timeout 10 bash -c overcommited & + timeout 10 bash -c expect_execution & + done; wait } @@ -43,4 +46,4 @@ else echo "OVERCOMMITED WITH USER LIMIT WAS KILLED" fi -$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u02104' +$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u1' diff --git a/tests/queries/0_stateless/02151_http_s_structure_set_eof.sh b/tests/queries/0_stateless/02151_http_s_structure_set_eof.sh index c3dfc4d03a8..448fa9bfede 100755 --- a/tests/queries/0_stateless/02151_http_s_structure_set_eof.sh +++ b/tests/queries/0_stateless/02151_http_s_structure_set_eof.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh @@ -26,9 +25,3 @@ timeout 0.15s ${CLICKHOUSE_CURL} -sS -F "s=@$tmp_file;" "${CLICKHOUSE_URL}&s_str echo $? timeout 0.15s ${CLICKHOUSE_CURL} -sS -F "s=@$tmp_file;" "${CLICKHOUSE_URL}&s_structure=key+Int&query=SELECT+dummy+IN+s&input_format_parallel_parsing=false" -o /dev/null echo $? - -# wait until the query above will start, -# so that clickhouse_test_wait_queries will see them. -sleep 5 - -clickhouse_test_wait_queries 60 From 9af6e237e135a7e35b0e3f1ee43d35659621b4c1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 19:25:56 +0200 Subject: [PATCH 020/101] remove retries --- tests/clickhouse-test | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index eb97118aecc..355f7b7a712 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -55,7 +55,6 @@ MESSAGES_TO_RETRY = [ "DB::Exception: New table appeared in database being dropped or detached. Try again", "is already started to be removing by another replica right now", "DB::Exception: Cannot enqueue query", - "line 1: wait_for: No record of process", # Something weird from bash internals, let's just retry "is executing longer than distributed_ddl_task_timeout", # FIXME ] From 70f39e08493532ea75c503a326c15f51ab9b2b59 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 19:27:40 +0200 Subject: [PATCH 021/101] Revert "Merge pull request #36731 from azat/fix-async_inserts_stress_long-test" This reverts commit d1d2f2c1a4979d17b7d58f591f56346bc79278f8, reversing changes made to 1ddb04b99294a050bd9c0e124c17617dd2f89011. --- tests/queries/0_stateless/02015_async_inserts_stress_long.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh index 81d70f79574..086419baa61 100755 --- a/tests/queries/0_stateless/02015_async_inserts_stress_long.sh +++ b/tests/queries/0_stateless/02015_async_inserts_stress_long.sh @@ -83,6 +83,3 @@ wait echo "OK" ${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS async_inserts"; - -# There is no other way to wait HTTP queries -clickhouse_test_wait_queries 30 From f3c2ca6a1378d09afd8e33696b623c125fd64266 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 19:28:38 +0200 Subject: [PATCH 022/101] Revert "Merge pull request #36634 from azat/01502_long_log_tinylog_deadlock_race-test" This reverts commit c04c62795e024dc115aa724d48f443550391bfae, reversing changes made to 88f05ac14ab60ac5353c06964c8470de21a5f627. --- .../01502_long_log_tinylog_deadlock_race.sh | 78 ++++++++++--------- 1 file changed, 42 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh index a3dadf48c38..1087a7ed96b 100755 --- a/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh +++ b/tests/queries/0_stateless/01502_long_log_tinylog_deadlock_race.sh @@ -10,40 +10,46 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -function thread_create() -{ - $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM +function thread_create { + while true; do + $CLICKHOUSE_CLIENT --query "CREATE TABLE IF NOT EXISTS $1 (x UInt64, s Array(Nullable(String))) ENGINE = $2" 2>&1 | grep -v -F 'Received exception from server' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done } -function thread_drop() -{ - $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM +function thread_drop { + while true; do + $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS $1" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done } -function thread_rename() -{ - $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' - sleep 0.0$RANDOM +function thread_rename { + while true; do + $CLICKHOUSE_CLIENT --query "RENAME TABLE $1 TO $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|57)' + sleep 0.0$RANDOM + done } -function thread_select() -{ - $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM +function thread_select { + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM $1 FORMAT Null" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done } -function thread_insert() -{ - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM +function thread_insert { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT rand64(1), [toString(rand64(2))] FROM numbers($2)" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done } -function thread_insert_select() -{ - $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' - sleep 0.0$RANDOM +function thread_insert_select { + while true; do + $CLICKHOUSE_CLIENT --query "INSERT INTO $1 SELECT * FROM $2" 2>&1 | grep -v -e 'Received exception from server' -e '^(query: ' | grep -v -P 'Code: (60|218)' + sleep 0.0$RANDOM + done } export -f thread_create @@ -59,18 +65,18 @@ export -f thread_insert_select function test_with_engine { echo "Testing $1" - clickhouse_client_loop_timeout 10 thread_create t1 $1 & - clickhouse_client_loop_timeout 10 thread_create t2 $1 & - clickhouse_client_loop_timeout 10 thread_drop t1 & - clickhouse_client_loop_timeout 10 thread_drop t2 & - clickhouse_client_loop_timeout 10 thread_rename t1 t2 & - clickhouse_client_loop_timeout 10 thread_rename t2 t1 & - clickhouse_client_loop_timeout 10 thread_select t1 & - clickhouse_client_loop_timeout 10 thread_select t2 & - clickhouse_client_loop_timeout 10 thread_insert t1 5 & - clickhouse_client_loop_timeout 10 thread_insert t2 10 & - clickhouse_client_loop_timeout 10 thread_insert_select t1 t2 & - clickhouse_client_loop_timeout 10 thread_insert_select t2 t1 & + timeout 10 bash -c "thread_create t1 $1" & + timeout 10 bash -c "thread_create t2 $1" & + timeout 10 bash -c 'thread_drop t1' & + timeout 10 bash -c 'thread_drop t2' & + timeout 10 bash -c 'thread_rename t1 t2' & + timeout 10 bash -c 'thread_rename t2 t1' & + timeout 10 bash -c 'thread_select t1' & + timeout 10 bash -c 'thread_select t2' & + timeout 10 bash -c 'thread_insert t1 5' & + timeout 10 bash -c 'thread_insert t2 10' & + timeout 10 bash -c 'thread_insert_select t1 t2' & + timeout 10 bash -c 'thread_insert_select t2 t1' & wait echo "Done $1" From be498b0658a2905f4c3f7b7459c261ed19ac6b64 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 19:40:26 +0200 Subject: [PATCH 023/101] fix test --- .../0_stateless/01133_begin_commit_race.sh | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01133_begin_commit_race.sh b/tests/queries/0_stateless/01133_begin_commit_race.sh index f64570950c7..7dadb35ccff 100755 --- a/tests/queries/0_stateless/01133_begin_commit_race.sh +++ b/tests/queries/0_stateless/01133_begin_commit_race.sh @@ -13,32 +13,40 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE mt (n Int64) ENGINE=MergeTree ORDER BY function begin_commit_readonly() { + while true; do $CLICKHOUSE_CLIENT --multiquery --query " SET wait_changes_become_visible_after_commit_mode='wait'; BEGIN TRANSACTION; COMMIT;" 2>&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_STATUS_OF_TRANSACTION + done } function begin_rollback_readonly() { + while true; do $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=wait_unknown --multiquery --query " BEGIN TRANSACTION; SET TRANSACTION SNAPSHOT 42; ROLLBACK;" + done } function begin_insert_commit() { + while true; do $CLICKHOUSE_CLIENT --wait_changes_become_visible_after_commit_mode=async --multiquery --query " BEGIN TRANSACTION; INSERT INTO mt VALUES ($RANDOM); COMMIT;" 2>&1| grep -Fa "Exception: " | grep -Fv UNKNOWN_STATUS_OF_TRANSACTION + done } function introspection() { + while true; do $CLICKHOUSE_CLIENT -q "SELECT * FROM system.transactions FORMAT Null" $CLICKHOUSE_CLIENT -q "SELECT transactionLatestSnapshot(), transactionOldestSnapshot() FORMAT Null" + done } export -f begin_commit_readonly @@ -48,10 +56,10 @@ export -f introspection TIMEOUT=20 -clickhouse_client_loop_timeout $TIMEOUT begin_commit_readonly & -clickhouse_client_loop_timeout $TIMEOUT begin_rollback_readonly & -clickhouse_client_loop_timeout $TIMEOUT begin_insert_commit & -clickhouse_client_loop_timeout $TIMEOUT introspection & +timeout $TIMEOUT bash -c begin_commit_readonly & +timeout $TIMEOUT bash -c begin_rollback_readonly & +timeout $TIMEOUT bash -c begin_insert_commit & +timeout $TIMEOUT bash -c introspection & wait From 9e3e7953535e35d9e3293925abf3eeae6d95422c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 20:44:58 +0200 Subject: [PATCH 024/101] fix --- tests/queries/0_stateless/02104_overcommit_memory.sh | 6 +++--- tests/queries/0_stateless/02235_remote_fs_cache_stress.sh | 2 +- tests/queries/0_stateless/02294_overcommit_overflow.sh | 6 +++++- 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh index 303c7cb44b6..73d65eb40ed 100755 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ b/tests/queries/0_stateless/02104_overcommit_memory.sh @@ -5,8 +5,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u1 IDENTIFIED WITH no_password' -$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u1' +$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u02104 IDENTIFIED WITH no_password' +$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u02104' function overcommited() { @@ -46,4 +46,4 @@ else echo "OVERCOMMITED WITH USER LIMIT WAS KILLED" fi -$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u1' +$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u02104' diff --git a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh index a5c0ee6ecff..bc1a4cbfdd1 100755 --- a/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh +++ b/tests/queries/0_stateless/02235_remote_fs_cache_stress.sh @@ -63,7 +63,7 @@ SELECT indexOf(['a', 'b', NULL], toLowCardinality('a')); """ } -for i in `seq 1 32`; do go | grep -q "Exception" && echo 'FAIL' || echo 'OK' ||: & done +for _ in `seq 1 32`; do go | grep -q "Exception" && echo 'FAIL' || echo 'OK' ||: & done wait diff --git a/tests/queries/0_stateless/02294_overcommit_overflow.sh b/tests/queries/0_stateless/02294_overcommit_overflow.sh index 64a4dddc67f..0fe7882cb3d 100755 --- a/tests/queries/0_stateless/02294_overcommit_overflow.sh +++ b/tests/queries/0_stateless/02294_overcommit_overflow.sh @@ -11,14 +11,18 @@ $CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u02294' function query() { + while true; do $CLICKHOUSE_CLIENT -u u02294 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,memory_overcommit_ratio_denominator=2000000000000000000,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null + done } export -f query +TIMEOUT=10 + for _ in {1..10}; do - clickhouse_client_loop_timeout 10 query & + timeout $TIMEOUT bash -c query & done wait From 313f4833d641993a4d090691c0f8d1921ded97a3 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 28 May 2022 23:21:55 +0200 Subject: [PATCH 025/101] fix --- tests/queries/0_stateless/00417_kill_query.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/00417_kill_query.sh b/tests/queries/0_stateless/00417_kill_query.sh index e40eb5c3f80..c3b57b8ef3f 100755 --- a/tests/queries/0_stateless/00417_kill_query.sh +++ b/tests/queries/0_stateless/00417_kill_query.sh @@ -22,4 +22,3 @@ $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 0 SYNC FORMAT TabSeparated" $CLICKHOUSE_CLIENT -q "KILL QUERY WHERE 1 TEST" &>/dev/null -clickhouse_test_wait_queries 60 From 6525bfc4cd80aff1d1672d02d933898625c0556d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 30 May 2022 16:57:51 +0800 Subject: [PATCH 026/101] Avoid context copy for InterpreterSelects --- .../IInterpreterUnionOrSelectQuery.h | 8 +++-- .../InterpreterSelectIntersectExceptQuery.cpp | 8 +++++ .../InterpreterSelectIntersectExceptQuery.h | 5 ++++ src/Interpreters/InterpreterSelectQuery.cpp | 30 +++++++++++++++++++ src/Interpreters/InterpreterSelectQuery.h | 17 +++++++++++ .../InterpreterSelectWithUnionQuery.cpp | 9 ++++-- .../InterpreterSelectWithUnionQuery.h | 6 ++++ 7 files changed, 79 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index a9262fbfa1e..66861fd2ae0 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -12,8 +12,13 @@ class IInterpreterUnionOrSelectQuery : public IInterpreter { public: IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_) + : IInterpreterUnionOrSelectQuery(query_ptr_, Context::createCopy(context_), options_) + { + } + + IInterpreterUnionOrSelectQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_, const SelectQueryOptions & options_) : query_ptr(query_ptr_) - , context(Context::createCopy(context_)) + , context(context_) , options(options_) , max_streams(context->getSettingsRef().max_threads) { @@ -60,4 +65,3 @@ protected: bool uses_view_source = false; }; } - diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index cad570ab420..d6172766fb6 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -51,6 +51,14 @@ InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery( const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_) + :InterpreterSelectIntersectExceptQuery(query_ptr_, Context::createCopy(context_), options_) +{ +} + +InterpreterSelectIntersectExceptQuery::InterpreterSelectIntersectExceptQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + const SelectQueryOptions & options_) : IInterpreterUnionOrSelectQuery(query_ptr_->clone(), context_, options_) { ASTSelectIntersectExceptQuery * ast = query_ptr->as(); diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h index 805565e4c51..79ee32e514a 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -24,6 +24,11 @@ public: ContextPtr context_, const SelectQueryOptions & options_); + InterpreterSelectIntersectExceptQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + const SelectQueryOptions & options_); + BlockIO execute() override; Block getSampleBlock() { return result_header; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d143295181e..6f19b26f560 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -165,6 +165,14 @@ InterpreterSelectQuery::InterpreterSelectQuery( : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_) {} +InterpreterSelectQuery::InterpreterSelectQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + const SelectQueryOptions & options_, + const Names & required_result_column_names_) + : InterpreterSelectQuery(query_ptr_, context_, std::nullopt, nullptr, options_, required_result_column_names_) +{} + InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, ContextPtr context_, @@ -280,6 +288,28 @@ InterpreterSelectQuery::InterpreterSelectQuery( const StorageMetadataPtr & metadata_snapshot_, SubqueriesForSets subquery_for_sets_, PreparedSets prepared_sets_) + : InterpreterSelectQuery( + query_ptr_, + Context::createCopy(context_), + std::move(input_pipe_), + storage_, + options_, + required_result_column_names, + metadata_snapshot_, + std::move(subquery_for_sets_), + std::move(prepared_sets_)) +{} + +InterpreterSelectQuery::InterpreterSelectQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + std::optional input_pipe_, + const StoragePtr & storage_, + const SelectQueryOptions & options_, + const Names & required_result_column_names, + const StorageMetadataPtr & metadata_snapshot_, + SubqueriesForSets subquery_for_sets_, + PreparedSets prepared_sets_) /// NOTE: the query almost always should be cloned because it will be modified during analysis. : IInterpreterUnionOrSelectQuery(options_.modify_inplace ? query_ptr_ : query_ptr_->clone(), context_, options_) , storage(storage_) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 3adbcad909c..8aee3c7c273 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -55,6 +55,12 @@ public: const SelectQueryOptions &, const Names & required_result_column_names_ = Names{}); + InterpreterSelectQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + const SelectQueryOptions &, + const Names & required_result_column_names_ = Names{}); + /// Read data not from the table specified in the query, but from the prepared pipe `input`. InterpreterSelectQuery( const ASTPtr & query_ptr_, @@ -133,6 +139,17 @@ private: SubqueriesForSets subquery_for_sets_ = {}, PreparedSets prepared_sets_ = {}); + InterpreterSelectQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + std::optional input_pipe, + const StoragePtr & storage_, + const SelectQueryOptions &, + const Names & required_result_column_names = {}, + const StorageMetadataPtr & metadata_snapshot_ = nullptr, + SubqueriesForSets subquery_for_sets_ = {}, + PreparedSets prepared_sets_ = {}); + ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } void addPrewhereAliasActions(); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 7506c3013cb..d7d2ab29b4b 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -31,8 +31,13 @@ namespace ErrorCodes } InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( - const ASTPtr & query_ptr_, ContextPtr context_, - const SelectQueryOptions & options_, const Names & required_result_column_names) + const ASTPtr & query_ptr_, ContextPtr context_, const SelectQueryOptions & options_, const Names & required_result_column_names) + : InterpreterSelectWithUnionQuery(query_ptr_, Context::createCopy(context_), options_, required_result_column_names) +{ +} + +InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( + const ASTPtr & query_ptr_, ContextMutablePtr context_, const SelectQueryOptions & options_, const Names & required_result_column_names) : IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_) { ASTSelectWithUnionQuery * ast = query_ptr->as(); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index adf8540d626..0e59e1e01ad 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -22,6 +22,12 @@ public: const SelectQueryOptions &, const Names & required_result_column_names = {}); + InterpreterSelectWithUnionQuery( + const ASTPtr & query_ptr_, + ContextMutablePtr context_, + const SelectQueryOptions &, + const Names & required_result_column_names = {}); + ~InterpreterSelectWithUnionQuery() override; /// Builds QueryPlan for current query. From b68e8efaf074842e7290ab9dd0fb48635b7e9147 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 30 May 2022 21:01:27 +0800 Subject: [PATCH 027/101] Fix joinGet with cannot be null type --- src/Interpreters/HashJoin.cpp | 2 +- .../queries/0_stateless/02307_join_get_array_null.reference | 1 + tests/queries/0_stateless/02307_join_get_array_null.sql | 6 ++++++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02307_join_get_array_null.reference create mode 100644 tests/queries/0_stateless/02307_join_get_array_null.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b58059312bd..b5c227fd481 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1698,7 +1698,7 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, throw Exception("StorageJoin doesn't contain column " + column_name, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); auto elem = sample_block_with_columns_to_add.getByName(column_name); - if (or_null) + if (or_null && JoinCommon::canBecomeNullable(elem.type)) elem.type = makeNullable(elem.type); return elem.type; } diff --git a/tests/queries/0_stateless/02307_join_get_array_null.reference b/tests/queries/0_stateless/02307_join_get_array_null.reference new file mode 100644 index 00000000000..fe51488c706 --- /dev/null +++ b/tests/queries/0_stateless/02307_join_get_array_null.reference @@ -0,0 +1 @@ +[] diff --git a/tests/queries/0_stateless/02307_join_get_array_null.sql b/tests/queries/0_stateless/02307_join_get_array_null.sql new file mode 100644 index 00000000000..b1bbc076c17 --- /dev/null +++ b/tests/queries/0_stateless/02307_join_get_array_null.sql @@ -0,0 +1,6 @@ +drop table if exists id_val; + +create table id_val(id Int32, val Array(Int32)) engine Join(ANY, LEFT, id) settings join_use_nulls = 1; +select joinGet(id_val, 'val', toInt32(number)) from numbers(1); + +drop table id_val; From 217f492264f5526a2372f95c9144be162934a34c Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 30 May 2022 21:38:07 +0800 Subject: [PATCH 028/101] Fix test --- tests/queries/0_stateless/01240_join_get_or_null.reference | 1 + tests/queries/0_stateless/01240_join_get_or_null.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01240_join_get_or_null.reference b/tests/queries/0_stateless/01240_join_get_or_null.reference index 322d026bea8..4bd89aa3755 100644 --- a/tests/queries/0_stateless/01240_join_get_or_null.reference +++ b/tests/queries/0_stateless/01240_join_get_or_null.reference @@ -1,5 +1,6 @@ \N \N +[] 1396-01-12 1396-01-13 \N diff --git a/tests/queries/0_stateless/01240_join_get_or_null.sql b/tests/queries/0_stateless/01240_join_get_or_null.sql index b230cc7bb8d..eb81860bffd 100644 --- a/tests/queries/0_stateless/01240_join_get_or_null.sql +++ b/tests/queries/0_stateless/01240_join_get_or_null.sql @@ -9,7 +9,7 @@ SELECT joinGetOrNull('join_test', 'num', 500); DROP TABLE join_test; CREATE TABLE join_test (id UInt16, num Array(UInt16)) engine = Join(ANY, LEFT, id); -SELECT joinGetOrNull('join_test', 'num', 500); -- { serverError 43 } +SELECT joinGetOrNull('join_test', 'num', 500); DROP TABLE join_test; drop table if exists test; From 8a3f4bda62d76136057c8ae9029504b26ffd8826 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 May 2022 13:50:22 +0000 Subject: [PATCH 029/101] Fix columns number mismatch in cross join --- src/Interpreters/HashJoin.cpp | 9 +++++++-- src/Interpreters/TreeRewriter.cpp | 2 +- .../02313_cross_join_dup_col_names.reference | 2 ++ .../02313_cross_join_dup_col_names.sql | 16 ++++++++++++++++ 4 files changed, 26 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02313_cross_join_dup_col_names.reference create mode 100644 tests/queries/0_stateless/02313_cross_join_dup_col_names.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index b58059312bd..521fbe139e7 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -707,6 +707,13 @@ namespace void HashJoin::initRightBlockStructure(Block & saved_block_sample) { + if (isCrossOrComma(kind)) + { + /// cross join doesn't have keys, just add all columns + saved_block_sample = sample_block_with_columns_to_add.cloneEmpty(); + return; + } + bool multiple_disjuncts = !table_join->oneDisjunct(); /// We could remove key columns for LEFT | INNER HashJoin but we should keep them for JoinSwitcher (if any). bool save_key_columns = !table_join->forceHashJoin() || isRightOrFull(kind) || multiple_disjuncts; @@ -724,9 +731,7 @@ void HashJoin::initRightBlockStructure(Block & saved_block_sample) for (auto & column : sample_block_with_columns_to_add) { if (!saved_block_sample.findByName(column.name)) - { saved_block_sample.insert(column); - } } } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index c90421d6f4f..9865928f4eb 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1151,7 +1151,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( if (remove_duplicates) renameDuplicatedColumns(select_query); - /// Perform it before analyzing JOINs, because it may change number of columns with names unique and break some login inside JOINs + /// Perform it before analyzing JOINs, because it may change number of columns with names unique and break some logic inside JOINs if (settings.optimize_normalize_count_variants) TreeOptimizer::optimizeCountConstantAndSumOne(query); diff --git a/tests/queries/0_stateless/02313_cross_join_dup_col_names.reference b/tests/queries/0_stateless/02313_cross_join_dup_col_names.reference new file mode 100644 index 00000000000..96e34d5a44c --- /dev/null +++ b/tests/queries/0_stateless/02313_cross_join_dup_col_names.reference @@ -0,0 +1,2 @@ +\N +\N diff --git a/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql b/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql new file mode 100644 index 00000000000..44a4797ae3c --- /dev/null +++ b/tests/queries/0_stateless/02313_cross_join_dup_col_names.sql @@ -0,0 +1,16 @@ +-- Tags: no-backward-compatibility-check + +-- https://github.com/ClickHouse/ClickHouse/issues/37561 + +SELECT NULL +FROM + (SELECT NULL) AS s1, + (SELECT count(2), count(1)) AS s2 +; + +SELECT NULL +FROM + (SELECT NULL) AS s1, + (SELECT count(2.), 9223372036854775806, count('-1'), NULL) AS s2, + (SELECT count('-2147483648')) AS any_query, (SELECT NULL) AS check_single_query +; From e2dd6f62495ee4056d9289028a93fada892cc147 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 30 May 2022 19:58:23 +0200 Subject: [PATCH 030/101] Removed prewhere_info.alias_actions --- src/Interpreters/ExpressionAnalyzer.cpp | 1 - src/Interpreters/InterpreterSelectQuery.cpp | 22 ------------------- src/Storages/IStorage.cpp | 5 ----- .../MergeTreeBaseSelectProcessor.cpp | 5 ----- .../MergeTree/MergeTreeBlockReadUtils.cpp | 19 ++++++---------- src/Storages/MergeTree/MergeTreeData.cpp | 13 ----------- .../MergeTree/MergeTreeRangeReader.cpp | 6 ----- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 -- src/Storages/SelectQueryInfo.h | 2 -- src/Storages/StorageBuffer.cpp | 9 -------- 10 files changed, 7 insertions(+), 77 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 0b1154f6fd1..9ac0fe46553 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1970,7 +1970,6 @@ void ExpressionAnalysisResult::checkActions() const }; check_actions(prewhere_info->prewhere_actions); - check_actions(prewhere_info->alias_actions); } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d143295181e..ebdd9612895 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1625,15 +1625,6 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan( { auto & prewhere_info = *prewhere_info_ptr; - if (prewhere_info.alias_actions) - { - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, - std::make_shared(prewhere_info.alias_actions)); - }); - } - if (prewhere_info.row_level_filter) { pipe.addSimpleTransform([&](const Block & header) @@ -1873,19 +1864,6 @@ void InterpreterSelectQuery::addPrewhereAliasActions() for (const auto & name : required_columns) prewhere_info->prewhere_actions->tryRestoreColumn(name); - auto analyzed_result - = TreeRewriter(context).analyze(required_columns_from_prewhere_expr, metadata_snapshot->getColumns().getAllPhysical()); - prewhere_info->alias_actions - = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, context).getActionsDAG(true, false); - - /// Add (physical?) columns required by alias actions. - auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns(); - Block prewhere_actions_result = prewhere_info->prewhere_actions->getResultColumns(); - for (auto & column : required_columns_from_alias) - if (!prewhere_actions_result.has(column.name)) - if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) - required_columns.push_back(column.name); - /// Add physical columns required by prewhere actions. for (const auto & column : required_columns_from_prewhere) if (!required_aliases_from_prewhere.contains(column)) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index f236cb5e98c..88d60e00b9c 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -231,11 +231,6 @@ std::string PrewhereInfo::dump() const WriteBufferFromOwnString ss; ss << "PrewhereDagInfo\n"; - if (alias_actions) - { - ss << "alias_actions " << alias_actions->dumpDAG() << "\n"; - } - if (prewhere_actions) { ss << "prewhere_actions " << prewhere_actions->dumpDAG() << "\n"; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index b8a3f0b1d1f..b9158bde6f1 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -72,8 +72,6 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( if (prewhere_info) { prewhere_actions = std::make_unique(); - if (prewhere_info->alias_actions) - prewhere_actions->alias_actions = std::make_shared(prewhere_info->alias_actions, actions_settings); if (prewhere_info->row_level_filter) prewhere_actions->row_level_filter = std::make_shared(prewhere_info->row_level_filter, actions_settings); @@ -556,9 +554,6 @@ Block MergeTreeBaseSelectProcessor::transformHeader( { if (prewhere_info) { - if (prewhere_info->alias_actions) - block = prewhere_info->alias_actions->updateHeader(std::move(block)); - if (prewhere_info->row_level_filter) { block = prewhere_info->row_level_filter->updateHeader(std::move(block)); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 5cc22503348..f74823eaec2 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -281,21 +281,16 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { - if (prewhere_info->alias_actions) - pre_column_names = prewhere_info->alias_actions->getRequiredColumnsNames(); - else + pre_column_names = prewhere_info->prewhere_actions->getRequiredColumnsNames(); + + if (prewhere_info->row_level_filter) { - pre_column_names = prewhere_info->prewhere_actions->getRequiredColumnsNames(); + NameSet names(pre_column_names.begin(), pre_column_names.end()); - if (prewhere_info->row_level_filter) + for (auto & name : prewhere_info->row_level_filter->getRequiredColumnsNames()) { - NameSet names(pre_column_names.begin(), pre_column_names.end()); - - for (auto & name : prewhere_info->row_level_filter->getRequiredColumnsNames()) - { - if (!names.contains(name)) - pre_column_names.push_back(name); - } + if (!names.contains(name)) + pre_column_names.push_back(name); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 7726a752cbe..a38412d5c8f 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5439,17 +5439,6 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg candidate.prewhere_info->row_level_filter = row_level_filter_actions; } - if (candidate.prewhere_info->alias_actions) - { - auto alias_actions = candidate.prewhere_info->alias_actions->clone(); - // alias_action should not add missing keys. - auto new_prewhere_required_columns - = alias_actions->foldActionsByProjection(prewhere_required_columns, projection.sample_block_for_keys, {}, false); - if (new_prewhere_required_columns.empty() && !prewhere_required_columns.empty()) - return false; - prewhere_required_columns = std::move(new_prewhere_required_columns); - candidate.prewhere_info->alias_actions = alias_actions; - } required_columns.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); } @@ -5619,8 +5608,6 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg if (minmax_count_projection_candidate->prewhere_info) { const auto & prewhere_info = minmax_count_projection_candidate->prewhere_info; - if (prewhere_info->alias_actions) - ExpressionActions(prewhere_info->alias_actions, actions_settings).execute(query_info.minmax_count_projection_block); if (prewhere_info->row_level_filter) { diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 7f9d1414886..addd9bdf7e0 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -564,9 +564,6 @@ MergeTreeRangeReader::MergeTreeRangeReader( if (prewhere_info) { - if (prewhere_info->alias_actions) - prewhere_info->alias_actions->execute(sample_block, true); - if (prewhere_info->row_level_filter) { prewhere_info->row_level_filter->execute(sample_block, true); @@ -1029,9 +1026,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r ++pos; } - if (prewhere_info->alias_actions) - prewhere_info->alias_actions->execute(block); - /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index ed5cc16add8..21ed35e6a78 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -21,8 +21,6 @@ using ExpressionActionsPtr = std::shared_ptr; /// The same as PrewhereInfo, but with ExpressionActions instead of ActionsDAG struct PrewhereExprInfo { - /// Actions which are executed in order to alias columns are used for prewhere actions. - ExpressionActionsPtr alias_actions; /// Actions for row level security filter. Applied separately before prewhere_actions. /// This actions are separate because prewhere condition should not be executed over filtered rows. ExpressionActionsPtr row_level_filter; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index b6643754db7..80194c5573f 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -48,8 +48,6 @@ using SubqueriesForSets = std::unordered_map; struct PrewhereInfo { - /// Actions which are executed in order to alias columns are used for prewhere actions. - ActionsDAGPtr alias_actions; /// Actions for row level security filter. Applied separately before prewhere_actions. /// This actions are separate because prewhere condition should not be executed over filtered rows. ActionsDAGPtr row_level_filter; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index e551abe0065..f54775a8706 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -383,15 +383,6 @@ void StorageBuffer::read( if (query_info.prewhere_info) { auto actions_settings = ExpressionActionsSettings::fromContext(local_context); - if (query_info.prewhere_info->alias_actions) - { - pipe_from_buffers.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - std::make_shared(query_info.prewhere_info->alias_actions, actions_settings)); - }); - } if (query_info.prewhere_info->row_level_filter) { From ad12adc31c5a5f440f52fdd72f5ae41880980920 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 27 May 2022 12:40:53 +0200 Subject: [PATCH 031/101] Measure and rework internal re2 caching This commit is based on local benchmarks of ClickHouse's re2 caching. Question 1: ----------------------------------------------------------- Is pattern caching useful for queries with const LIKE/REGEX patterns? E.g. SELECT LIKE(col_haystack, '%HelloWorld') FROM T; The short answer is: no. Runtime is (unsurprisingly) dominated by pattern evaluation + other stuff going on in queries, but definitely not pattern compilation. For space reasons, I omit details of the local experiments. (Side note: the current caching scheme is unbounded in size which poses a DoS risk (think of multi-tenancy). This risk is more pronounced when unbounded caching is used with non-const patterns ..., see next question) Question 2: ----------------------------------------------------------- Is pattern caching useful for queries with non-const LIKE/REGEX patterns? E.g. SELECT LIKE(col_haystack, col_needle) FROM T; I benchmarked five caching strategies: 1. no caching as a baseline (= recompile for each row) 2. unbounded cache (= threadsafe global hash-map) 3. LRU cache (= threadsafe global hash-map + LRU queue) 4. lightweight local cache 1 (= not threadsafe local hashmap with collision list which grows to a certain size (here: 10 elements) and afterwards never changes) 5. lightweight local cache 2 (not threadsafe local hashmap without collision list in which a collision replaces the stored element, idea by Alexey) ... using a haystack of 2 mio strings and A). 2 mio distinct simple patterns B). 10 simple patterns C) 2 mio distinct complex patterns D) 10 complex patterns Fo A) and C), caching does not help but these queries still allow to judge the static overhead of caching on query runtimes. B) and D) are extreme but common cases in practice. They include queries like "SELECT ... WHERE LIKE (col_haystack, flag ? '%pattern1%' : '%pattern2%'). Caching should help significantly. Because LIKE patterns are internally translated to re2 expressions, I show only measurements for MATCH queries. Results in sec, averaged over on multiple measurements; 1.A): 2.12 B): 1.68 C): 9.75 D): 9.45 2.A): 2.17 B): 1.73 C): 9.78 D): 9.47 3.A): 9.8 B): 0.63 C): 31.8 D): 0.98 4.A): 2.14 B): 0.29 C): 9.82 D): 0.41 5.A) 2.12 / 2.15 / 2.26 B) 1.51 / 0.43 / 0.30 C) 9.97 / 9.88 / 10.13 D) 5.70 / 0.42 / 0.43 (10/100/1000 buckets, resp. 10/1/0.1% collision rate) Evaluation: 1. This is the baseline. It was surprised that complex patterns (C, D) slow down the queries so badly compared to simple patterns (A, B). The runtime includes evaluation costs, but as caching only helps with compilation, and looking at 4.D and 5.D, compilation makes up over 90% of the runtime! 2. No speedup compared to 1, probably due to locking overhead. The cache is unbounded, and in experiments with data sets > 2 mio rows, 2. is the only scheme to throw OOM exceptions which is not acceptable. 3. Unique patterns (A and C) lead to thrashing of the LRU cache and very bad runtimes due to LRU queue maintenance and locking. Works pretty well however with few distinct patterns (B and D). 4. This scheme is tailored to queries B and D where it performs pretty good. More importantly, the caching is lightweight enough to not deteriorate performance on datasets A and C. 5. After some tuning of the hash map size, 100 buckets seem optimal to be in the same ballpark with 10 distinct patterns as 4. Performance also does not deteriorate on A and C compared to the baseline. Unlike 4., this scheme behaves LRU-like and can adjust to changing pattern distributions. As a conclusion, this commit implementes two things: 1. Based on Q1, pattern search with const needle no longer uses caching. This applies to LIKE and MATCH + a few (exotic) other SQL functions. The code for the unbounded caching was removed. 2. Based on Q2, pattern search with non-const needles now use method 5. --- src/Functions/FunctionsStringArray.h | 4 +- src/Functions/MatchImpl.h | 30 +++++--- src/Functions/Regexps.h | 73 ++++++++++++++----- src/Functions/countMatches.h | 6 +- src/Functions/extract.cpp | 6 +- src/Functions/extractAllGroups.h | 4 +- src/Functions/extractGroups.cpp | 4 +- .../like_and_match_pattern_caching.xml | 62 ++++++++++++++++ 8 files changed, 145 insertions(+), 44 deletions(-) create mode 100644 tests/performance/like_and_match_pattern_caching.xml diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index 6545c3e3549..b0f415be58a 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -477,7 +477,7 @@ public: ErrorCodes::ILLEGAL_COLUMN); if (!col->getValue().empty()) - re = Regexps::get(col->getValue()); + re = std::make_shared(Regexps::createRegexp(col->getValue())); } @@ -560,7 +560,7 @@ public: + " of first argument of function " + getName() + ". Must be constant string.", ErrorCodes::ILLEGAL_COLUMN); - re = Regexps::get(col->getValue()); + re = std::make_shared(Regexps::createRegexp(col->getValue())); capture = re->getNumberOfSubpatterns() > 0 ? 1 : 0; matches.resize(capture + 1); diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 9779eb8d608..96ce0ca2eb0 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -166,17 +166,17 @@ struct MatchImpl } else { - auto regexp = Regexps::get(needle); + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); String required_substring; bool is_trivial; bool required_substring_is_prefix; /// for `anchored` execution of the regexp. - regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); if (required_substring.empty()) { - if (!regexp->getRE2()) /// An empty regexp. Always matches. + if (!regexp.getRE2()) /// An empty regexp. Always matches. { if (haystack_size) memset(res.data(), !negate, haystack_size * sizeof(res[0])); @@ -186,7 +186,7 @@ struct MatchImpl size_t prev_offset = 0; for (size_t i = 0; i < haystack_size; ++i) { - const bool match = regexp->getRE2()->Match( + const bool match = regexp.getRE2()->Match( {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, 0, haystack_offsets[i] - prev_offset - 1, @@ -241,7 +241,7 @@ struct MatchImpl const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; const size_t end_pos = str_size; - const bool match = regexp->getRE2()->Match( + const bool match = regexp.getRE2()->Match( {str_data, str_size}, start_pos, end_pos, @@ -325,17 +325,17 @@ struct MatchImpl } else { - auto regexp = Regexps::get(needle); + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); String required_substring; bool is_trivial; bool required_substring_is_prefix; /// for `anchored` execution of the regexp. - regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); if (required_substring.empty()) { - if (!regexp->getRE2()) /// An empty regexp. Always matches. + if (!regexp.getRE2()) /// An empty regexp. Always matches. { if (haystack_size) memset(res.data(), !negate, haystack_size * sizeof(res[0])); @@ -345,7 +345,7 @@ struct MatchImpl size_t offset = 0; for (size_t i = 0; i < haystack_size; ++i) { - const bool match = regexp->getRE2()->Match( + const bool match = regexp.getRE2()->Match( {reinterpret_cast(&haystack[offset]), N}, 0, N, @@ -403,7 +403,7 @@ struct MatchImpl const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; const size_t end_pos = N; - const bool match = regexp->getRE2()->Match( + const bool match = regexp.getRE2()->Match( {str_data, N}, start_pos, end_pos, @@ -454,6 +454,9 @@ struct MatchImpl size_t prev_haystack_offset = 0; size_t prev_needle_offset = 0; + Regexps::LocalCacheTable cache; + Regexps::RegexpPtr regexp; + for (size_t i = 0; i < haystack_size; ++i) { const auto * const cur_haystack_data = &haystack_data[prev_haystack_offset]; @@ -479,7 +482,7 @@ struct MatchImpl } else { - auto regexp = Regexps::get(needle); + cache.getOrSet(needle, regexp); regexp->getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); @@ -565,6 +568,9 @@ struct MatchImpl size_t prev_haystack_offset = 0; size_t prev_needle_offset = 0; + Regexps::LocalCacheTable cache; + Regexps::RegexpPtr regexp; + for (size_t i = 0; i < haystack_size; ++i) { const auto * const cur_haystack_data = &haystack[prev_haystack_offset]; @@ -590,7 +596,7 @@ struct MatchImpl } else { - auto regexp = Regexps::get(needle); + cache.getOrSet(needle, regexp); regexp->getAnalyzeResult(required_substr, is_trivial, required_substring_is_prefix); diff --git a/src/Functions/Regexps.h b/src/Functions/Regexps.h index be3ce6cdeee..30afccbbac2 100644 --- a/src/Functions/Regexps.h +++ b/src/Functions/Regexps.h @@ -9,7 +9,6 @@ #include #include #include -#include #include #include #include @@ -39,38 +38,72 @@ namespace ErrorCodes namespace Regexps { using Regexp = OptimizedRegularExpressionSingleThreaded; -using Cache = LRUCache; -using RegexpPtr = Cache::MappedPtr; +using RegexpPtr = std::shared_ptr; -template -inline int buildRe2Flags() +template +inline Regexp createRegexp(const std::string & pattern) { int flags = OptimizedRegularExpression::RE_DOT_NL; if constexpr (no_capture) flags |= OptimizedRegularExpression::RE_NO_CAPTURE; if constexpr (case_insensitive) flags |= OptimizedRegularExpression::RE_CASELESS; - return flags; + + if constexpr (like) + return {likePatternToRegexp(pattern), flags}; + else + return {pattern, flags}; } -/// Probes the cache of known compiled regexps for the given string pattern and returns a compiled regexp if -/// found. Otherwise, a new cache entry is created. -template -inline RegexpPtr get(const String & pattern) +/// Caches compiled re2 objects for given string patterns. Intended to support the common situation of a small set of patterns which are +/// evaluated over and over within the same query. In these situations, usage of the cache will save unnecessary pattern re-compilation. +/// However, we must be careful that caching does not add too much static overhead to overall pattern evaluation. Therefore, the cache is +/// intentionally very lightweight: a) no thread-safety/mutexes, b) small & fixed capacity, c) no collision list, d) but also no open +/// addressing, instead collisions simply replace the existing element. +class LocalCacheTable { - static Cache known_regexps(42'000); +public: + using RegexpPtr = std::shared_ptr; - auto [regexp_ptr, _] = known_regexps.getOrSet(pattern, [&pattern]() + LocalCacheTable() + : known_regexps(max_regexp_cache_size, {"", nullptr}) { - const int flags = buildRe2Flags(); - ProfileEvents::increment(ProfileEvents::RegexpCreated); - if constexpr (like) - return std::make_shared(likePatternToRegexp(pattern), flags); + } + + template + void getOrSet(const String & pattern, RegexpPtr & regexp) + { + StringAndRegexp & bucket = known_regexps[hasher(pattern) % max_regexp_cache_size]; + + if (likely(bucket.regexp != nullptr)) + { + if (pattern == bucket.pattern) + regexp = bucket.regexp; + else + { + regexp = std::make_shared(createRegexp(pattern)); + bucket = {pattern, regexp}; + } + } else - return std::make_shared(pattern, flags); - }); - return regexp_ptr; -} + { + regexp = std::make_shared(createRegexp(pattern)); + bucket = {pattern, regexp}; + } + } + +private: + std::hash hasher; + struct StringAndRegexp + { + std::string pattern; + RegexpPtr regexp; + }; + using CacheTable = std::vector; + CacheTable known_regexps; + + constexpr static size_t max_regexp_cache_size = 100; // collision probability +}; } diff --git a/src/Functions/countMatches.h b/src/Functions/countMatches.h index 1d43b66d867..397515c8bba 100644 --- a/src/Functions/countMatches.h +++ b/src/Functions/countMatches.h @@ -55,7 +55,7 @@ public: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { const ColumnConst * column_pattern = checkAndGetColumnConstStringOrFixedString(arguments[1].column.get()); - Regexps::RegexpPtr re = Regexps::get(column_pattern->getValue()); + const Regexps::Regexp re = Regexps::createRegexp(column_pattern->getValue()); OptimizedRegularExpression::MatchVec matches; const IColumn * column_haystack = arguments[0].column.get(); @@ -95,7 +95,7 @@ public: throw Exception(ErrorCodes::LOGICAL_ERROR, "Error in FunctionCountMatches::getReturnTypeImpl()"); } - static uint64_t countMatches(StringRef src, Regexps::RegexpPtr & re, OptimizedRegularExpression::MatchVec & matches) + static uint64_t countMatches(StringRef src, const Regexps::Regexp & re, OptimizedRegularExpression::MatchVec & matches) { /// Only one match is required, no need to copy more. static const unsigned matches_limit = 1; @@ -108,7 +108,7 @@ public: { if (pos >= end) break; - if (!re->match(pos, end - pos, matches, matches_limit)) + if (!re.match(pos, end - pos, matches, matches_limit)) break; /// Progress should be made, but with empty match the progress will not be done. /// Also note that simply check is pattern empty is not enough, diff --git a/src/Functions/extract.cpp b/src/Functions/extract.cpp index 5b138d19747..f5917015e27 100644 --- a/src/Functions/extract.cpp +++ b/src/Functions/extract.cpp @@ -21,9 +21,9 @@ struct ExtractImpl res_data.reserve(data.size() / 5); res_offsets.resize(offsets.size()); - const auto & regexp = Regexps::get(pattern); + const Regexps::Regexp regexp = Regexps::createRegexp(pattern); - unsigned capture = regexp->getNumberOfSubpatterns() > 0 ? 1 : 0; + unsigned capture = regexp.getNumberOfSubpatterns() > 0 ? 1 : 0; OptimizedRegularExpression::MatchVec matches; matches.reserve(capture + 1); size_t prev_offset = 0; @@ -34,7 +34,7 @@ struct ExtractImpl size_t cur_offset = offsets[i]; unsigned count - = regexp->match(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1, matches, capture + 1); + = regexp.match(reinterpret_cast(&data[prev_offset]), cur_offset - prev_offset - 1, matches, capture + 1); if (count > capture && matches[capture].offset != std::string::npos) { const auto & match = matches[capture]; diff --git a/src/Functions/extractAllGroups.h b/src/Functions/extractAllGroups.h index e6d31e00616..1a40afbbe8e 100644 --- a/src/Functions/extractAllGroups.h +++ b/src/Functions/extractAllGroups.h @@ -95,8 +95,8 @@ public: throw Exception("Length of 'needle' argument must be greater than 0.", ErrorCodes::BAD_ARGUMENTS); using StringPiece = typename Regexps::Regexp::StringPieceType; - auto holder = Regexps::get(needle); - const auto & regexp = holder->getRE2(); + const Regexps::Regexp holder = Regexps::createRegexp(needle); + const auto & regexp = holder.getRE2(); if (!regexp) throw Exception("There are no groups in regexp: " + needle, ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Functions/extractGroups.cpp b/src/Functions/extractGroups.cpp index c5b958ec345..940e76df1c0 100644 --- a/src/Functions/extractGroups.cpp +++ b/src/Functions/extractGroups.cpp @@ -63,8 +63,8 @@ public: if (needle.empty()) throw Exception(getName() + " length of 'needle' argument must be greater than 0.", ErrorCodes::BAD_ARGUMENTS); - auto regexp = Regexps::get(needle); - const auto & re2 = regexp->getRE2(); + const Regexps::Regexp regexp = Regexps::createRegexp(needle); + const auto & re2 = regexp.getRE2(); if (!re2) throw Exception("There are no groups in regexp: " + needle, ErrorCodes::BAD_ARGUMENTS); diff --git a/tests/performance/like_and_match_pattern_caching.xml b/tests/performance/like_and_match_pattern_caching.xml new file mode 100644 index 00000000000..c0a8ec9442e --- /dev/null +++ b/tests/performance/like_and_match_pattern_caching.xml @@ -0,0 +1,62 @@ + + + + + + numbers + + numbers_mt(2000000) + + + + needle_like + + simple patterns, all unique + '%' || toString(number) || '_' + simple patterns, low distinctness (10 patterns) + '%' || toString(number % 10) || '_' + + + + needle_match + + + '.*' || toString(number) || '.' + + '.*' || toString(number % 10) || '.' + + '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number) + + '([a-zA-Z][a-zA-Z0-9]*)://([^ /]+)(/[^ ]*)?([^ @]+)@([^ @]+)([0-9][0-9]?)/([0-9][0-9]?)/([0-9][0-9]([0-9][0-9])?)(?:(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])\.){3}(?:25[0-5]|2[0-4][0-9]|[01]?[0-9][0-9])' || toString(number % 10) + + + + + + + + + select toString(number) as haystack, like(haystack, '%x_') + from(select * from {numbers}) + + + + select toString(number) as haystack, match(haystack, '.*x.') + from(select * from {numbers}) + + + + + + select toString(number) as haystack, {needle_like} as needle, like(haystack, needle) + from (select * from {numbers}); + + + + select toString(number) as haystack, {needle_match} as needle, match(haystack, needle) + from (select * from {numbers}); + + + From 9d04305a5a8614d61437c0ca1b598fde4014dca1 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 30 May 2022 23:00:28 +0200 Subject: [PATCH 032/101] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9111e1d80da..df8bbf0a307 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -371,7 +371,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, memory_profiler_step, (4 * 1024 * 1024), "Whenever query memory usage becomes larger than every next step in number of bytes the memory profiler will collect the allocating stack trace. Zero means disabled memory profiler. Values lower than a few megabytes will slow down query processing.", 0) \ M(Float, memory_profiler_sample_probability, 0., "Collect random allocations and deallocations and write them into system.trace_log with 'MemorySample' trace_type. The probability is for every alloc/free regardless to the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds 'max_untracked_memory'. You may want to set 'max_untracked_memory' to 0 for extra fine grained sampling.", 0) \ \ - M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit on user level. If timeout is reached and memory is not freed, exception is thrown.", 0) \ + M(UInt64, memory_usage_overcommit_max_wait_microseconds, 5'000'000, "Maximum time thread will wait for memory to be freed in the case of memory overcommit. If timeout is reached and memory is not freed, exception is thrown.", 0) \ \ M(UInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ M(UInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \ From 30f8eb800a9b2249462f2a72e07ddbae60ec876d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 30 May 2022 22:29:35 +0000 Subject: [PATCH 033/101] optimize function coalesce with two arguments --- src/Functions/coalesce.cpp | 12 ++++++++++-- tests/performance/conditional.xml | 3 +++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Functions/coalesce.cpp b/src/Functions/coalesce.cpp index 96aa110a489..b2b234df515 100644 --- a/src/Functions/coalesce.cpp +++ b/src/Functions/coalesce.cpp @@ -112,7 +112,6 @@ public: auto is_not_null = FunctionFactory::instance().get("isNotNull", context); auto assume_not_null = FunctionFactory::instance().get("assumeNotNull", context); - auto multi_if = FunctionFactory::instance().get("multiIf", context); ColumnsWithTypeAndName multi_if_args; ColumnsWithTypeAndName tmp_args(1); @@ -144,7 +143,16 @@ public: if (multi_if_args.size() == 1) return multi_if_args.front().column; - ColumnPtr res = multi_if->build(multi_if_args)->execute(multi_if_args, result_type, input_rows_count); + /// If there was only two arguments (3 arguments passed to multiIf) + /// use function "if" instead, because it's implemented more efficient. + /// TODO: make "multiIf" the same efficient. + FunctionOverloadResolverPtr if_function; + if (multi_if_args.size() == 3) + if_function = FunctionFactory::instance().get("if", context); + else + if_function = FunctionFactory::instance().get("multiIf", context); + + ColumnPtr res = if_function->build(multi_if_args)->execute(multi_if_args, result_type, input_rows_count); /// if last argument is not nullable, result should be also not nullable if (!multi_if_args.back().column->isNullable() && res->isNullable()) diff --git a/tests/performance/conditional.xml b/tests/performance/conditional.xml index 91b6cb95ff2..e1d15852767 100644 --- a/tests/performance/conditional.xml +++ b/tests/performance/conditional.xml @@ -8,4 +8,7 @@ SELECT count() FROM zeros(10000000) WHERE NOT ignore(multiIf(rand() % 2, toDateTime(rand()), toDate(rand()))) SELECT count() FROM zeros(10000000) WHERE NOT ignore(if(rand() % 2, [toDateTime(rand())], [toDate(rand())])) SELECT count() FROM zeros(10000000) WHERE NOT ignore(multiIf(rand() % 2, [toDateTime(rand())], [toDate(rand())])) + + SELECT count() FROM numbers(50000000) WHERE NOT ignore(COALESCE(toNullable(number), 0)) + SELECT count() FROM numbers(10000000) WHERE NOT ignore(multiIf(number % 7 = 0, 1, number % 3 = 0, 2, number % 5 = 0, 3, number % 11 = 0, 4, 5)) From a2857491c42d560e882debe6aa2f5563079a7841 Mon Sep 17 00:00:00 2001 From: yaqi-zhao Date: Fri, 27 May 2022 13:54:11 -0400 Subject: [PATCH 034/101] add avx512 support for mergetreereader --- src/Common/TargetSpecific.cpp | 3 ++ src/Common/TargetSpecific.h | 1 + .../MergeTree/MergeTreeRangeReader.cpp | 29 +++++++++++++++++++ 3 files changed, 33 insertions(+) diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index 43319eff44b..369c21490d4 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -16,6 +16,8 @@ UInt32 getSupportedArchs() result |= static_cast(TargetArch::AVX2); if (Cpu::CpuFlagsCache::have_AVX512F) result |= static_cast(TargetArch::AVX512F); + if (Cpu::CpuFlagsCache::have_AVX512BW) + result |= static_cast(TargetArch::AVX512BW); return result; } @@ -34,6 +36,7 @@ String toString(TargetArch arch) case TargetArch::AVX: return "avx"; case TargetArch::AVX2: return "avx2"; case TargetArch::AVX512F: return "avx512f"; + case TargetArch::AVX512BW: return "avx512bw"; } __builtin_unreachable(); diff --git a/src/Common/TargetSpecific.h b/src/Common/TargetSpecific.h index d7fa55fbb08..522dd6e43c3 100644 --- a/src/Common/TargetSpecific.h +++ b/src/Common/TargetSpecific.h @@ -80,6 +80,7 @@ enum class TargetArch : UInt32 AVX = (1 << 1), AVX2 = (1 << 2), AVX512F = (1 << 3), + AVX512BW = (1 << 4), }; /// Runtime detection. diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index d8dba458203..84a1ab91906 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -10,6 +11,7 @@ #include #endif + namespace DB { namespace ErrorCodes @@ -449,6 +451,33 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con { size_t count = 0; +#if defined(__AVX512F__) && defined(__AVX512BW__) /// check if avx512 instructions are compiled + if (isArchSupported(TargetArch::AVX512BW)) { + /// check if cpu support avx512 dynamically, haveAVX512BW contains check of haveAVX512F + const __m512i zero64 = _mm512_setzero_epi32(); + while (end - begin >= 64) + { + end -= 64; + const auto * pos = end; + UInt64 val = static_cast(_mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(pos)), zero64, _MM_CMPINT_EQ)); + val = ~val; + if (val == 0) + { + count += 64; + } else + { + count += __builtin_clzll(val); + return count; + } + } + while (end > begin && *(--end) == 0) + { + ++count; + } + return count; + } +#endif + #if defined(__SSE2__) && defined(__POPCNT__) const __m128i zero16 = _mm_setzero_si128(); while (end - begin >= 64) From ba4cdd43bd19e79d2d94d491968a93755a20c9e4 Mon Sep 17 00:00:00 2001 From: xlwh <1079039435@qq.com> Date: Tue, 31 May 2022 14:37:30 +0800 Subject: [PATCH 035/101] Cleanup unused file --- src/Storages/ColumnCodec.h | 11 ----------- src/Storages/ColumnsDescription.h | 1 - 2 files changed, 12 deletions(-) delete mode 100644 src/Storages/ColumnCodec.h diff --git a/src/Storages/ColumnCodec.h b/src/Storages/ColumnCodec.h deleted file mode 100644 index 63a604c8198..00000000000 --- a/src/Storages/ColumnCodec.h +++ /dev/null @@ -1,11 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - using ColumnCodecs = std::unordered_map; -} diff --git a/src/Storages/ColumnsDescription.h b/src/Storages/ColumnsDescription.h index c81ccb5d217..209dee885f4 100644 --- a/src/Storages/ColumnsDescription.h +++ b/src/Storages/ColumnsDescription.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include From 7e95bf31b24eafe4955dc66ef4b6a7e6c86f6fe1 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 31 May 2022 09:26:26 +0200 Subject: [PATCH 036/101] more verbose sanity checks --- programs/server/Server.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 75deeeb2eb5..17a9e641a57 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -547,7 +547,7 @@ static void sanityChecks(Server & server) try { if (readString("/sys/devices/system/clocksource/clocksource0/current_clocksource").find("tsc") == std::string::npos) - server.context()->addWarningMessage("Linux is not using a fast TSC clock source. Performance can be degraded."); + server.context()->addWarningMessage("Linux is not using a fast TSC clock source. Performance can be degraded. Check /sys/devices/system/clocksource/clocksource0/current_clocksource"); } catch (...) { @@ -556,7 +556,7 @@ static void sanityChecks(Server & server) try { if (readNumber("/proc/sys/vm/overcommit_memory") == 2) - server.context()->addWarningMessage("Linux memory overcommit is disabled."); + server.context()->addWarningMessage("Linux memory overcommit is disabled. Check /proc/sys/vm/overcommit_memory"); } catch (...) { @@ -565,7 +565,7 @@ static void sanityChecks(Server & server) try { if (readString("/sys/kernel/mm/transparent_hugepage/enabled").find("[always]") != std::string::npos) - server.context()->addWarningMessage("Linux transparent hugepages are set to \"always\"."); + server.context()->addWarningMessage("Linux transparent hugepages are set to \"always\". Check /sys/kernel/mm/transparent_hugepage/enabled"); } catch (...) { @@ -574,7 +574,7 @@ static void sanityChecks(Server & server) try { if (readNumber("/proc/sys/kernel/pid_max") < 30000) - server.context()->addWarningMessage("Linux max PID is too low."); + server.context()->addWarningMessage("Linux max PID is too low. Check /proc/sys/kernel/pid_max"); } catch (...) { @@ -583,7 +583,7 @@ static void sanityChecks(Server & server) try { if (readNumber("/proc/sys/kernel/threads-max") < 30000) - server.context()->addWarningMessage("Linux threads max count is too low."); + server.context()->addWarningMessage("Linux threads max count is too low. Check /proc/sys/kernel/threads-max"); } catch (...) { @@ -591,7 +591,7 @@ static void sanityChecks(Server & server) std::string dev_id = getBlockDeviceId(data_path); if (getBlockDeviceType(dev_id) == BlockDeviceType::ROT && getBlockDeviceReadAheadBytes(dev_id) == 0) - server.context()->addWarningMessage("Rotational disk with disabled readahead is in use. Performance can be degraded."); + server.context()->addWarningMessage("Rotational disk with disabled readahead is in use. Performance can be degraded. Used for data: " + String(data_path)); #endif try From 026e073b0bb25dec684c292fa149a3ea7fc57ecd Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Tue, 31 May 2022 13:50:09 +0200 Subject: [PATCH 037/101] minor improvement --- programs/server/Server.cpp | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 17a9e641a57..311a9c284da 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -546,8 +546,9 @@ static void sanityChecks(Server & server) #if defined(OS_LINUX) try { - if (readString("/sys/devices/system/clocksource/clocksource0/current_clocksource").find("tsc") == std::string::npos) - server.context()->addWarningMessage("Linux is not using a fast TSC clock source. Performance can be degraded. Check /sys/devices/system/clocksource/clocksource0/current_clocksource"); + const char * filename = "/sys/devices/system/clocksource/clocksource0/current_clocksource"; + if (readString(filename).find("tsc") == std::string::npos) + server.context()->addWarningMessage("Linux is not using a fast TSC clock source. Performance can be degraded. Check " + String(filename)); } catch (...) { @@ -555,8 +556,9 @@ static void sanityChecks(Server & server) try { - if (readNumber("/proc/sys/vm/overcommit_memory") == 2) - server.context()->addWarningMessage("Linux memory overcommit is disabled. Check /proc/sys/vm/overcommit_memory"); + const char * filename = "/proc/sys/vm/overcommit_memory"; + if (readNumber(filename) == 2) + server.context()->addWarningMessage("Linux memory overcommit is disabled. Check " + String(filename)); } catch (...) { @@ -564,8 +566,9 @@ static void sanityChecks(Server & server) try { - if (readString("/sys/kernel/mm/transparent_hugepage/enabled").find("[always]") != std::string::npos) - server.context()->addWarningMessage("Linux transparent hugepages are set to \"always\". Check /sys/kernel/mm/transparent_hugepage/enabled"); + const char * filename = "/sys/kernel/mm/transparent_hugepage/enabled"; + if (readString(filename).find("[always]") != std::string::npos) + server.context()->addWarningMessage("Linux transparent hugepages are set to \"always\". Check " + String(filename)); } catch (...) { @@ -573,8 +576,9 @@ static void sanityChecks(Server & server) try { - if (readNumber("/proc/sys/kernel/pid_max") < 30000) - server.context()->addWarningMessage("Linux max PID is too low. Check /proc/sys/kernel/pid_max"); + const char * filename = "/proc/sys/kernel/pid_max"; + if (readNumber(filename) < 30000) + server.context()->addWarningMessage("Linux max PID is too low. Check " + String(filename)); } catch (...) { @@ -582,8 +586,9 @@ static void sanityChecks(Server & server) try { - if (readNumber("/proc/sys/kernel/threads-max") < 30000) - server.context()->addWarningMessage("Linux threads max count is too low. Check /proc/sys/kernel/threads-max"); + const char * filename = "/proc/sys/kernel/threads-max"; + if (readNumber(filename) < 30000) + server.context()->addWarningMessage("Linux threads max count is too low. Check " + String(filename)); } catch (...) { From 582be423298958b72bfc7c88da90bc6a58ce9dc0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 31 May 2022 08:31:00 +0000 Subject: [PATCH 038/101] Wait for leader election --- tests/integration/test_keeper_force_recovery/test.py | 12 +++++++++++- .../test_keeper_force_recovery_single_node/test.py | 12 +++++++++++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_keeper_force_recovery/test.py b/tests/integration/test_keeper_force_recovery/test.py index e4f42ba21f6..5f1b7d1e4e4 100644 --- a/tests/integration/test_keeper_force_recovery/test.py +++ b/tests/integration/test_keeper_force_recovery/test.py @@ -132,7 +132,17 @@ def test_cluster_recovery(started_cluster): nodes[0].stop_clickhouse() - add_data(node_zks[1], "/test_force_recovery_extra", "somedataextra") + # we potentially killed the leader node so we give time for election + for _ in range(100): + try: + node_zks[1] = get_fake_zk(nodes[1].name, timeout=30.0) + add_data(node_zks[1], "/test_force_recovery_extra", "somedataextra") + break + except Exception as ex: + time.sleep(0.5) + print(f"Retrying create on {nodes[1].name}, exception {ex}") + else: + raise Exception(f"Failed creating a node on {nodes[1].name}") for node_zk in node_zks[2:CLUSTER_SIZE]: wait_and_assert_data(node_zk, "/test_force_recovery_extra", "somedataextra") diff --git a/tests/integration/test_keeper_force_recovery_single_node/test.py b/tests/integration/test_keeper_force_recovery_single_node/test.py index 1e58a25221e..0a554e33119 100644 --- a/tests/integration/test_keeper_force_recovery_single_node/test.py +++ b/tests/integration/test_keeper_force_recovery_single_node/test.py @@ -121,7 +121,17 @@ def test_cluster_recovery(started_cluster): nodes[0].stop_clickhouse() - add_data(node_zks[1], "/test_force_recovery_extra", "somedataextra") + # we potentially killed the leader node so we give time for election + for _ in range(100): + try: + node_zks[1] = get_fake_zk(nodes[1].name, timeout=30.0) + add_data(node_zks[1], "/test_force_recovery_extra", "somedataextra") + break + except Exception as ex: + time.sleep(0.5) + print(f"Retrying create on {nodes[1].name}, exception {ex}") + else: + raise Exception(f"Failed creating a node on {nodes[1].name}") for node_zk in node_zks[2:CLUSTER_SIZE]: wait_and_assert_data(node_zk, "/test_force_recovery_extra", "somedataextra") From 69cd3a2b1060c1a2df4f4ab836ce4203b5c9a8ac Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 31 May 2022 14:20:31 +0200 Subject: [PATCH 039/101] Fix --- src/Databases/MySQL/DatabaseMySQL.cpp | 13 +++++++++++-- .../integration/test_mysql_database_engine/test.py | 4 ++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 446518be5cd..58be682bd73 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -26,6 +26,7 @@ # include # include # include +# include namespace fs = std::filesystem; @@ -148,8 +149,16 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context auto storage_engine_arguments = ast_storage->engine->arguments; /// Add table_name to engine arguments - auto mysql_table_name = std::make_shared(table_name); - storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); + if (typeid_cast(storage_engine_arguments->children[0].get())) + { + storage_engine_arguments->children.push_back( + makeASTFunction("equals", std::make_shared("table"), std::make_shared(table_name))); + } + else + { + auto mysql_table_name = std::make_shared(table_name); + storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); + } /// Unset settings std::erase_if(storage_children, [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }); diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 3e0a1a549d1..500d9176f4b 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -930,6 +930,10 @@ def test_predefined_connection_configuration(started_cluster): == "100" ) + result = clickhouse_node.query("show create table test_database.test_table") + print(result) + assert(result.strip() == "CREATE TABLE test_database.test_table\\n(\\n `id` Int32\\n)\\nENGINE = MySQL(mysql1, table = \\'test_table\\')") + clickhouse_node.query("DROP DATABASE test_database") clickhouse_node.query_and_get_error( "CREATE DATABASE test_database ENGINE = MySQL(mysql2)" From c2087b3145d53d6569578c2dff35f90981f30571 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 31 May 2022 14:38:11 +0200 Subject: [PATCH 040/101] Fix --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 3 ++- tests/integration/test_storage_rabbitmq/test.py | 11 +++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 678416ddd42..6d0a3f4ab6c 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -1154,7 +1154,8 @@ void registerStorageRabbitMQ(StorageFactory & factory) if (!with_named_collection && !args.storage_def->settings) throw Exception(ErrorCodes::BAD_ARGUMENTS, "RabbitMQ engine must have settings"); - rabbitmq_settings->loadFromQuery(*args.storage_def); + if (args.storage_def->settings) + rabbitmq_settings->loadFromQuery(*args.storage_def); if (!rabbitmq_settings->rabbitmq_host_port.changed && !rabbitmq_settings->rabbitmq_address.changed) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index b18d9c26d88..c1bd136126f 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -27,6 +27,7 @@ instance = cluster.add_instance( ], user_configs=["configs/users.xml"], with_rabbitmq=True, + stay_alive=True, ) @@ -2732,6 +2733,16 @@ def test_rabbitmq_predefined_configuration(rabbitmq_cluster): ) if result == "1\t2\n": break + instance.restart_clickhouse() + channel.basic_publish( + exchange="named", routing_key="", body=json.dumps({"key": 1, "value": 2}) + ) + while True: + result = instance.query( + "SELECT * FROM test.rabbitmq ORDER BY key", ignore_error=True + ) + if result == "1\t2\n": + break if __name__ == "__main__": From 4b427336e3654f0880cb510feb544f01c7b33c23 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Tue, 31 May 2022 09:37:34 -0400 Subject: [PATCH 041/101] tests with overridden and appended parameters --- tests/integration/test_storage_dict/configs/conf.xml | 9 +++++++++ tests/integration/test_storage_dict/test.py | 12 ++++++++++++ 2 files changed, 21 insertions(+) diff --git a/tests/integration/test_storage_dict/configs/conf.xml b/tests/integration/test_storage_dict/configs/conf.xml index c2ecb518884..e37ca358e63 100644 --- a/tests/integration/test_storage_dict/configs/conf.xml +++ b/tests/integration/test_storage_dict/configs/conf.xml @@ -12,5 +12,14 @@ /test_dict TabSeparated + + http://nginx:80/ + /test_dict + + + http://nginx:80/ + /test_dict + CSV + diff --git a/tests/integration/test_storage_dict/test.py b/tests/integration/test_storage_dict/test.py index a5270a42114..1ed974f267d 100644 --- a/tests/integration/test_storage_dict/test.py +++ b/tests/integration/test_storage_dict/test.py @@ -32,3 +32,15 @@ def test_storage_dict(cluster): ) result = node1.query(f"select * from dict") assert result.strip() == "foo\tbar" + + node1.query( + f"create dictionary dict1 (k String, v String) primary key k source(http(name urldict1 format TabSeparated)) layout(complex_key_hashed()) lifetime(min 0 max 100)" + ) + result = node1.query(f"select * from dict1") + assert result.strip() == "foo\tbar" + + node1.query( + f"create dictionary dict2 (k String, v String) primary key k source(http(name urldict2 format TabSeparated)) layout(complex_key_hashed()) lifetime(min 0 max 100)" + ) + result = node1.query(f"select * from dict2") + assert result.strip() == "foo\tbar" From 792adb05767a8010d40ae98991d1d25673f2be9c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 31 May 2022 13:53:45 +0000 Subject: [PATCH 042/101] Update jepsen and scp --- tests/jepsen.clickhouse-keeper/project.clj | 2 +- .../src/jepsen/clickhouse_keeper/db.clj | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/tests/jepsen.clickhouse-keeper/project.clj b/tests/jepsen.clickhouse-keeper/project.clj index a168ef6920d..187e91cd44d 100644 --- a/tests/jepsen.clickhouse-keeper/project.clj +++ b/tests/jepsen.clickhouse-keeper/project.clj @@ -7,7 +7,7 @@ :main jepsen.clickhouse-keeper.main :plugins [[lein-cljfmt "0.7.0"]] :dependencies [[org.clojure/clojure "1.10.1"] - [jepsen "0.2.3"] + [jepsen "0.2.6"] [zookeeper-clj "0.9.4"] [org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]] :repl-options {:init-ns jepsen.clickhouse-keeper.main}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index 0df704f0d4a..39d4cac8521 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -10,6 +10,25 @@ [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu])) + +(ns jepsen.control.scp) + +(defn scp! + "Runs an SCP command by shelling out. Takes a conn-spec (used for port, key, + etc), a seq of sources, and a single destination, all as strings." + [conn-spec sources dest] + (apply util/sh "scp" "-rpC" + "-P" (str (:port conn-spec)) + (concat (when-let [k (:private-key-path conn-spec)] + ["-i" k]) + (if-not (:strict-host-key-checking conn-spec) + ["-o StrictHostKeyChecking=no"]) + sources + [dest])) + nil) + +(ns jepsen.clickhouse-keeper.db) + (defn get-clickhouse-url [url] (non-precise-cached-wget! url)) From 3e71a716f53862186a318c5357f01fb6e8eccb8b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 31 May 2022 13:55:01 +0000 Subject: [PATCH 043/101] Enable only jepsen tests --- .github/workflows/pull_request.yml | 3195 +--------------------------- 1 file changed, 1 insertion(+), 3194 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f6e9880d088..375d3e1cd6e 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -109,196 +109,8 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json - StyleCheck: - needs: DockerHubPush - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{ runner.temp }}/style_check - EOF - - name: Download changed images - # even if artifact does not exist, e.g. on `do not test` label or failed Docker job - continue-on-error: true - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.TEMP_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Style Check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 style_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FastTest: - needs: DockerHubPush - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/fasttest - REPO_COPY=${{runner.temp}}/fasttest/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" - mkdir "$GITHUB_WORKSPACE" - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.TEMP_PATH }} - - name: Fast Test - run: | - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - CompatibilityCheck: - needs: [BuilderDebRelease] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/compatibility_check - REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - SplitBuildSmokeTest: - needs: [BuilderDebSplitted] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/split_build_check - REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Split build check - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -######################################################################################### -#################################### ORDINARY BUILDS #################################### -######################################################################################### - BuilderDebRelease: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_release - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # for performance artifact - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" BuilderBinRelease: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -342,3014 +154,9 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - # BuilderBinGCC: - # needs: [DockerHubPush, FastTest] - # runs-on: [self-hosted, builder] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/build_check - # IMAGES_PATH=${{runner.temp}}/images_path - # REPO_COPY=${{runner.temp}}/build_check/ClickHouse - # CACHES_PATH=${{runner.temp}}/../ccaches - # BUILD_NAME=binary_gcc - # EOF - # - name: Download changed images - # uses: actions/download-artifact@v2 - # with: - # name: changed_images - # path: ${{ runner.temp }}/images_path - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # - name: Build - # run: | - # git -C "$GITHUB_WORKSPACE" submodule sync - # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - # - name: Upload build URLs to artifacts - # if: ${{ success() || failure() }} - # uses: actions/upload-artifact@v2 - # with: - # name: ${{ env.BUILD_URLS }} - # path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - # - name: Cleanup - # if: always() - # run: | - # # shellcheck disable=SC2046 - # docker kill $(docker ps -q) ||: - # # shellcheck disable=SC2046 - # docker rm -f $(docker ps -a -q) ||: - # sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebAarch64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # for performance artifact - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebAsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_asan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebUBsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_ubsan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebTsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_tsan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebMsan: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_msan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebDebug: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_debug - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -########################################################################################## -##################################### SPECIAL BUILDS ##################################### -########################################################################################## - BuilderDebSplitted: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_splitted - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinTidy: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_tidy - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinDarwin: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_darwin - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinAarch64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinFreeBSD: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_freebsd - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinDarwinAarch64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_darwin_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinPPC64: - needs: [DockerHubPush, FastTest] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_ppc64le - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -############################################################################################ -##################################### Docker images ####################################### -############################################################################################ - DockerServerImages: - needs: - - BuilderDebRelease - - BuilderDebAarch64 - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - - name: Check docker clickhouse/clickhouse-server building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head --no-push - python3 docker_server.py --release-type head --no-push --no-ubuntu \ - --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################ -##################################### BUILD REPORTER ####################################### -############################################################################################ - BuilderReport: - needs: - - BuilderBinRelease - - BuilderDebAarch64 - - BuilderDebAsan - - BuilderDebDebug - - BuilderDebMsan - - BuilderDebRelease - - BuilderDebTsan - - BuilderDebUBsan - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check (actions) - REPORTS_PATH=${{runner.temp}}/reports_dir - TEMP_PATH=${{runner.temp}}/report_check - NEEDS_DATA_PATH=${{runner.temp}}/needs.json - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cat > "$NEEDS_DATA_PATH" << 'EOF' - ${{ toJSON(needs) }} - EOF - cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - BuilderSpecialReport: - needs: - - BuilderBinAarch64 - - BuilderBinDarwin - - BuilderBinDarwinAarch64 - - BuilderBinFreeBSD - # - BuilderBinGCC - - BuilderBinPPC64 - - BuilderBinTidy - - BuilderDebSplitted - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=ClickHouse special build check (actions) - NEEDS_DATA_PATH=${{runner.temp}}/needs.json - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cat > "$NEEDS_DATA_PATH" << 'EOF' - ${{ toJSON(needs) }} - EOF - cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -########################### FUNCTIONAl STATELESS TESTS ####################################### -############################################################################################## - FunctionalStatelessTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, actions) - REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseDatabaseReplicated0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_database_replicated - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) - REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseDatabaseReplicated1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_database_replicated - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) - REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseWideParts: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_wide_parts - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, wide parts enabled, actions) - REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseS3: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, s3 storage, actions) - REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAarch64: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (aarch64, actions) - REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAsan0: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAsan1: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (address, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (ubsan, actions) - REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan0: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan1: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan2: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug0: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug1: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug2: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_flaky_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests flaky check (address, actions) - REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - TestsBugfixCheck: - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/tests_bugfix_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Tests bugfix validate check (actions) - KILL_TIMEOUT=3600 - REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Bugfix test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - - TEMP_PATH="${TEMP_PATH}/integration" \ - REPORTS_PATH="${REPORTS_PATH}/integration" \ - python3 integration_test_check.py "Integration tests bugfix validate check" \ - --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - - TEMP_PATH="${TEMP_PATH}/stateless" \ - REPORTS_PATH="${REPORTS_PATH}/stateless" \ - python3 functional_test_check.py "Stateless tests bugfix validate check" "$KILL_TIMEOUT" \ - --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - - python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -############################ FUNCTIONAl STATEFUL TESTS ####################################### -############################################################################################## - FunctionalStatefulTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release, actions) - REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestAarch64: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (aarch64, actions) - REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (address, actions) - REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (thread, actions) - REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (memory, actions) - REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (ubsan, actions) - REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug, actions) - REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -######################################### STRESS TESTS ####################################### -############################################################################################## - StressTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_thread - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (address, actions) - REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestTsan: - needs: [BuilderDebTsan] - # func testers have 16 cores + 128 GB memory - # while stress testers have 36 cores + 72 memory - # It would be better to have something like 32 + 128, - # but such servers almost unavailable as spot instances. - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_thread - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (thread, actions) - REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (memory, actions) - REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_undefined - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (undefined, actions) - REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - StressTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (debug, actions) - REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -##################################### AST FUZZERS ############################################ -############################################################################################## - ASTFuzzerTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ASan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (TSan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestUBSan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (UBSan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestMSan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (MSan, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (debug, actions) - REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -############################# INTEGRATION TESTS ############################################# -############################################################################################# - IntegrationTestsAsan0: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsAsan1: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsAsan2: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan3: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (thread, actions) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsRelease0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) - REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsRelease1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release, actions) - REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests flaky check (asan, actions) - REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -#################################### UNIT TESTS ############################################# -############################################################################################# - UnitTestsAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (asan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsReleaseClang: - needs: [BuilderBinRelease] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang, actions) - REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - # UnitTestsReleaseGCC: - # needs: [BuilderBinGCC] - # runs-on: [self-hosted, fuzzer-unit-tester] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/unit_tests_asan - # REPORTS_PATH=${{runner.temp}}/reports_dir - # CHECK_NAME=Unit tests (release-gcc, actions) - # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - # EOF - # - name: Download json reports - # uses: actions/download-artifact@v2 - # with: - # path: ${{ env.REPORTS_PATH }} - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # - name: Unit test - # run: | - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" - # python3 unit_tests_check.py "$CHECK_NAME" - # - name: Cleanup - # if: always() - # run: | - # # shellcheck disable=SC2046 - # docker kill $(docker ps -q) ||: - # # shellcheck disable=SC2046 - # docker rm -f $(docker ps -a -q) ||: - # sudo rm -fr "$TEMP_PATH" - UnitTestsTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (tsan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (msan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (ubsan, actions) - REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -#################################### PERFORMANCE TESTS ###################################### -############################################################################################# - PerformanceComparison0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison (actions) - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparison1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison (actions) - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparison2: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison (actions) - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparison3: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison (actions) - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - # shellcheck disable=SC2046 - docker kill $(docker ps -q) ||: - # shellcheck disable=SC2046 - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# Jepsen: needs: [BuilderBinRelease] uses: ./.github/workflows/jepsen.yml - - FinishCheck: - needs: - - StyleCheck - - DockerHubPush - - DockerServerImages - - CheckLabels - - BuilderReport - - FastTest - - FunctionalStatelessTestDebug0 - - FunctionalStatelessTestDebug1 - - FunctionalStatelessTestDebug2 - - FunctionalStatelessTestRelease - - FunctionalStatelessTestReleaseDatabaseReplicated0 - - FunctionalStatelessTestReleaseDatabaseReplicated1 - - FunctionalStatelessTestReleaseWideParts - - FunctionalStatelessTestAarch64 - - FunctionalStatelessTestAsan0 - - FunctionalStatelessTestAsan1 - - FunctionalStatelessTestTsan0 - - FunctionalStatelessTestTsan1 - - FunctionalStatelessTestTsan2 - - FunctionalStatelessTestMsan0 - - FunctionalStatelessTestMsan1 - - FunctionalStatelessTestMsan2 - - FunctionalStatelessTestUBsan - - FunctionalStatefulTestDebug - - FunctionalStatefulTestRelease - - FunctionalStatefulTestAarch64 - - FunctionalStatefulTestAsan - - FunctionalStatefulTestTsan - - FunctionalStatefulTestMsan - - FunctionalStatefulTestUBsan - - FunctionalStatelessTestReleaseS3 - - StressTestDebug - - StressTestAsan - - StressTestTsan - - StressTestMsan - - StressTestUBsan - - ASTFuzzerTestDebug - - ASTFuzzerTestAsan - - ASTFuzzerTestTsan - - ASTFuzzerTestMSan - - ASTFuzzerTestUBSan - - IntegrationTestsAsan0 - - IntegrationTestsAsan1 - - IntegrationTestsAsan2 - - IntegrationTestsRelease0 - - IntegrationTestsRelease1 - - IntegrationTestsTsan0 - - IntegrationTestsTsan1 - - IntegrationTestsTsan2 - - IntegrationTestsTsan3 - - PerformanceComparison0 - - PerformanceComparison1 - - PerformanceComparison2 - - PerformanceComparison3 - - UnitTestsAsan - - UnitTestsTsan - - UnitTestsMsan - - UnitTestsUBsan - - UnitTestsReleaseClang - - SplitBuildSmokeTest - - CompatibilityCheck - - IntegrationTestsFlakyCheck - - Jepsen - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Finish label - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 finish_check.py From 49f815060aaa573d7b4534c957cbc8ce0daf7723 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 31 May 2022 15:18:44 +0000 Subject: [PATCH 044/101] Use tar for logs --- .../src/jepsen/clickhouse_keeper/db.clj | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index 39d4cac8521..fcc28a5a272 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -155,12 +155,12 @@ (info node "Coordination files exists, going to compress") (c/cd data-dir (c/exec :tar :czf "coordination.tar.gz" "coordination")))) - (if (cu/exists? (str logs-dir "/clickhouse-keeper.log")) + (if (cu/exists? (str logs-dir)) (do - (info node "Clickhouse logs exist, going to compress") - (c/cd logs-dir - (c/exec :gzip "clickhouse-keeper.log"))) (info node "Logs are missing"))) - (let [common-logs [stderr-file (str logs-dir "/clickhouse-keeper.log.gz") (str data-dir "/coordination.tar.gz")] + (info node "Logs exist, going to compress") + (c/cd common-prefix + (c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing"))) + (let [common-logs [(str common-prefix "/logs.tar.gz") (str data-dir "/coordination.tar.gz")] gdb-log (str logs-dir "/gdb.log")] (if (cu/exists? (str logs-dir "/gdb.log")) (conj common-logs gdb-log) From d1a4550b4fac51579fbe8f1aa0106e7a7a98be62 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 31 May 2022 17:23:41 +0200 Subject: [PATCH 045/101] Fix create or drop of sql user defined functions in readonly mode --- src/Access/ContextAccess.cpp | 3 ++- .../0_stateless/02315_readonly_create_function.reference | 1 + .../queries/0_stateless/02315_readonly_create_function.sh | 7 +++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02315_readonly_create_function.reference create mode 100755 tests/queries/0_stateless/02315_readonly_create_function.sh diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 46fdba9d65e..89cdcc76234 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -445,10 +445,11 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg const AccessFlags dictionary_ddl = AccessType::CREATE_DICTIONARY | AccessType::DROP_DICTIONARY; const AccessFlags function_ddl = AccessType::CREATE_FUNCTION | AccessType::DROP_FUNCTION; const AccessFlags table_and_dictionary_ddl = table_ddl | dictionary_ddl; + const AccessFlags table_and_dictionary_and_function_ddl = table_ddl | dictionary_ddl | function_ddl; const AccessFlags write_table_access = AccessType::INSERT | AccessType::OPTIMIZE; const AccessFlags write_dcl_access = AccessType::ACCESS_MANAGEMENT - AccessType::SHOW_ACCESS; - const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; + const AccessFlags not_readonly_flags = write_table_access | table_and_dictionary_and_function_ddl | write_dcl_access | AccessType::SYSTEM | AccessType::KILL_QUERY; const AccessFlags not_readonly_1_flags = AccessType::CREATE_TEMPORARY_TABLE; const AccessFlags ddl_flags = table_ddl | dictionary_ddl | function_ddl; diff --git a/tests/queries/0_stateless/02315_readonly_create_function.reference b/tests/queries/0_stateless/02315_readonly_create_function.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02315_readonly_create_function.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02315_readonly_create_function.sh b/tests/queries/0_stateless/02315_readonly_create_function.sh new file mode 100755 index 00000000000..03a4feb3038 --- /dev/null +++ b/tests/queries/0_stateless/02315_readonly_create_function.sh @@ -0,0 +1,7 @@ +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --readonly 1 --query "CREATE FUNCTION test_function AS (x) -> x + 1;" 2>&1 | grep -c -F 'Code: 164' From 544be03b41bb5368bdc65c9d9d3011033d1465e3 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 31 May 2022 11:46:53 -0400 Subject: [PATCH 046/101] Update caches.md spelling --- docs/en/operations/caches.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/caches.md b/docs/en/operations/caches.md index f2427810184..27e81256315 100644 --- a/docs/en/operations/caches.md +++ b/docs/en/operations/caches.md @@ -5,7 +5,7 @@ sidebar_label: Caches # Cache Types {#cache-types} -When performing queries, ClichHouse uses different caches. +When performing queries, ClickHouse uses different caches. Main cache types: From 2476c6a988603ed5ba3a56692e640253455f8d70 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 25 May 2022 14:13:40 +0000 Subject: [PATCH 047/101] Fix error on joining with dictionary on some conditions --- src/Interpreters/HashJoin.cpp | 7 ++++++- src/Interpreters/TableJoin.cpp | 7 +++++++ .../queries/0_stateless/01391_join_on_dict_crash.reference | 1 + tests/queries/0_stateless/01391_join_on_dict_crash.sql | 4 ++-- 4 files changed, 16 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 521fbe139e7..1f3e7af0f45 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -538,6 +538,7 @@ void HashJoin::dataMapInit(MapsVariant & map) bool HashJoin::overDictionary() const { + assert(data->type != Type::DICT || table_join->getDictionaryReader()); return data->type == Type::DICT; } @@ -910,13 +911,14 @@ public: AddedColumns( const Block & block_with_columns_to_add, const Block & block, - const Block & saved_block_sample, + const Block & saved_block_sample_, const HashJoin & join, std::vector && join_on_keys_, bool is_asof_join, bool is_join_get_) : join_on_keys(join_on_keys_) , rows_to_add(block.rows()) + , saved_block_sample(saved_block_sample_) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -959,6 +961,8 @@ public: template void appendFromBlock(const Block & block, size_t row_num) { + assertBlocksHaveEqualStructure(saved_block_sample, block, "appendFromBlock"); + if constexpr (has_defaults) applyLazyDefaults(); @@ -1024,6 +1028,7 @@ private: size_t lazy_defaults_count = 0; /// for ASOF const IColumn * left_asof_key = nullptr; + Block saved_block_sample; bool is_join_get; diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 10a27b9efc5..87502e5965e 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -479,6 +479,13 @@ bool TableJoin::tryInitDictJoin(const Block & sample_block, ContextPtr context) src_names.push_back(original); dst_columns.push_back({col.name, col.type}); } + else + { + /// Can't extract column from dictionary table + /// TODO: Sometimes it should be possible to recunstruct required column, + /// e.g. if it's an expression depending on dictionary attributes + return false; + } } dictionary_reader = std::make_shared(dict_name, src_names, dst_columns, context); diff --git a/tests/queries/0_stateless/01391_join_on_dict_crash.reference b/tests/queries/0_stateless/01391_join_on_dict_crash.reference index 573541ac970..aa47d0d46d4 100644 --- a/tests/queries/0_stateless/01391_join_on_dict_crash.reference +++ b/tests/queries/0_stateless/01391_join_on_dict_crash.reference @@ -1 +1,2 @@ 0 +0 diff --git a/tests/queries/0_stateless/01391_join_on_dict_crash.sql b/tests/queries/0_stateless/01391_join_on_dict_crash.sql index ac3ca94ddf6..854da04b334 100644 --- a/tests/queries/0_stateless/01391_join_on_dict_crash.sql +++ b/tests/queries/0_stateless/01391_join_on_dict_crash.sql @@ -20,8 +20,8 @@ SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() USER 'default' DB 'db_01391' t LIFETIME(MIN 1 MAX 1) LAYOUT(HASHED()); -select click_country_id from t cc -left join d on toUInt32(d.id) = cc.click_city_id; +SELECT click_country_id FROM t AS cc LEFT JOIN d ON toUInt32(d.id) = cc.click_city_id; +SELECT click_country_id FROM t AS cc LEFT JOIN d ON d.country_id < 99 AND d.id = cc.click_city_id; DROP DICTIONARY d; DROP TABLE t; From 7f4ddb16670ec6c0943e700071ab9ed0e6377abe Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 May 2022 12:59:47 +0000 Subject: [PATCH 048/101] Fix assert for 02244_lowcardinality_hash_join --- src/Interpreters/HashJoin.cpp | 33 ++++++++++++++++----------------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 1f3e7af0f45..c52c01930a4 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -911,14 +911,14 @@ public: AddedColumns( const Block & block_with_columns_to_add, const Block & block, - const Block & saved_block_sample_, + const Block & saved_block_sample, const HashJoin & join, std::vector && join_on_keys_, bool is_asof_join, bool is_join_get_) : join_on_keys(join_on_keys_) , rows_to_add(block.rows()) - , saved_block_sample(saved_block_sample_) + , sample_block(block_with_columns_to_add) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -959,24 +959,27 @@ public: } template - void appendFromBlock(const Block & block, size_t row_num) + void appendFromBlock(Block block, size_t row_num) { - assertBlocksHaveEqualStructure(saved_block_sample, block, "appendFromBlock"); - if constexpr (has_defaults) applyLazyDefaults(); + for (size_t j = 0, size = right_indexes.size(); j < size; ++j) + { + ColumnWithTypeAndName & column_from_block = block.getByPosition(right_indexes[j]); + if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality()) + { + JoinCommon::changeLowCardinalityInplace(column_from_block); + } + } + assertBlocksHaveEqualStructure(sample_block, block, "appendFromBlock"); + if (is_join_get) { /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. for (size_t j = 0, size = right_indexes.size(); j < size; ++j) { - auto column_from_block = block.getByPosition(right_indexes[j]); - if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality()) - { - JoinCommon::changeLowCardinalityInplace(column_from_block); - } - + const auto & column_from_block = block.getByPosition(right_indexes[j]); if (auto * nullable_col = typeid_cast(columns[j].get()); nullable_col && !column_from_block.column->isNullable()) nullable_col->insertFromNotNullable(*column_from_block.column, row_num); @@ -988,11 +991,7 @@ public: { for (size_t j = 0, size = right_indexes.size(); j < size; ++j) { - auto column_from_block = block.getByPosition(right_indexes[j]); - if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality()) - { - JoinCommon::changeLowCardinalityInplace(column_from_block); - } + const auto & column_from_block = block.getByPosition(right_indexes[j]); columns[j]->insertFrom(*column_from_block.column, row_num); } } @@ -1028,7 +1027,7 @@ private: size_t lazy_defaults_count = 0; /// for ASOF const IColumn * left_asof_key = nullptr; - Block saved_block_sample; + Block sample_block; bool is_join_get; From 673bc84bfc4b70a3163c74d45cd63746cb92d1f4 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 May 2022 12:59:58 +0000 Subject: [PATCH 049/101] Reformat 02244_lowcardinality_hash_join --- .../02244_lowcardinality_hash_join.sql | 20 ++++++++----------- 1 file changed, 8 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02244_lowcardinality_hash_join.sql b/tests/queries/0_stateless/02244_lowcardinality_hash_join.sql index f2a601adf06..4ea949e4255 100644 --- a/tests/queries/0_stateless/02244_lowcardinality_hash_join.sql +++ b/tests/queries/0_stateless/02244_lowcardinality_hash_join.sql @@ -8,20 +8,16 @@ CREATE TABLE lc_table INSERT INTO lc_table VALUES('x'); -SELECT * -FROM lc_table -INNER JOIN lc_table AS lc_table2 ON lc_table.col = lc_table2.col; +SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2 +ON lc_table.col = lc_table2.col; -SELECT * -FROM lc_table -INNER JOIN lc_table AS lc_table2 ON CAST(lc_table.col AS String) = CAST(lc_table2.col AS String); +SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2 +ON CAST(lc_table.col AS String) = CAST(lc_table2.col AS String); -SELECT * -FROM lc_table -INNER JOIN lc_table AS lc_table2 ON (lc_table.col = lc_table2.col) OR (lc_table.col = lc_table2.col); +SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2 +ON (lc_table.col = lc_table2.col) OR (lc_table.col = lc_table2.col); -SELECT * -FROM lc_table -INNER JOIN lc_table AS lc_table2 ON (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)) OR (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)); +SELECT * FROM lc_table INNER JOIN lc_table AS lc_table2 +ON (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)) OR (CAST(lc_table.col AS String) = CAST(lc_table2.col AS String)); DROP TABLE IF EXISTS lc_table; From e7be677fca158279578f6712dc1f407d21c3879e Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 May 2022 15:57:10 +0000 Subject: [PATCH 050/101] Assert structure match up to locard in appendFromBlock --- src/Interpreters/HashJoin.cpp | 33 +++++++++++++++++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index c52c01930a4..9f770b5cd46 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -918,7 +918,7 @@ public: bool is_join_get_) : join_on_keys(join_on_keys_) , rows_to_add(block.rows()) - , sample_block(block_with_columns_to_add) + , sample_block(saved_block_sample) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -958,6 +958,33 @@ public: return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].type, type_name[i].qualified_name); } + static void assertBlockEqualsStructureUpToLowCard(const Block & lhs_block, const Block & rhs_block) + { + if (lhs_block.columns() != rhs_block.columns()) + throw Exception("Different number of columns in blocks", ErrorCodes::LOGICAL_ERROR); + + for (size_t i = 0; i < lhs_block.columns(); ++i) + { + const auto & lhs = lhs_block.getByPosition(i); + const auto & rhs = rhs_block.getByPosition(i); + if (lhs.name != rhs.name) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]", + lhs_block.dumpStructure(), rhs_block.dumpStructure()); + + const auto & ltype = recursiveRemoveLowCardinality(lhs.type); + const auto & rtype = recursiveRemoveLowCardinality(rhs.type); + if (!ltype->equals(*rtype)) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]", + lhs_block.dumpStructure(), rhs_block.dumpStructure()); + + const auto & lcol = recursiveRemoveLowCardinality(lhs.column); + const auto & rcol = recursiveRemoveLowCardinality(rhs.column); + if (lcol->getDataType() != rcol->getDataType()) + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Block structure mismatch: [{}] != [{}]", + lhs_block.dumpStructure(), rhs_block.dumpStructure()); + } + } + template void appendFromBlock(Block block, size_t row_num) { @@ -972,7 +999,9 @@ public: JoinCommon::changeLowCardinalityInplace(column_from_block); } } - assertBlocksHaveEqualStructure(sample_block, block, "appendFromBlock"); + + /// Like assertBlocksHaveEqualStructure but doesn't check low cardinality + assertBlockEqualsStructureUpToLowCard(sample_block, block); if (is_join_get) { From ca0bd754b59897878924dcdc11ee0f5449fc5449 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 30 May 2022 15:58:57 +0000 Subject: [PATCH 051/101] Add no-backward-compatibility-check to 01391_join_on_dict_crash.sql --- tests/queries/0_stateless/01391_join_on_dict_crash.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01391_join_on_dict_crash.sql b/tests/queries/0_stateless/01391_join_on_dict_crash.sql index 854da04b334..13ebd080621 100644 --- a/tests/queries/0_stateless/01391_join_on_dict_crash.sql +++ b/tests/queries/0_stateless/01391_join_on_dict_crash.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-backward-compatibility-check DROP DATABASE IF EXISTS db_01391; CREATE DATABASE db_01391; From c5ac6294ae7d75c70f6691746e9c78fdcb2afc69 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 May 2022 16:16:13 +0000 Subject: [PATCH 052/101] Display entires for failed tests at the top of report --- tests/ci/report.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/ci/report.py b/tests/ci/report.py index c8bd32bf642..83d89e628a2 100644 --- a/tests/ci/report.py +++ b/tests/ci/report.py @@ -165,6 +165,11 @@ def create_test_html_report( num_fails = 0 has_test_time = False has_test_logs = False + + if with_raw_logs: + # Display entires with logs at the top (they correspond to failed tests) + test_result.sort(key=lambda result: len(result) <= 3) + for result in test_result: test_name = result[0] test_status = result[1] From b11749ca2c97700c2c6d1c96f1e1b6ad8c190065 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 31 May 2022 16:45:29 +0000 Subject: [PATCH 053/101] Make GROUPING function skip constant folding --- src/Interpreters/ActionsVisitor.cpp | 3 ++ .../02315_grouping_constant_folding.reference | 29 +++++++++++++++++++ .../02315_grouping_constant_folding.sql | 13 +++++++++ 3 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/02315_grouping_constant_folding.reference create mode 100644 tests/queries/0_stateless/02315_grouping_constant_folding.sql diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 99d2217cba3..eacdd221d6e 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -839,6 +839,9 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & if (node.name == "grouping") { + if (data.only_consts) + return; // Can not perform constant folding, because this function can be executed only after GROUP BY + size_t arguments_size = node.arguments->children.size(); if (arguments_size == 0) throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function GROUPING expects at least one argument"); diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.reference b/tests/queries/0_stateless/02315_grouping_constant_folding.reference new file mode 100644 index 00000000000..5aa979b1453 --- /dev/null +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.reference @@ -0,0 +1,29 @@ +-- { echoOn } +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); +1 0 0 3 +1 0 2 3 +1 0 4 3 +1 0 6 3 +1 0 8 3 +1 1 1 3 +1 1 3 3 +1 1 5 3 +1 1 7 3 +1 1 9 3 +5 0 0 2 +5 1 0 2 +10 0 0 0 +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); +1 0 0 3 +1 0 2 3 +1 0 4 3 +1 0 6 3 +1 0 8 3 +1 1 1 3 +1 1 3 3 +1 1 5 3 +1 1 7 3 +1 1 9 3 +5 0 0 2 +5 1 0 2 +10 0 0 0 diff --git a/tests/queries/0_stateless/02315_grouping_constant_folding.sql b/tests/queries/0_stateless/02315_grouping_constant_folding.sql new file mode 100644 index 00000000000..c4ef087a308 --- /dev/null +++ b/tests/queries/0_stateless/02315_grouping_constant_folding.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test02315; + +CREATE TABLE test02315(a UInt64, b UInt64) ENGINE=MergeTree() ORDER BY (a, b); + +INSERT INTO test02315 SELECT number % 2 as a, number as b FROM numbers(10); + +-- { echoOn } +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY GROUPING SETS ((a, b), (a), ()) ORDER BY (amount, a, b); + +SELECT count() AS amount, a, b, GROUPING(a, b) FROM test02315 GROUP BY ROLLUP(a, b) ORDER BY (amount, a, b); + +-- { echoOff } +DROP TABLE test02315; From 66f43b9ad340a3e1caadea0670222c8b145492ea Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 31 May 2022 18:46:33 +0200 Subject: [PATCH 054/101] Fix executable user default functions execution with Nullable arguments --- .../UserDefinedExecutableFunctionFactory.cpp | 2 +- .../functions/test_function_config.xml | 22 +++++++++++++++++++ .../test.py | 12 ++++++++++ .../user_scripts/input_nullable.py | 12 ++++++++++ 4 files changed, 47 insertions(+), 1 deletion(-) create mode 100755 tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py diff --git a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp index 5b5c7911735..b67e9c16ed5 100644 --- a/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp +++ b/src/Interpreters/UserDefinedExecutableFunctionFactory.cpp @@ -43,7 +43,7 @@ public: size_t getNumberOfArguments() const override { return executable_function->getConfiguration().arguments.size(); } bool useDefaultImplementationForConstants() const override { return true; } - bool useDefaultImplementationForNulls() const override { return true; } + bool useDefaultImplementationForNulls() const override { return false; } bool isDeterministic() const override { return false; } bool isDeterministicInScopeOfQuery() const override { return false; } diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index b2b7db83fbc..dce7ab2eacd 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -289,4 +289,26 @@ input_sum_json_named_args.py + + executable + test_function_nullable_python + String + + Nullable(UInt64) + + TabSeparated + input_nullable.py + + + + executable + test_function_nullable_pool_python + String + + Nullable(UInt64) + + TabSeparated + input_nullable.py + + diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 10993e9c5dd..20beded2284 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -228,3 +228,15 @@ def test_executable_function_sum_json_python(started_cluster): ) node.query("DROP TABLE test_table;") + +def test_executable_function_input_nullable_python(started_cluster): + skip_test_msan(node) + + node.query("CREATE TABLE test_table_nullable (value Nullable(UInt64)) ENGINE=TinyLog;") + node.query("INSERT INTO test_table_nullable VALUES (0), (NULL), (2);") + + assert(node.query("SELECT test_function_nullable_python(1), test_function_nullable_python(NULL)") == "Key 1\tKey Nullable\n") + assert(node.query("SELECT test_function_nullable_python(value) FROM test_table_nullable;") == "Key 0\nKey Nullable\nKey 2\n") + + assert(node.query("SELECT test_function_nullable_pool_python(1), test_function_nullable_pool_python(NULL)") == "Key 1\tKey Nullable\n") + assert(node.query("SELECT test_function_nullable_pool_python(value) FROM test_table_nullable;") == "Key 0\nKey Nullable\nKey 2\n") diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py new file mode 100755 index 00000000000..f2f00a335bd --- /dev/null +++ b/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py @@ -0,0 +1,12 @@ +#!/usr/bin/python3 + +import sys + +if __name__ == "__main__": + for line in sys.stdin: + if (line == "\\N\n"): + print("Key Nullable", end="\n") + else: + print("Key " + line, end="") + + sys.stdout.flush() From 284c9bc68bddab2badad7a699bb552bb28808663 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 May 2022 16:02:06 +0000 Subject: [PATCH 055/101] Rollback some changes from appendFromBlock --- src/Interpreters/HashJoin.cpp | 28 ++++++++++++++++------------ 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9f770b5cd46..07151b2bb1b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -986,29 +986,29 @@ public: } template - void appendFromBlock(Block block, size_t row_num) + void appendFromBlock(const Block & block, size_t row_num) { if constexpr (has_defaults) applyLazyDefaults(); - for (size_t j = 0, size = right_indexes.size(); j < size; ++j) - { - ColumnWithTypeAndName & column_from_block = block.getByPosition(right_indexes[j]); - if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality()) - { - JoinCommon::changeLowCardinalityInplace(column_from_block); - } - } - +#ifndef NDEBUG /// Like assertBlocksHaveEqualStructure but doesn't check low cardinality assertBlockEqualsStructureUpToLowCard(sample_block, block); +#else + UNUSED(assertBlockEqualsStructureUpToLowCard); +#endif if (is_join_get) { /// If it's joinGetOrNull, we need to wrap not-nullable columns in StorageJoin. for (size_t j = 0, size = right_indexes.size(); j < size; ++j) { - const auto & column_from_block = block.getByPosition(right_indexes[j]); + auto column_from_block = block.getByPosition(right_indexes[j]); + if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality()) + { + JoinCommon::changeLowCardinalityInplace(column_from_block); + } + if (auto * nullable_col = typeid_cast(columns[j].get()); nullable_col && !column_from_block.column->isNullable()) nullable_col->insertFromNotNullable(*column_from_block.column, row_num); @@ -1020,7 +1020,11 @@ public: { for (size_t j = 0, size = right_indexes.size(); j < size; ++j) { - const auto & column_from_block = block.getByPosition(right_indexes[j]); + auto column_from_block = block.getByPosition(right_indexes[j]); + if (type_name[j].type->lowCardinality() != column_from_block.type->lowCardinality()) + { + JoinCommon::changeLowCardinalityInplace(column_from_block); + } columns[j]->insertFrom(*column_from_block.column, row_num); } } From fdd190d010b7d9199b895f5ad3d6e600ebf4447e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 May 2022 18:40:14 +0000 Subject: [PATCH 056/101] Fix typo in 00814_replicated_minimalistic_part_header_zookeeper.sql --- .../00814_replicated_minimalistic_part_header_zookeeper.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 5d03823dde3..f1fb4a8c4dc 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -44,7 +44,7 @@ SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/2r1/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; From fbfab8558abd619e4535238330cc7ea1137b6f0d Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 May 2022 18:52:18 +0000 Subject: [PATCH 057/101] Add retries to 00814_replicated_minimalistic_part_header_zookeeper --- ...ted_minimalistic_part_header_zookeeper.sh} | 26 +++++++++++++++++-- 1 file changed, 24 insertions(+), 2 deletions(-) rename tests/queries/0_stateless/{00814_replicated_minimalistic_part_header_zookeeper.sql => 00814_replicated_minimalistic_part_header_zookeeper.sh} (78%) mode change 100644 => 100755 diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh old mode 100644 new mode 100755 similarity index 78% rename from tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql rename to tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index f1fb4a8c4dc..5917ed29533 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -1,4 +1,13 @@ --- Tags: replica +#!/usr/bin/env bash +# Tags: replica + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; @@ -36,7 +45,18 @@ SELECT _part, x FROM part_header_r1 ORDER BY x; SELECT '*** replica 2 ***'; SELECT _part, x FROM part_header_r2 ORDER BY x; -SELECT sleep(3) FORMAT Null; +" + +elapsed=1 +until [ $elapsed -eq 5 ]; +do + sleep $(( elapsed++ )) + count1=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'") + count2=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/2r1/parts'") + [[ $count1 == 1 && $count2 == 1 ]] && break +done + +$CLICKHOUSE_CLIENT -nm -q " SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; @@ -63,3 +83,5 @@ SELECT x, length(y) FROM part_header_r2 ORDER BY x; DROP TABLE part_header_r1; DROP TABLE part_header_r2; + +" From cf82df6ce782301db97f8424299cea3124c3a22b Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 31 May 2022 19:26:44 +0000 Subject: [PATCH 058/101] Use CLICKHOUSE_TEST_ZOOKEEPER_PREFIX in 00814_replicated_minimalistic_part_header --- ..._replicated_minimalistic_part_header_zookeeper.sh | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 5917ed29533..6f609065c01 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -15,13 +15,13 @@ DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/{shard}', '1{replica}') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/{shard}', '1{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; CREATE TABLE part_header_r2(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/{shard}', '2{replica}') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/{shard}', '2{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, cleanup_delay_period = 0, @@ -51,8 +51,8 @@ elapsed=1 until [ $elapsed -eq 5 ]; do sleep $(( elapsed++ )) - count1=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'") - count2=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/2r1/parts'") + count1=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/1r1/parts'") + count2=$($CLICKHOUSE_CLIENT --query="SELECT count(name) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/2r1/parts'") [[ $count1 == 1 && $count2 == 1 ]] && break done @@ -61,10 +61,10 @@ $CLICKHOUSE_CLIENT -nm -q " SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/2r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/test_00814/part_header/s1/replicas/2r1/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; From 26609a18751b38f0f866e3a3ad3f5309cc951136 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 31 May 2022 21:41:10 +0200 Subject: [PATCH 059/101] Style fixes --- src/Common/TargetSpecific.cpp | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Common/TargetSpecific.cpp b/src/Common/TargetSpecific.cpp index 369c21490d4..c52c8c2bcf0 100644 --- a/src/Common/TargetSpecific.cpp +++ b/src/Common/TargetSpecific.cpp @@ -17,7 +17,7 @@ UInt32 getSupportedArchs() if (Cpu::CpuFlagsCache::have_AVX512F) result |= static_cast(TargetArch::AVX512F); if (Cpu::CpuFlagsCache::have_AVX512BW) - result |= static_cast(TargetArch::AVX512BW); + result |= static_cast(TargetArch::AVX512BW); return result; } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 84a1ab91906..56bd62ee271 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -452,7 +452,8 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con size_t count = 0; #if defined(__AVX512F__) && defined(__AVX512BW__) /// check if avx512 instructions are compiled - if (isArchSupported(TargetArch::AVX512BW)) { + if (isArchSupported(TargetArch::AVX512BW)) + { /// check if cpu support avx512 dynamically, haveAVX512BW contains check of haveAVX512F const __m512i zero64 = _mm512_setzero_epi32(); while (end - begin >= 64) @@ -461,10 +462,9 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con const auto * pos = end; UInt64 val = static_cast(_mm512_cmp_epi8_mask(_mm512_loadu_si512(reinterpret_cast(pos)), zero64, _MM_CMPINT_EQ)); val = ~val; - if (val == 0) - { + if (val == 0) count += 64; - } else + else { count += __builtin_clzll(val); return count; From 6cf9405f09a5f50ef4f959d23b96b060c8d1fb51 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 Jun 2022 00:50:28 +0000 Subject: [PATCH 060/101] fix optimize_monotonous_functions_in_order_by in distributed queries --- src/Interpreters/SelectQueryOptions.h | 2 +- src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp | 9 ++++++++- src/Storages/ProjectionsDescription.cpp | 4 ++-- ...ize_monotonous_functions_in_order_by_remote.reference | 2 ++ ..._optimize_monotonous_functions_in_order_by_remote.sql | 6 ++++++ 5 files changed, 19 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference create mode 100644 tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql diff --git a/src/Interpreters/SelectQueryOptions.h b/src/Interpreters/SelectQueryOptions.h index b0183e2761b..032befe1993 100644 --- a/src/Interpreters/SelectQueryOptions.h +++ b/src/Interpreters/SelectQueryOptions.h @@ -135,7 +135,7 @@ struct SelectQueryOptions return *this; } - SelectQueryOptions & ignoreASTOptimizationsAlias(bool value = true) + SelectQueryOptions & ignoreASTOptimizations(bool value = true) { ignore_ast_optimizations = value; return *this; diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index ed9cdf547c2..854a677afb9 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -46,8 +46,15 @@ std::unique_ptr createLocalPlan( checkStackSize(); auto query_plan = std::make_unique(); + /// Do not apply AST optimizations, because query + /// is already optimized and some optimizations + /// can be applied only for non-distributed tables + /// and we can produce query, inconsistent with remote plans. auto interpreter = InterpreterSelectQuery( - query_ast, context, SelectQueryOptions(processed_stage).setShardInfo(shard_num, shard_count)); + query_ast, context, + SelectQueryOptions(processed_stage) + .setShardInfo(shard_num, shard_count) + .ignoreASTOptimizations()); interpreter.setProperClientInfo(); if (coordinator) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 5e9966a2794..ad35c5d420b 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -109,7 +109,7 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const InterpreterSelectQuery select( result.query_ast, query_context, storage, {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizationsAlias()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); @@ -221,7 +221,7 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( InterpreterSelectQuery select( result.query_ast, query_context, storage, {}, /// Here we ignore ast optimizations because otherwise aggregation keys may be removed from result header as constants. - SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizationsAlias()); + SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias().ignoreASTOptimizations()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); diff --git a/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference new file mode 100644 index 00000000000..aa47d0d46d4 --- /dev/null +++ b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.reference @@ -0,0 +1,2 @@ +0 +0 diff --git a/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql new file mode 100644 index 00000000000..6a5e4a0ae65 --- /dev/null +++ b/tests/queries/0_stateless/02315_optimize_monotonous_functions_in_order_by_remote.sql @@ -0,0 +1,6 @@ +SET prefer_localhost_replica = 1; +SET optimize_monotonous_functions_in_order_by = 1; + +SELECT * +FROM cluster(test_cluster_two_shards_localhost, system, one) +ORDER BY toDateTime(dummy); From 32d267ec6c1999cf89e0f1714c3fae1c0b177a0f Mon Sep 17 00:00:00 2001 From: Paul Loyd Date: Sun, 22 May 2022 20:17:16 +0800 Subject: [PATCH 061/101] Stop removing UTF-8 BOM in RowBinary* formats Fixes #37420 --- src/Processors/Formats/Impl/BinaryRowInputFormat.cpp | 1 + src/Processors/Formats/Impl/CSVRowInputFormat.cpp | 10 +++++++++- .../Formats/Impl/CustomSeparatedRowInputFormat.cpp | 1 + .../Formats/Impl/JSONCompactEachRowRowInputFormat.cpp | 1 + .../Formats/Impl/TabSeparatedRowInputFormat.cpp | 10 +++++++++- .../Formats/RowInputFormatWithNamesAndTypes.cpp | 9 ++++++--- .../Formats/RowInputFormatWithNamesAndTypes.h | 5 ++++- .../0_stateless/02306_rowbinary_has_no_bom.reference | 1 + .../queries/0_stateless/02306_rowbinary_has_no_bom.sh | 10 ++++++++++ 9 files changed, 42 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02306_rowbinary_has_no_bom.reference create mode 100755 tests/queries/0_stateless/02306_rowbinary_has_no_bom.sh diff --git a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp index d3de2fbf494..91bebd0daa4 100644 --- a/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/BinaryRowInputFormat.cpp @@ -18,6 +18,7 @@ BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Param header, in_, params_, + true, with_names_, with_types_, format_settings_, diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index bddd4203a5d..8f5591e6aa8 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -41,7 +41,15 @@ CSVRowInputFormat::CSVRowInputFormat( bool with_types_, const FormatSettings & format_settings_, std::unique_ptr format_reader_) - : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_, std::move(format_reader_)) + : RowInputFormatWithNamesAndTypes( + header_, + in_, + params_, + false, + with_names_, + with_types_, + format_settings_, + std::move(format_reader_)) { const String bad_delimiters = " \t\"'.UL"; if (bad_delimiters.find(format_settings.csv.delimiter) != String::npos) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 56a639a0e30..61488a94ccd 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -47,6 +47,7 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( header_, *buf_, params_, + false, with_names_, with_types_, format_settings_, diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp index 0b7cc6669be..1bc5223a712 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -28,6 +28,7 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat( header_, in_, params_, + false, with_names_, with_types_, format_settings_, diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 877ba224fd5..a50302697e6 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -40,7 +40,15 @@ TabSeparatedRowInputFormat::TabSeparatedRowInputFormat( bool with_types_, bool is_raw_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_, std::make_unique(in_, format_settings_, is_raw_)) + : RowInputFormatWithNamesAndTypes( + header_, + in_, + params_, + false, + with_names_, + with_types_, + format_settings_, + std::make_unique(in_, format_settings_, is_raw_)) { } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 0a20ac9eefd..a3dcbe914bb 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -17,6 +17,7 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( const Block & header_, ReadBuffer & in_, const Params & params_, + bool is_binary_, bool with_names_, bool with_types_, const FormatSettings & format_settings_, @@ -24,6 +25,7 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes( : RowInputFormatWithDiagnosticInfo(header_, in_, params_) , format_settings(format_settings_) , data_types(header_.getDataTypes()) + , is_binary(is_binary_) , with_names(with_names_) , with_types(with_types_) , format_reader(std::move(format_reader_)) @@ -38,10 +40,11 @@ void RowInputFormatWithNamesAndTypes::readPrefix() if (getCurrentUnitNumber() != 0) return; - if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8()) + /// Search and remove BOM only in textual formats (CSV, TSV etc), not in binary ones (RowBinary*). + /// Also, we assume that column name or type cannot contain BOM, so, if format has header, + /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. + if (!is_binary && (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8())) { - /// We assume that column name or type cannot contain BOM, so, if format has header, - /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. skipBOMIfExists(*in); } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 72fbc444dfc..9fc8b2083df 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -24,13 +24,15 @@ class FormatWithNamesAndTypesReader; class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo { protected: - /** with_names - in the first line the header with column names + /** is_binary - it is a binary format (e.g. don't search for BOM) + * with_names - in the first line the header with column names * with_types - in the second line the header with column names */ RowInputFormatWithNamesAndTypes( const Block & header_, ReadBuffer & in_, const Params & params_, + bool is_binary_, bool with_names_, bool with_types_, const FormatSettings & format_settings_, @@ -51,6 +53,7 @@ private: bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; + bool is_binary; bool with_names; bool with_types; std::unique_ptr format_reader; diff --git a/tests/queries/0_stateless/02306_rowbinary_has_no_bom.reference b/tests/queries/0_stateless/02306_rowbinary_has_no_bom.reference new file mode 100644 index 00000000000..2e006a3fdd7 --- /dev/null +++ b/tests/queries/0_stateless/02306_rowbinary_has_no_bom.reference @@ -0,0 +1 @@ +1651760768976141295 diff --git a/tests/queries/0_stateless/02306_rowbinary_has_no_bom.sh b/tests/queries/0_stateless/02306_rowbinary_has_no_bom.sh new file mode 100755 index 00000000000..b784d7c3984 --- /dev/null +++ b/tests/queries/0_stateless/02306_rowbinary_has_no_bom.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +echo "DROP TABLE IF EXISTS table_with_uint64" | ${CLICKHOUSE_CURL} -d@- -sS "${CLICKHOUSE_URL}" +echo "CREATE TABLE table_with_uint64(no UInt64) ENGINE = MergeTree ORDER BY no" | ${CLICKHOUSE_CURL} -d@- -sS "${CLICKHOUSE_URL}" +echo -en '\xef\xbb\xbf\x00\xab\x3b\xec\x16' | ${CLICKHOUSE_CURL} --data-binary @- "${CLICKHOUSE_URL}&query=INSERT+INTO+table_with_uint64(no)+FORMAT+RowBinary" +echo "SELECT * FROM table_with_uint64" | ${CLICKHOUSE_CURL} -d@- -sS "${CLICKHOUSE_URL}" From 31e1e678366956ce6585a611baeb3dc53301641a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 06:25:32 +0000 Subject: [PATCH 062/101] Initialize ParallelReadBuffer after construction --- src/IO/ParallelReadBuffer.cpp | 13 +++++++++++++ src/IO/ParallelReadBuffer.h | 5 +++++ src/Storages/StorageS3.cpp | 5 ++++- src/Storages/StorageURL.cpp | 10 +++++----- 4 files changed, 27 insertions(+), 6 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 512f1c856b7..8d776a115a7 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -48,8 +48,15 @@ ParallelReadBuffer::ParallelReadBuffer( , max_working_readers(max_working_readers_) , schedule(std::move(schedule_)) , reader_factory(std::move(reader_factory_)) +{} + +void ParallelReadBuffer::initialize() { + if (initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ParallelReadBuffer is initialized twice"); + addReaders(); + initialized = true; } bool ParallelReadBuffer::addReaderToPool() @@ -76,6 +83,9 @@ void ParallelReadBuffer::addReaders() off_t ParallelReadBuffer::seek(off_t offset, int whence) { + if (!initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ParallelReadBuffer is not initialized"); + if (whence != SEEK_SET) throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); @@ -172,6 +182,9 @@ void ParallelReadBuffer::handleEmergencyStop() bool ParallelReadBuffer::nextImpl() { + if (!initialized) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ParallelReadBuffer is not initialized"); + if (all_completed) return false; diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index 83b978848f8..0568a5f0298 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -40,6 +40,9 @@ public: explicit ParallelReadBuffer(std::unique_ptr reader_factory_, CallbackRunner schedule_, size_t max_working_readers); + // some readers can throw exception during constructor call so we can't initialize ParallelReadBuffer there + void initialize(); + ~ParallelReadBuffer() override { finishAndWait(); } off_t seek(off_t off, int whence) override; @@ -96,6 +99,8 @@ private: off_t current_position{0}; bool all_completed{false}; + + bool initialized{false}; }; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 393ea0e24ff..39408e8ef36 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -479,7 +479,10 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k LOG_TRACE( log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size); - return std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num); + auto parallel_read_buffer + = std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num); + parallel_read_buffer->initialize(); + return parallel_read_buffer; } String StorageS3Source::getName() const diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 062241797e0..fdc6adb8c15 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -345,12 +345,12 @@ namespace /* use_external_buffer */ false, /* skip_url_not_found_error */ skip_url_not_found_error); + auto parallel_read_buffer = std::make_unique( + std::move(read_buffer_factory), threadPoolCallbackRunner(IOThreadPool::get()), download_threads); + parallel_read_buffer->initialize(); + return wrapReadBufferWithCompressionMethod( - std::make_unique( - std::move(read_buffer_factory), - threadPoolCallbackRunner(IOThreadPool::get()), - download_threads), - chooseCompressionMethod(request_uri.getPath(), compression_method)); + std::move(parallel_read_buffer), chooseCompressionMethod(request_uri.getPath(), compression_method)); } } catch (const Poco::Exception & e) From e55c2d07a3dd5d46793e9fdb1430ab29ded05efd Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 06:27:24 +0000 Subject: [PATCH 063/101] Revert "Enable only jepsen tests" This reverts commit 3e71a716f53862186a318c5357f01fb6e8eccb8b. --- .github/workflows/pull_request.yml | 3195 +++++++++++++++++++++++++++- 1 file changed, 3194 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 375d3e1cd6e..f6e9880d088 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -109,8 +109,196 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json + StyleCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{ runner.temp }}/style_check + EOF + - name: Download changed images + # even if artifact does not exist, e.g. on `do not test` label or failed Docker job + continue-on-error: true + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 style_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FastTest: + needs: DockerHubPush + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/fasttest + REPO_COPY=${{runner.temp}}/fasttest/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" + mkdir "$GITHUB_WORKSPACE" + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} + - name: Fast Test + run: | + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + CompatibilityCheck: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheck + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + SplitBuildSmokeTest: + needs: [BuilderDebSplitted] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/split_build_check + REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Split build check + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +######################################################################################### +#################################### ORDINARY BUILDS #################################### +######################################################################################### + BuilderDebRelease: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_release + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # for performance artifact + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" BuilderBinRelease: - needs: [DockerHubPush] + needs: [DockerHubPush, FastTest] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -154,9 +342,3014 @@ jobs: # shellcheck disable=SC2046 docker rm -f $(docker ps -a -q) ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + # BuilderBinGCC: + # needs: [DockerHubPush, FastTest] + # runs-on: [self-hosted, builder] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/build_check + # IMAGES_PATH=${{runner.temp}}/images_path + # REPO_COPY=${{runner.temp}}/build_check/ClickHouse + # CACHES_PATH=${{runner.temp}}/../ccaches + # BUILD_NAME=binary_gcc + # EOF + # - name: Download changed images + # uses: actions/download-artifact@v2 + # with: + # name: changed_images + # path: ${{ runner.temp }}/images_path + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # - name: Build + # run: | + # git -C "$GITHUB_WORKSPACE" submodule sync + # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + # - name: Upload build URLs to artifacts + # if: ${{ success() || failure() }} + # uses: actions/upload-artifact@v2 + # with: + # name: ${{ env.BUILD_URLS }} + # path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + # - name: Cleanup + # if: always() + # run: | + # # shellcheck disable=SC2046 + # docker kill $(docker ps -q) ||: + # # shellcheck disable=SC2046 + # docker rm -f $(docker ps -a -q) ||: + # sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # for performance artifact + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebAsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_asan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebUBsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_ubsan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebTsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_tsan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebMsan: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_msan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebDebug: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_debug + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +########################################################################################## +##################################### SPECIAL BUILDS ##################################### +########################################################################################## + BuilderDebSplitted: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_splitted + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinTidy: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_tidy + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwin: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinFreeBSD: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_freebsd + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwinAarch64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinPPC64: + needs: [DockerHubPush, FastTest] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_ppc64le + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +############################################################################################ +##################################### Docker images ####################################### +############################################################################################ + DockerServerImages: + needs: + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + - name: Check docker clickhouse/clickhouse-server building + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_server.py --release-type head --no-push + python3 docker_server.py --release-type head --no-push --no-ubuntu \ + --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################ +##################################### BUILD REPORTER ####################################### +############################################################################################ + BuilderReport: + needs: + - BuilderBinRelease + - BuilderDebAarch64 + - BuilderDebAsan + - BuilderDebDebug + - BuilderDebMsan + - BuilderDebRelease + - BuilderDebTsan + - BuilderDebUBsan + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + CHECK_NAME=ClickHouse build check (actions) + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + BuilderSpecialReport: + needs: + - BuilderBinAarch64 + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderBinFreeBSD + # - BuilderBinGCC + - BuilderBinPPC64 + - BuilderBinTidy + - BuilderDebSplitted + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check (actions) + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +########################### FUNCTIONAl STATELESS TESTS ####################################### +############################################################################################## + FunctionalStatelessTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, actions) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated, actions) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseWideParts: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_wide_parts + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, wide parts enabled, actions) + REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseS3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, s3 storage, actions) + REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (aarch64, actions) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (address, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan0: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan1: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan2: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug0: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug1: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug2: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_flaky_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests flaky check (address, actions) + REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + TestsBugfixCheck: + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/tests_bugfix_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Tests bugfix validate check (actions) + KILL_TIMEOUT=3600 + REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Bugfix test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + + TEMP_PATH="${TEMP_PATH}/integration" \ + REPORTS_PATH="${REPORTS_PATH}/integration" \ + python3 integration_test_check.py "Integration tests bugfix validate check" \ + --validate-bugfix --post-commit-status=file || echo 'ignore exit code' + + TEMP_PATH="${TEMP_PATH}/stateless" \ + REPORTS_PATH="${REPORTS_PATH}/stateless" \ + python3 functional_test_check.py "Stateless tests bugfix validate check" "$KILL_TIMEOUT" \ + --validate-bugfix --post-commit-status=file || echo 'ignore exit code' + + python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +############################ FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################## + FunctionalStatefulTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release, actions) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (aarch64, actions) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (address, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (thread, actions) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (memory, actions) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug, actions) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +######################################### STRESS TESTS ####################################### +############################################################################################## + StressTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (address, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestTsan: + needs: [BuilderDebTsan] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (thread, actions) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (memory, actions) + REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_undefined + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (undefined, actions) + REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (debug, actions) + REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### AST FUZZERS ############################################ +############################################################################################## + ASTFuzzerTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (ASan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (TSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestUBSan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (UBSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestMSan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (MSan, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (debug, actions) + REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +############################# INTEGRATION TESTS ############################################# +############################################################################################# + IntegrationTestsAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan3: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (thread, actions) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release, actions) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests flaky check (asan, actions) + REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +#################################### UNIT TESTS ############################################# +############################################################################################# + UnitTestsAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (asan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsReleaseClang: + needs: [BuilderBinRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-clang, actions) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + # UnitTestsReleaseGCC: + # needs: [BuilderBinGCC] + # runs-on: [self-hosted, fuzzer-unit-tester] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/unit_tests_asan + # REPORTS_PATH=${{runner.temp}}/reports_dir + # CHECK_NAME=Unit tests (release-gcc, actions) + # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + # EOF + # - name: Download json reports + # uses: actions/download-artifact@v2 + # with: + # path: ${{ env.REPORTS_PATH }} + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # - name: Unit test + # run: | + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" + # python3 unit_tests_check.py "$CHECK_NAME" + # - name: Cleanup + # if: always() + # run: | + # # shellcheck disable=SC2046 + # docker kill $(docker ps -q) ||: + # # shellcheck disable=SC2046 + # docker rm -f $(docker ps -a -q) ||: + # sudo rm -fr "$TEMP_PATH" + UnitTestsTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (tsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (msan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (ubsan, actions) + REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +#################################### PERFORMANCE TESTS ###################################### +############################################################################################# + PerformanceComparison0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparison1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparison2: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparison3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison (actions) + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + # shellcheck disable=SC2046 + docker kill $(docker ps -q) ||: + # shellcheck disable=SC2046 + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# Jepsen: needs: [BuilderBinRelease] uses: ./.github/workflows/jepsen.yml + + FinishCheck: + needs: + - StyleCheck + - DockerHubPush + - DockerServerImages + - CheckLabels + - BuilderReport + - FastTest + - FunctionalStatelessTestDebug0 + - FunctionalStatelessTestDebug1 + - FunctionalStatelessTestDebug2 + - FunctionalStatelessTestRelease + - FunctionalStatelessTestReleaseDatabaseReplicated0 + - FunctionalStatelessTestReleaseDatabaseReplicated1 + - FunctionalStatelessTestReleaseWideParts + - FunctionalStatelessTestAarch64 + - FunctionalStatelessTestAsan0 + - FunctionalStatelessTestAsan1 + - FunctionalStatelessTestTsan0 + - FunctionalStatelessTestTsan1 + - FunctionalStatelessTestTsan2 + - FunctionalStatelessTestMsan0 + - FunctionalStatelessTestMsan1 + - FunctionalStatelessTestMsan2 + - FunctionalStatelessTestUBsan + - FunctionalStatefulTestDebug + - FunctionalStatefulTestRelease + - FunctionalStatefulTestAarch64 + - FunctionalStatefulTestAsan + - FunctionalStatefulTestTsan + - FunctionalStatefulTestMsan + - FunctionalStatefulTestUBsan + - FunctionalStatelessTestReleaseS3 + - StressTestDebug + - StressTestAsan + - StressTestTsan + - StressTestMsan + - StressTestUBsan + - ASTFuzzerTestDebug + - ASTFuzzerTestAsan + - ASTFuzzerTestTsan + - ASTFuzzerTestMSan + - ASTFuzzerTestUBSan + - IntegrationTestsAsan0 + - IntegrationTestsAsan1 + - IntegrationTestsAsan2 + - IntegrationTestsRelease0 + - IntegrationTestsRelease1 + - IntegrationTestsTsan0 + - IntegrationTestsTsan1 + - IntegrationTestsTsan2 + - IntegrationTestsTsan3 + - PerformanceComparison0 + - PerformanceComparison1 + - PerformanceComparison2 + - PerformanceComparison3 + - UnitTestsAsan + - UnitTestsTsan + - UnitTestsMsan + - UnitTestsUBsan + - UnitTestsReleaseClang + - SplitBuildSmokeTest + - CompatibilityCheck + - IntegrationTestsFlakyCheck + - Jepsen + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 finish_check.py From df0a1d523ee82e25f4eee92ad1e9fc013b9c42ce Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 06:32:52 +0000 Subject: [PATCH 064/101] add comment for overwrite --- .../src/jepsen/clickhouse_keeper/db.clj | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj index fcc28a5a272..c354e36e430 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj @@ -13,6 +13,9 @@ (ns jepsen.control.scp) +;; We need to overwrite Jepsen's implementation of scp! because it +;; doesn't use strict-host-key-checking + (defn scp! "Runs an SCP command by shelling out. Takes a conn-spec (used for port, key, etc), a seq of sources, and a single destination, all as strings." From 249fe561f4373a3c381a8c8ffc21e66ab476119a Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Jun 2022 09:42:57 +0200 Subject: [PATCH 065/101] Fix build with -DENABLE_LIBRARIES=0 / -DENABLE_REPLXX=0 Replxx: When disabled via -DENABLE_LIBRARIES=0 or -DENABLE_REPLXX (the latter was undocumented) the build broke because replxx symbols were used since [0] in header LineReader.h. This header should in theory stay clean of replxx but doesn't for efficiency reasons. This change makes compilation of replxx mandatory. As replxx is quite small, I guess this is okay. (The alternative is to litter the code with ifdefs for non-replxx and a replxx paths.) [0] https://github.com/ClickHouse/ClickHouse/pull/33201 --- base/base/CMakeLists.txt | 5 +---- contrib/replxx-cmake/CMakeLists.txt | 7 ------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index adffb91625c..175a4836e64 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -17,15 +17,12 @@ set (SRCS sleep.cpp terminalColors.cpp errnoToString.cpp + ReplxxLineReader.cpp StringRef.cpp safeExit.cpp throwError.cpp ) -if (ENABLE_REPLXX) - list (APPEND SRCS ReplxxLineReader.cpp) -endif () - if (USE_DEBUG_HELPERS) get_target_property(MAGIC_ENUM_INCLUDE_DIR ch_contrib::magic_enum INTERFACE_INCLUDE_DIRECTORIES) # CMake generator expression will do insane quoting when it encounters special character like quotes, spaces, etc. diff --git a/contrib/replxx-cmake/CMakeLists.txt b/contrib/replxx-cmake/CMakeLists.txt index 8487ad520bc..c7cf6eb7687 100644 --- a/contrib/replxx-cmake/CMakeLists.txt +++ b/contrib/replxx-cmake/CMakeLists.txt @@ -1,10 +1,3 @@ -option (ENABLE_REPLXX "Enable replxx support" ${ENABLE_LIBRARIES}) - -if (NOT ENABLE_REPLXX) - message (STATUS "Not using replxx") - return() -endif() - set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/replxx") set(SRCS From 12871a43e1090f2a48f1070989286857f7183723 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Jun 2022 10:10:06 +0200 Subject: [PATCH 066/101] Cosmetics --- contrib/abseil-cpp-cmake/CMakeLists.txt | 3 -- contrib/amqpcpp-cmake/CMakeLists.txt | 1 + contrib/arrow-cmake/CMakeLists.txt | 2 +- contrib/bzip2-cmake/CMakeLists.txt | 6 +--- contrib/cassandra-cmake/CMakeLists.txt | 1 + contrib/cppkafka-cmake/CMakeLists.txt | 2 +- contrib/fastops-cmake/CMakeLists.txt | 2 +- contrib/fmtlib-cmake/CMakeLists.txt | 32 ++++++++++--------- contrib/h3-cmake/CMakeLists.txt | 34 ++++++++++----------- contrib/hive-metastore-cmake/CMakeLists.txt | 2 +- contrib/libcpuid-cmake/CMakeLists.txt | 2 +- contrib/libgsasl-cmake/CMakeLists.txt | 2 +- contrib/libuv-cmake/CMakeLists.txt | 1 + contrib/minizip-ng-cmake/CMakeLists.txt | 2 +- contrib/nanodbc-cmake/CMakeLists.txt | 4 +-- contrib/thrift-cmake/CMakeLists.txt | 2 +- 16 files changed, 48 insertions(+), 50 deletions(-) diff --git a/contrib/abseil-cpp-cmake/CMakeLists.txt b/contrib/abseil-cpp-cmake/CMakeLists.txt index 4fb02327d17..4c31ecfc599 100644 --- a/contrib/abseil-cpp-cmake/CMakeLists.txt +++ b/contrib/abseil-cpp-cmake/CMakeLists.txt @@ -1,7 +1,4 @@ set(ABSL_ROOT_DIR "${ClickHouse_SOURCE_DIR}/contrib/abseil-cpp") -if(NOT EXISTS "${ABSL_ROOT_DIR}/CMakeLists.txt") - message(FATAL_ERROR " submodule third_party/abseil-cpp is missing. To fix try run: \n git submodule update --init --recursive") -endif() set(BUILD_TESTING OFF) set(ABSL_PROPAGATE_CXX_STD ON) add_subdirectory("${ABSL_ROOT_DIR}" "${ClickHouse_BINARY_DIR}/contrib/abseil-cpp") diff --git a/contrib/amqpcpp-cmake/CMakeLists.txt b/contrib/amqpcpp-cmake/CMakeLists.txt index 6e655d3c255..e5c17c234e9 100644 --- a/contrib/amqpcpp-cmake/CMakeLists.txt +++ b/contrib/amqpcpp-cmake/CMakeLists.txt @@ -5,6 +5,7 @@ if (NOT ENABLE_AMQPCPP) return() endif() +# can be removed once libuv build on MacOS with GCC is possible if (NOT TARGET ch_contrib::uv) message(STATUS "Not using AMQP-CPP because libuv is disabled") return() diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index a4574493440..74bbb300fa5 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -20,7 +20,7 @@ endif() option (ENABLE_PARQUET "Enable parquet" ${ENABLE_PARQUET_DEFAULT}) if (NOT ENABLE_PARQUET) - message(STATUS "Building without Parquet support") + message(STATUS "Not using parquet") return() endif() diff --git a/contrib/bzip2-cmake/CMakeLists.txt b/contrib/bzip2-cmake/CMakeLists.txt index 2e01a624000..693d4c1663c 100644 --- a/contrib/bzip2-cmake/CMakeLists.txt +++ b/contrib/bzip2-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option(ENABLE_BZIP2 "Enable bzip2 compression support" ${ENABLE_LIBRARIES}) if (NOT ENABLE_BZIP2) - message (STATUS "bzip2 compression disabled") + message (STATUS "Not using bzip2") return() endif() @@ -26,8 +26,4 @@ configure_file ( add_library(_bzip2 ${SRCS}) add_library(ch_contrib::bzip2 ALIAS _bzip2) -# To avoid -Wreserved-id-macro we use SYSTEM: -# -# clickhouse/contrib/bzip2/bzlib.h:23:9: error: macro name is a reserved identifier [-Werror,-Wreserved-id-macro] -# #define _BZLIB_H target_include_directories(_bzip2 SYSTEM BEFORE PUBLIC "${BZIP2_SOURCE_DIR}" "${BZIP2_BINARY_DIR}") diff --git a/contrib/cassandra-cmake/CMakeLists.txt b/contrib/cassandra-cmake/CMakeLists.txt index 986ac438bb2..59ff908b63a 100644 --- a/contrib/cassandra-cmake/CMakeLists.txt +++ b/contrib/cassandra-cmake/CMakeLists.txt @@ -5,6 +5,7 @@ if (NOT ENABLE_CASSANDRA) return() endif() +# can be removed once libuv build on MacOS with GCC is possible if (NOT TARGET ch_contrib::uv) message(STATUS "Not using cassandra because libuv is disabled") return() diff --git a/contrib/cppkafka-cmake/CMakeLists.txt b/contrib/cppkafka-cmake/CMakeLists.txt index 87bf2356a80..fa1c52180e8 100644 --- a/contrib/cppkafka-cmake/CMakeLists.txt +++ b/contrib/cppkafka-cmake/CMakeLists.txt @@ -1,5 +1,5 @@ if (NOT ENABLE_KAFKA) - message(STATUS "Not using librdkafka (skip cppkafka)") + message(STATUS "Not using kafka") return() endif() diff --git a/contrib/fastops-cmake/CMakeLists.txt b/contrib/fastops-cmake/CMakeLists.txt index 17d6a7f5fcb..e9aa4803583 100644 --- a/contrib/fastops-cmake/CMakeLists.txt +++ b/contrib/fastops-cmake/CMakeLists.txt @@ -5,7 +5,7 @@ elseif(ENABLE_FASTOPS) endif() if(NOT ENABLE_FASTOPS) - message(STATUS "Not using fast vectorized mathematical functions library by Mikhail Parakhin") + message(STATUS "Not using fastops") return() endif() diff --git a/contrib/fmtlib-cmake/CMakeLists.txt b/contrib/fmtlib-cmake/CMakeLists.txt index fecec5f3e43..fe399ddc6e1 100644 --- a/contrib/fmtlib-cmake/CMakeLists.txt +++ b/contrib/fmtlib-cmake/CMakeLists.txt @@ -1,22 +1,24 @@ +set(FMT_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/fmtlib") + set (SRCS # NOTE: do not build module for now: # ../fmtlib/src/fmt.cc - ../fmtlib/src/format.cc - ../fmtlib/src/os.cc + ${FMT_SOURCE_DIR}/src/format.cc + ${FMT_SOURCE_DIR}/src/os.cc - ../fmtlib/include/fmt/args.h - ../fmtlib/include/fmt/chrono.h - ../fmtlib/include/fmt/color.h - ../fmtlib/include/fmt/compile.h - ../fmtlib/include/fmt/core.h - ../fmtlib/include/fmt/format.h - ../fmtlib/include/fmt/format-inl.h - ../fmtlib/include/fmt/locale.h - ../fmtlib/include/fmt/os.h - ../fmtlib/include/fmt/ostream.h - ../fmtlib/include/fmt/printf.h - ../fmtlib/include/fmt/ranges.h - ../fmtlib/include/fmt/xchar.h + ${FMT_SOURCE_DIR}/include/fmt/args.h + ${FMT_SOURCE_DIR}/include/fmt/chrono.h + ${FMT_SOURCE_DIR}/include/fmt/color.h + ${FMT_SOURCE_DIR}/include/fmt/compile.h + ${FMT_SOURCE_DIR}/include/fmt/core.h + ${FMT_SOURCE_DIR}/include/fmt/format.h + ${FMT_SOURCE_DIR}/include/fmt/format-inl.h + ${FMT_SOURCE_DIR}/include/fmt/locale.h + ${FMT_SOURCE_DIR}/include/fmt/os.h + ${FMT_SOURCE_DIR}/include/fmt/ostream.h + ${FMT_SOURCE_DIR}/include/fmt/printf.h + ${FMT_SOURCE_DIR}/include/fmt/ranges.h + ${FMT_SOURCE_DIR}/include/fmt/xchar.h ) add_library(_fmt ${SRCS}) diff --git a/contrib/h3-cmake/CMakeLists.txt b/contrib/h3-cmake/CMakeLists.txt index 984d1b1ae7c..869550224e6 100644 --- a/contrib/h3-cmake/CMakeLists.txt +++ b/contrib/h3-cmake/CMakeLists.txt @@ -9,23 +9,23 @@ set(H3_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/h3/src/h3lib") set(H3_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/h3/src/h3lib") set(SRCS -"${H3_SOURCE_DIR}/lib/algos.c" -"${H3_SOURCE_DIR}/lib/coordijk.c" -"${H3_SOURCE_DIR}/lib/bbox.c" -"${H3_SOURCE_DIR}/lib/polygon.c" -"${H3_SOURCE_DIR}/lib/h3Index.c" -"${H3_SOURCE_DIR}/lib/vec2d.c" -"${H3_SOURCE_DIR}/lib/vec3d.c" -"${H3_SOURCE_DIR}/lib/vertex.c" -"${H3_SOURCE_DIR}/lib/linkedGeo.c" -"${H3_SOURCE_DIR}/lib/localij.c" -"${H3_SOURCE_DIR}/lib/latLng.c" -"${H3_SOURCE_DIR}/lib/directedEdge.c" -"${H3_SOURCE_DIR}/lib/mathExtensions.c" -"${H3_SOURCE_DIR}/lib/iterators.c" -"${H3_SOURCE_DIR}/lib/vertexGraph.c" -"${H3_SOURCE_DIR}/lib/faceijk.c" -"${H3_SOURCE_DIR}/lib/baseCells.c" + "${H3_SOURCE_DIR}/lib/algos.c" + "${H3_SOURCE_DIR}/lib/coordijk.c" + "${H3_SOURCE_DIR}/lib/bbox.c" + "${H3_SOURCE_DIR}/lib/polygon.c" + "${H3_SOURCE_DIR}/lib/h3Index.c" + "${H3_SOURCE_DIR}/lib/vec2d.c" + "${H3_SOURCE_DIR}/lib/vec3d.c" + "${H3_SOURCE_DIR}/lib/vertex.c" + "${H3_SOURCE_DIR}/lib/linkedGeo.c" + "${H3_SOURCE_DIR}/lib/localij.c" + "${H3_SOURCE_DIR}/lib/latLng.c" + "${H3_SOURCE_DIR}/lib/directedEdge.c" + "${H3_SOURCE_DIR}/lib/mathExtensions.c" + "${H3_SOURCE_DIR}/lib/iterators.c" + "${H3_SOURCE_DIR}/lib/vertexGraph.c" + "${H3_SOURCE_DIR}/lib/faceijk.c" + "${H3_SOURCE_DIR}/lib/baseCells.c" ) configure_file("${H3_SOURCE_DIR}/include/h3api.h.in" "${H3_BINARY_DIR}/include/h3api.h") diff --git a/contrib/hive-metastore-cmake/CMakeLists.txt b/contrib/hive-metastore-cmake/CMakeLists.txt index 9069d46cea7..a5e16c739af 100644 --- a/contrib/hive-metastore-cmake/CMakeLists.txt +++ b/contrib/hive-metastore-cmake/CMakeLists.txt @@ -5,7 +5,7 @@ elseif(ENABLE_HIVE) endif() if (NOT ENABLE_HIVE) - message("Hive disabled") + message(STATUS "Not using hive") return() endif() diff --git a/contrib/libcpuid-cmake/CMakeLists.txt b/contrib/libcpuid-cmake/CMakeLists.txt index 1940b39b6aa..95f653c7ea2 100644 --- a/contrib/libcpuid-cmake/CMakeLists.txt +++ b/contrib/libcpuid-cmake/CMakeLists.txt @@ -6,7 +6,7 @@ elseif(ENABLE_CPUID) endif() if (NOT ENABLE_CPUID) - message("Not using cpuid") + message(STATUS "Not using cpuid") return() endif() diff --git a/contrib/libgsasl-cmake/CMakeLists.txt b/contrib/libgsasl-cmake/CMakeLists.txt index 4bb4ca9dc33..3cf087c2f4c 100644 --- a/contrib/libgsasl-cmake/CMakeLists.txt +++ b/contrib/libgsasl-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ option(ENABLE_GSASL_LIBRARY "Enable gsasl library" ${ENABLE_LIBRARIES}) if (NOT ENABLE_GSASL_LIBRARY) - message(STATUS "Not using gsasl library") + message(STATUS "Not using gsasl") return() endif() diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt index 45f6d8e2083..1a7714e47ce 100644 --- a/contrib/libuv-cmake/CMakeLists.txt +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -1,3 +1,4 @@ +# once fixed, please remove similar places in CMakeLists of libuv users (search "ch_contrib::uv") if (OS_DARWIN AND COMPILER_GCC) message (WARNING "libuv cannot be built with GCC in macOS due to a bug: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=93082") return() diff --git a/contrib/minizip-ng-cmake/CMakeLists.txt b/contrib/minizip-ng-cmake/CMakeLists.txt index 4aabbd3c9fb..043f0fc68f9 100644 --- a/contrib/minizip-ng-cmake/CMakeLists.txt +++ b/contrib/minizip-ng-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option(ENABLE_MINIZIP "Enable minizip-ng the zip manipulation library" ${ENABLE_LIBRARIES}) if (NOT ENABLE_MINIZIP) - message (STATUS "minizip-ng disabled") + message (STATUS "Not using minizip-ng") return() endif() diff --git a/contrib/nanodbc-cmake/CMakeLists.txt b/contrib/nanodbc-cmake/CMakeLists.txt index 9ed6c9525b6..7aacf5bed7e 100644 --- a/contrib/nanodbc-cmake/CMakeLists.txt +++ b/contrib/nanodbc-cmake/CMakeLists.txt @@ -2,12 +2,12 @@ if (NOT ENABLE_ODBC) return () endif () -set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/nanodbc") - if (NOT TARGET ch_contrib::unixodbc) message(FATAL_ERROR "Configuration error: unixodbc is not a target") endif() +set (LIBRARY_DIR "${ClickHouse_SOURCE_DIR}/contrib/nanodbc") + set (SRCS "${LIBRARY_DIR}/nanodbc/nanodbc.cpp" ) diff --git a/contrib/thrift-cmake/CMakeLists.txt b/contrib/thrift-cmake/CMakeLists.txt index 2a62a6fe7ab..6f94c1ebdc0 100644 --- a/contrib/thrift-cmake/CMakeLists.txt +++ b/contrib/thrift-cmake/CMakeLists.txt @@ -1,7 +1,7 @@ option(ENABLE_THRIFT "Enable Thrift" ${ENABLE_LIBRARIES}) if (NOT ENABLE_THRIFT) - message (STATUS "thrift disabled") + message (STATUS "Not using thrift") return() endif() From a4e037c728e3f8fbbc0e038d1173dbac4c9bde67 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Jun 2022 10:24:12 +0200 Subject: [PATCH 067/101] Remove unused M_LIBRARY link --- contrib/brotli-cmake/CMakeLists.txt | 3 --- contrib/h3-cmake/CMakeLists.txt | 3 --- contrib/libxml2-cmake/CMakeLists.txt | 3 --- contrib/s2geometry-cmake/CMakeLists.txt | 4 ---- 4 files changed, 13 deletions(-) diff --git a/contrib/brotli-cmake/CMakeLists.txt b/contrib/brotli-cmake/CMakeLists.txt index c81a6bf9076..b89e81ecda1 100644 --- a/contrib/brotli-cmake/CMakeLists.txt +++ b/contrib/brotli-cmake/CMakeLists.txt @@ -45,7 +45,4 @@ add_library(ch_contrib::brotli ALIAS _brotli) target_include_directories(_brotli SYSTEM BEFORE PUBLIC "${BROTLI_SOURCE_DIR}/include") -if(M_LIBRARY) - target_link_libraries(_brotli PRIVATE ${M_LIBRARY}) -endif() target_compile_definitions(_brotli PRIVATE BROTLI_BUILD_PORTABLE=1) diff --git a/contrib/h3-cmake/CMakeLists.txt b/contrib/h3-cmake/CMakeLists.txt index 869550224e6..c0c2162bd26 100644 --- a/contrib/h3-cmake/CMakeLists.txt +++ b/contrib/h3-cmake/CMakeLists.txt @@ -34,8 +34,5 @@ add_library(_h3 ${SRCS}) target_include_directories(_h3 SYSTEM PUBLIC "${H3_SOURCE_DIR}/include") target_include_directories(_h3 SYSTEM PUBLIC "${H3_BINARY_DIR}/include") target_compile_definitions(_h3 PRIVATE H3_HAVE_VLA) -if(M_LIBRARY) - target_link_libraries(_h3 PRIVATE ${M_LIBRARY}) -endif() add_library(ch_contrib::h3 ALIAS _h3) diff --git a/contrib/libxml2-cmake/CMakeLists.txt b/contrib/libxml2-cmake/CMakeLists.txt index e9c4641c161..a84936f8e3a 100644 --- a/contrib/libxml2-cmake/CMakeLists.txt +++ b/contrib/libxml2-cmake/CMakeLists.txt @@ -53,9 +53,6 @@ set(SRCS add_library(_libxml2 ${SRCS}) target_link_libraries(_libxml2 PRIVATE ch_contrib::zlib) -if(M_LIBRARY) - target_link_libraries(_libxml2 PRIVATE ${M_LIBRARY}) -endif() target_include_directories(_libxml2 BEFORE PUBLIC "${CMAKE_CURRENT_SOURCE_DIR}/linux_x86_64/include") target_include_directories(_libxml2 BEFORE PUBLIC "${LIBXML2_SOURCE_DIR}/include") diff --git a/contrib/s2geometry-cmake/CMakeLists.txt b/contrib/s2geometry-cmake/CMakeLists.txt index 49c80e45b18..102ceb0db3c 100644 --- a/contrib/s2geometry-cmake/CMakeLists.txt +++ b/contrib/s2geometry-cmake/CMakeLists.txt @@ -149,7 +149,3 @@ target_link_libraries(_s2 PRIVATE target_include_directories(_s2 SYSTEM BEFORE PUBLIC "${S2_SOURCE_DIR}/") target_include_directories(_s2 SYSTEM PUBLIC "${ABSL_SOURCE_DIR}") - -if(M_LIBRARY) - target_link_libraries(_s2 PRIVATE ${M_LIBRARY}) -endif() From 933f98a900a5dfe4ad06774133f81281b73780b5 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Jun 2022 10:47:11 +0200 Subject: [PATCH 068/101] Removed warning flags in contribs warnings are disabled for all contribs in contrib/CMakeLists.txt already --- contrib/amqpcpp-cmake/CMakeLists.txt | 15 --------------- contrib/avro-cmake/CMakeLists.txt | 8 -------- contrib/azure-cmake/CMakeLists.txt | 14 -------------- contrib/capnproto-cmake/CMakeLists.txt | 12 ++++-------- contrib/icu-cmake/CMakeLists.txt | 4 ---- contrib/jemalloc-cmake/CMakeLists.txt | 1 - contrib/libcpuid-cmake/CMakeLists.txt | 3 --- contrib/replxx-cmake/CMakeLists.txt | 5 ----- contrib/unixodbc-cmake/CMakeLists.txt | 10 +--------- 9 files changed, 5 insertions(+), 67 deletions(-) diff --git a/contrib/amqpcpp-cmake/CMakeLists.txt b/contrib/amqpcpp-cmake/CMakeLists.txt index e5c17c234e9..6f6a0188e6f 100644 --- a/contrib/amqpcpp-cmake/CMakeLists.txt +++ b/contrib/amqpcpp-cmake/CMakeLists.txt @@ -38,21 +38,6 @@ set (SRCS add_library(_amqp-cpp ${SRCS}) -target_compile_options (_amqp-cpp - PRIVATE - -Wno-old-style-cast - -Wno-inconsistent-missing-destructor-override - -Wno-deprecated - -Wno-unused-parameter - -Wno-shadow - -Wno-tautological-type-limit-compare - -Wno-extra-semi -# NOTE: disable all warnings at last because the warning: - # "conversion function converting 'XXX' to itself will never be used" - # doesn't have it's own diagnostic flag yet. - -w -) - target_include_directories (_amqp-cpp SYSTEM BEFORE PUBLIC "${LIBRARY_DIR}/include" "${LIBRARY_DIR}") target_link_libraries (_amqp-cpp PUBLIC OpenSSL::Crypto OpenSSL::SSL ch_contrib::uv) add_library (ch_contrib::amqp_cpp ALIAS _amqp-cpp) diff --git a/contrib/avro-cmake/CMakeLists.txt b/contrib/avro-cmake/CMakeLists.txt index c5bda41782d..25474650d0e 100644 --- a/contrib/avro-cmake/CMakeLists.txt +++ b/contrib/avro-cmake/CMakeLists.txt @@ -60,14 +60,6 @@ target_compile_definitions (_avrocpp PUBLIC SNAPPY_CODEC_AVAILABLE) target_include_directories (_avrocpp PRIVATE ${SNAPPY_INCLUDE_DIR}) target_link_libraries (_avrocpp PRIVATE ch_contrib::snappy) -if (COMPILER_GCC) - set (SUPPRESS_WARNINGS -Wno-non-virtual-dtor) -elseif (COMPILER_CLANG) - set (SUPPRESS_WARNINGS -Wno-non-virtual-dtor) -endif () - -target_compile_options(_avrocpp PRIVATE ${SUPPRESS_WARNINGS}) - # create a symlink to include headers with set(AVRO_INCLUDE_DIR "${CMAKE_CURRENT_BINARY_DIR}/include") ADD_CUSTOM_TARGET(avro_symlink_headers ALL diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 031d8dc9a0b..19f2940cbf0 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -52,20 +52,6 @@ include("${AZURE_DIR}/cmake-modules/AzureTransportAdapters.cmake") add_library(_azure_sdk ${AZURE_SDK_UNIFIED_SRC}) -if (COMPILER_CLANG) - target_compile_options(_azure_sdk PRIVATE - -Wno-deprecated-copy-dtor - -Wno-extra-semi - -Wno-suggest-destructor-override - -Wno-inconsistent-missing-destructor-override - -Wno-error=unknown-warning-option - ) - - if (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER_EQUAL 13) - target_compile_options(_azure_sdk PRIVATE -Wno-reserved-identifier) - endif() -endif() - # Originally, on Windows azure-core is built with bcrypt and crypt32 by default if (TARGET OpenSSL::SSL) target_link_libraries(_azure_sdk PRIVATE OpenSSL::Crypto OpenSSL::SSL) diff --git a/contrib/capnproto-cmake/CMakeLists.txt b/contrib/capnproto-cmake/CMakeLists.txt index 297b847cd58..e76268592ee 100644 --- a/contrib/capnproto-cmake/CMakeLists.txt +++ b/contrib/capnproto-cmake/CMakeLists.txt @@ -81,16 +81,12 @@ set (CAPNPC_SRCS add_library(_capnpc ${CAPNPC_SRCS}) target_link_libraries(_capnpc PUBLIC _capnp) -# The library has substandard code -if (COMPILER_GCC) - set (SUPPRESS_WARNINGS -w) -elseif (COMPILER_CLANG) - set (SUPPRESS_WARNINGS -w) +if (COMPILER_CLANG) set (CAPNP_PRIVATE_CXX_FLAGS -fno-char8_t) endif () -target_compile_options(_kj PRIVATE ${SUPPRESS_WARNINGS} ${CAPNP_PRIVATE_CXX_FLAGS}) -target_compile_options(_capnp PRIVATE ${SUPPRESS_WARNINGS} ${CAPNP_PRIVATE_CXX_FLAGS}) -target_compile_options(_capnpc PRIVATE ${SUPPRESS_WARNINGS} ${CAPNP_PRIVATE_CXX_FLAGS}) +target_compile_options(_kj PRIVATE ${CAPNP_PRIVATE_CXX_FLAGS}) +target_compile_options(_capnp PRIVATE ${CAPNP_PRIVATE_CXX_FLAGS}) +target_compile_options(_capnpc PRIVATE ${CAPNP_PRIVATE_CXX_FLAGS}) add_library(ch_contrib::capnp ALIAS _capnpc) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index 9c34228e2a0..ce82155218c 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -481,10 +481,6 @@ target_include_directories(_icui18n SYSTEM PUBLIC "${ICU_SOURCE_DIR}/i18n/") target_compile_definitions(_icuuc PRIVATE -DU_COMMON_IMPLEMENTATION) target_compile_definitions(_icui18n PRIVATE -DU_I18N_IMPLEMENTATION) -if (COMPILER_CLANG) - target_compile_options(_icudata PRIVATE -Wno-unused-command-line-argument) -endif () - add_library(_icu INTERFACE) target_link_libraries(_icu INTERFACE _icui18n _icuuc _icudata) add_library(ch_contrib::icu ALIAS _icu) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index c59b4da890b..fdb0fd0e8af 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -180,7 +180,6 @@ if (USE_UNWIND) target_link_libraries (_jemalloc PRIVATE unwind) endif () -target_compile_options(_jemalloc PRIVATE -Wno-redundant-decls) # for RTLD_NEXT target_compile_options(_jemalloc PRIVATE -D_GNU_SOURCE) diff --git a/contrib/libcpuid-cmake/CMakeLists.txt b/contrib/libcpuid-cmake/CMakeLists.txt index 95f653c7ea2..fd5af925c57 100644 --- a/contrib/libcpuid-cmake/CMakeLists.txt +++ b/contrib/libcpuid-cmake/CMakeLists.txt @@ -27,8 +27,5 @@ add_library (_cpuid ${SRCS}) target_include_directories (_cpuid SYSTEM PUBLIC "${LIBRARY_DIR}") target_compile_definitions (_cpuid PRIVATE VERSION="v0.4.1") -if (COMPILER_CLANG) - target_compile_options (_cpuid PRIVATE -Wno-reserved-id-macro) -endif () add_library(ch_contrib::cpuid ALIAS _cpuid) diff --git a/contrib/replxx-cmake/CMakeLists.txt b/contrib/replxx-cmake/CMakeLists.txt index c7cf6eb7687..95a19875621 100644 --- a/contrib/replxx-cmake/CMakeLists.txt +++ b/contrib/replxx-cmake/CMakeLists.txt @@ -15,9 +15,4 @@ set(SRCS add_library (_replxx ${SRCS}) target_include_directories(_replxx SYSTEM PUBLIC "${LIBRARY_DIR}/include") - -if (COMPILER_CLANG) - target_compile_options(_replxx PRIVATE -Wno-documentation) -endif () - add_library(ch_contrib::replxx ALIAS _replxx) diff --git a/contrib/unixodbc-cmake/CMakeLists.txt b/contrib/unixodbc-cmake/CMakeLists.txt index b594ead3ba0..3317654cd67 100644 --- a/contrib/unixodbc-cmake/CMakeLists.txt +++ b/contrib/unixodbc-cmake/CMakeLists.txt @@ -294,14 +294,6 @@ target_include_directories (_unixodbc "${LIBRARY_DIR}/include" ) target_compile_definitions (_unixodbc PRIVATE -DHAVE_CONFIG_H) -target_compile_options (_unixodbc - PRIVATE - -Wno-dangling-else - -Wno-parentheses - -Wno-misleading-indentation - -Wno-unknown-warning-option - -Wno-reserved-id-macro - -O2 -) +target_compile_options (_unixodbc PRIVATE -O2) # intended? add_library (ch_contrib::unixodbc ALIAS _unixodbc) From 393b97763a7879d5df91023b16e296a6a5c2492e Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Jun 2022 11:18:56 +0200 Subject: [PATCH 069/101] Make SSL a mandatory dependency for now - SSL is a dependency of too many libs + unit tests (via poco crypto which requires SSL) - optional SSL is desirable but right now, turning off SSL (via -DENABLE_LIBRARIES=0 or =DENABLE_SSL=0) breaks the build - therefore make SSL mandatory for now + add a TODO comment --- contrib/boringssl-cmake/CMakeLists.txt | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/boringssl-cmake/CMakeLists.txt b/contrib/boringssl-cmake/CMakeLists.txt index 180fb3874c1..faee2dfddb3 100644 --- a/contrib/boringssl-cmake/CMakeLists.txt +++ b/contrib/boringssl-cmake/CMakeLists.txt @@ -1,7 +1,12 @@ # Needed for: # - securely connecting to an external server, e.g. clickhouse-client --host ... --secure # - lots of thirdparty libraries -option(ENABLE_SSL "Enable ssl" ${ENABLE_LIBRARIES}) + +# Actually, so many 3rd party libraries + unit tests need SSL that we cannot disable it +# without breaking the build ... +option(ENABLE_SSL "Enable ssl" ON) # breaks if OFF +# TODO: Making SSL dependent on ENABLE_LIBRARIES is desirable but needs fixing dependent libs + tests. +# option(ENABLE_SSL "Enable ssl" ${ENABLE_LIBRARIES}) if(NOT ENABLE_SSL) message(STATUS "Not using openssl") From 600512cc08622b672a0876d14d3b61a6f011d6a3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 31 May 2022 09:15:59 +0200 Subject: [PATCH 070/101] Replace exceptions thrown for programming errors by asserts --- src/Functions/MatchImpl.h | 31 ++++++++++++------------------- 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 96ce0ca2eb0..78ce2627c35 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -18,8 +18,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_COLUMN; - extern const int LOGICAL_ERROR; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; } namespace impl @@ -112,16 +110,14 @@ struct MatchImpl const ColumnString::Chars & haystack_data, const ColumnString::Offsets & haystack_offsets, const String & needle, - const ColumnPtr & start_pos_, + [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res) { const size_t haystack_size = haystack_offsets.size(); - if (haystack_size != res.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks and results", name); + assert(haystack_size == res.size()); - if (start_pos_ != nullptr) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name); + assert(start_pos_ == nullptr); if (haystack_offsets.empty()) return; @@ -274,8 +270,7 @@ struct MatchImpl { const size_t haystack_size = haystack.size() / N; - if (haystack_size != res.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks and results", name); + assert(haystack_size == res.size()); if (haystack.empty()) return; @@ -433,16 +428,15 @@ struct MatchImpl const ColumnString::Offsets & haystack_offsets, const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offset, - const ColumnPtr & start_pos_, + [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res) { const size_t haystack_size = haystack_offsets.size(); - if (haystack_size != needle_offset.size() || haystack_size != res.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks, needles and results", name); + assert(haystack_size == needle_offset.size()); + assert(haystack_size == res.size()); - if (start_pos_ != nullptr) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name); + assert(start_pos_ == nullptr); if (haystack_offsets.empty()) return; @@ -547,16 +541,15 @@ struct MatchImpl size_t N, const ColumnString::Chars & needle_data, const ColumnString::Offsets & needle_offset, - const ColumnPtr & start_pos_, + [[maybe_unused]] const ColumnPtr & start_pos_, PaddedPODArray & res) { const size_t haystack_size = haystack.size()/N; - if (haystack_size != needle_offset.size() || haystack_size != res.size()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Function '{}' unexpectedly received a different number of haystacks, needles and results", name); + assert(haystack_size == needle_offset.size()); + assert(haystack_size == res.size()); - if (start_pos_ != nullptr) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function '{}' doesn't support start_pos argument", name); + assert(start_pos_ == nullptr); if (haystack.empty()) return; From 81318e07d642def5090753c34082e859f2a42a65 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 31 May 2022 09:29:04 +0200 Subject: [PATCH 071/101] Try to fix performance test results --- ...h_pattern_caching.xml => re2_regex_caching.xml} | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) rename tests/performance/{like_and_match_pattern_caching.xml => re2_regex_caching.xml} (88%) diff --git a/tests/performance/like_and_match_pattern_caching.xml b/tests/performance/re2_regex_caching.xml similarity index 88% rename from tests/performance/like_and_match_pattern_caching.xml rename to tests/performance/re2_regex_caching.xml index c0a8ec9442e..6edc83097ba 100644 --- a/tests/performance/like_and_match_pattern_caching.xml +++ b/tests/performance/re2_regex_caching.xml @@ -5,15 +5,15 @@ numbers - numbers_mt(2000000) + numbers_mt(1500000) needle_like - simple patterns, all unique + '%' || toString(number) || '_' - simple patterns, low distinctness (10 patterns) + '%' || toString(number % 10) || '_' @@ -40,23 +40,27 @@ select toString(number) as haystack, like(haystack, '%x_') from(select * from {numbers}) + format Null select toString(number) as haystack, match(haystack, '.*x.') from(select * from {numbers}) + format Null select toString(number) as haystack, {needle_like} as needle, like(haystack, needle) - from (select * from {numbers}); + from (select * from {numbers}) + format Null select toString(number) as haystack, {needle_match} as needle, match(haystack, needle) - from (select * from {numbers}); + from (select * from {numbers}) + format Null From d84a21aea50d8683426e6c56e2c6522e750282e8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 1 Jun 2022 12:57:19 +0200 Subject: [PATCH 072/101] Fixed tests --- tests/queries/0_stateless/02315_readonly_create_function.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02315_readonly_create_function.sh b/tests/queries/0_stateless/02315_readonly_create_function.sh index 03a4feb3038..07e97355883 100755 --- a/tests/queries/0_stateless/02315_readonly_create_function.sh +++ b/tests/queries/0_stateless/02315_readonly_create_function.sh @@ -1,4 +1,4 @@ -CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none +#!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 9aee3f3156c888a2ad816347898d30684edf66b2 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 1 Jun 2022 13:06:44 +0200 Subject: [PATCH 073/101] Fixed tests --- .../functions/test_function_config.xml | 2 +- .../test.py | 33 ++++++++++++++++--- .../user_scripts/input_nullable.py | 2 +- 3 files changed, 30 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml index dce7ab2eacd..5da2e854da8 100644 --- a/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml +++ b/tests/integration/test_executable_user_defined_function/functions/test_function_config.xml @@ -301,7 +301,7 @@ - executable + executable_pool test_function_nullable_pool_python String diff --git a/tests/integration/test_executable_user_defined_function/test.py b/tests/integration/test_executable_user_defined_function/test.py index 20beded2284..f48547a1437 100644 --- a/tests/integration/test_executable_user_defined_function/test.py +++ b/tests/integration/test_executable_user_defined_function/test.py @@ -229,14 +229,37 @@ def test_executable_function_sum_json_python(started_cluster): node.query("DROP TABLE test_table;") + def test_executable_function_input_nullable_python(started_cluster): skip_test_msan(node) - node.query("CREATE TABLE test_table_nullable (value Nullable(UInt64)) ENGINE=TinyLog;") + node.query( + "CREATE TABLE test_table_nullable (value Nullable(UInt64)) ENGINE=TinyLog;" + ) node.query("INSERT INTO test_table_nullable VALUES (0), (NULL), (2);") - assert(node.query("SELECT test_function_nullable_python(1), test_function_nullable_python(NULL)") == "Key 1\tKey Nullable\n") - assert(node.query("SELECT test_function_nullable_python(value) FROM test_table_nullable;") == "Key 0\nKey Nullable\nKey 2\n") + assert ( + node.query( + "SELECT test_function_nullable_python(1), test_function_nullable_python(NULL)" + ) + == "Key 1\tKey Nullable\n" + ) + assert ( + node.query( + "SELECT test_function_nullable_python(value) FROM test_table_nullable;" + ) + == "Key 0\nKey Nullable\nKey 2\n" + ) - assert(node.query("SELECT test_function_nullable_pool_python(1), test_function_nullable_pool_python(NULL)") == "Key 1\tKey Nullable\n") - assert(node.query("SELECT test_function_nullable_pool_python(value) FROM test_table_nullable;") == "Key 0\nKey Nullable\nKey 2\n") + assert ( + node.query( + "SELECT test_function_nullable_pool_python(1), test_function_nullable_pool_python(NULL)" + ) + == "Key 1\tKey Nullable\n" + ) + assert ( + node.query( + "SELECT test_function_nullable_pool_python(value) FROM test_table_nullable;" + ) + == "Key 0\nKey Nullable\nKey 2\n" + ) diff --git a/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py b/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py index f2f00a335bd..d1a825cf849 100755 --- a/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py +++ b/tests/integration/test_executable_user_defined_function/user_scripts/input_nullable.py @@ -4,7 +4,7 @@ import sys if __name__ == "__main__": for line in sys.stdin: - if (line == "\\N\n"): + if line == "\\N\n": print("Key Nullable", end="\n") else: print("Key " + line, end="") From ded1398565ab3bafb049f1f40d6461204b86f15a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 10:26:43 +0000 Subject: [PATCH 074/101] Fix intersect with const string --- src/Common/ColumnsHashing.h | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Common/ColumnsHashing.h b/src/Common/ColumnsHashing.h index c3a087c0a6e..e921f4fbf9a 100644 --- a/src/Common/ColumnsHashing.h +++ b/src/Common/ColumnsHashing.h @@ -6,9 +6,11 @@ #include #include #include +#include "Columns/IColumn.h" #include #include +#include #include #include @@ -83,8 +85,11 @@ struct HashMethodString HashMethodString(const ColumnRawPtrs & key_columns, const Sizes & /*key_sizes*/, const HashMethodContextPtr &) { - const IColumn & column = *key_columns[0]; - const ColumnString & column_string = assert_cast(column); + const IColumn * column = key_columns[0]; + if (isColumnConst(*column)) + column = &assert_cast(*column).getDataColumn(); + + const ColumnString & column_string = assert_cast(*column); offsets = column_string.getOffsets().data(); chars = column_string.getChars().data(); } From 6c31d06b2ecb3e9a3a0afa8e257a27abc7b0629a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 11:17:56 +0000 Subject: [PATCH 075/101] Add test for const string intersect --- tests/queries/0_stateless/02316_const_string_intersact.reference | 1 + tests/queries/0_stateless/02316_const_string_intersact.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02316_const_string_intersact.reference create mode 100644 tests/queries/0_stateless/02316_const_string_intersact.sql diff --git a/tests/queries/0_stateless/02316_const_string_intersact.reference b/tests/queries/0_stateless/02316_const_string_intersact.reference new file mode 100644 index 00000000000..957124d5fdd --- /dev/null +++ b/tests/queries/0_stateless/02316_const_string_intersact.reference @@ -0,0 +1 @@ +Play ClickHouse diff --git a/tests/queries/0_stateless/02316_const_string_intersact.sql b/tests/queries/0_stateless/02316_const_string_intersact.sql new file mode 100644 index 00000000000..ace3c8d03c5 --- /dev/null +++ b/tests/queries/0_stateless/02316_const_string_intersact.sql @@ -0,0 +1 @@ +SELECT 'Play ClickHouse' InterSect SELECT 'Play ClickHouse' From f49dd19e7ad2d7c7fa94459bce1e3226ba684da7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 11:43:58 +0000 Subject: [PATCH 076/101] Revert "Initialize ParallelReadBuffer after construction" This reverts commit 31e1e678366956ce6585a611baeb3dc53301641a. --- src/IO/ParallelReadBuffer.cpp | 13 ------------- src/IO/ParallelReadBuffer.h | 5 ----- src/Storages/StorageS3.cpp | 5 +---- src/Storages/StorageURL.cpp | 10 +++++----- 4 files changed, 6 insertions(+), 27 deletions(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 8d776a115a7..512f1c856b7 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -48,15 +48,8 @@ ParallelReadBuffer::ParallelReadBuffer( , max_working_readers(max_working_readers_) , schedule(std::move(schedule_)) , reader_factory(std::move(reader_factory_)) -{} - -void ParallelReadBuffer::initialize() { - if (initialized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ParallelReadBuffer is initialized twice"); - addReaders(); - initialized = true; } bool ParallelReadBuffer::addReaderToPool() @@ -83,9 +76,6 @@ void ParallelReadBuffer::addReaders() off_t ParallelReadBuffer::seek(off_t offset, int whence) { - if (!initialized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ParallelReadBuffer is not initialized"); - if (whence != SEEK_SET) throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); @@ -182,9 +172,6 @@ void ParallelReadBuffer::handleEmergencyStop() bool ParallelReadBuffer::nextImpl() { - if (!initialized) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ParallelReadBuffer is not initialized"); - if (all_completed) return false; diff --git a/src/IO/ParallelReadBuffer.h b/src/IO/ParallelReadBuffer.h index 0568a5f0298..83b978848f8 100644 --- a/src/IO/ParallelReadBuffer.h +++ b/src/IO/ParallelReadBuffer.h @@ -40,9 +40,6 @@ public: explicit ParallelReadBuffer(std::unique_ptr reader_factory_, CallbackRunner schedule_, size_t max_working_readers); - // some readers can throw exception during constructor call so we can't initialize ParallelReadBuffer there - void initialize(); - ~ParallelReadBuffer() override { finishAndWait(); } off_t seek(off_t off, int whence) override; @@ -99,8 +96,6 @@ private: off_t current_position{0}; bool all_completed{false}; - - bool initialized{false}; }; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 39408e8ef36..393ea0e24ff 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -479,10 +479,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k LOG_TRACE( log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size); - auto parallel_read_buffer - = std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num); - parallel_read_buffer->initialize(); - return parallel_read_buffer; + return std::make_unique(std::move(factory), threadPoolCallbackRunner(IOThreadPool::get()), download_thread_num); } String StorageS3Source::getName() const diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index fdc6adb8c15..062241797e0 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -345,12 +345,12 @@ namespace /* use_external_buffer */ false, /* skip_url_not_found_error */ skip_url_not_found_error); - auto parallel_read_buffer = std::make_unique( - std::move(read_buffer_factory), threadPoolCallbackRunner(IOThreadPool::get()), download_threads); - parallel_read_buffer->initialize(); - return wrapReadBufferWithCompressionMethod( - std::move(parallel_read_buffer), chooseCompressionMethod(request_uri.getPath(), compression_method)); + std::make_unique( + std::move(read_buffer_factory), + threadPoolCallbackRunner(IOThreadPool::get()), + download_threads), + chooseCompressionMethod(request_uri.getPath(), compression_method)); } } catch (const Poco::Exception & e) From 08c20be4d087dd379c481030caefdb8d98862f10 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 1 Jun 2022 11:51:01 +0000 Subject: [PATCH 077/101] Cleaner exception handling in ParallelReadBuffer --- src/IO/ParallelReadBuffer.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/IO/ParallelReadBuffer.cpp b/src/IO/ParallelReadBuffer.cpp index 512f1c856b7..926d10bda5b 100644 --- a/src/IO/ParallelReadBuffer.cpp +++ b/src/IO/ParallelReadBuffer.cpp @@ -49,7 +49,15 @@ ParallelReadBuffer::ParallelReadBuffer( , schedule(std::move(schedule_)) , reader_factory(std::move(reader_factory_)) { - addReaders(); + try + { + addReaders(); + } + catch (const Exception &) + { + finishAndWait(); + throw; + } } bool ParallelReadBuffer::addReaderToPool() From 5a1b873f7bcbd247a5809f6794d0517efdc34dbc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 1 Jun 2022 13:54:53 +0200 Subject: [PATCH 078/101] No need to checkout submodules/contribs recursively Also verified locally by building from a freshly cloned ClickHouse and "flat" checkout of submodules without recursion --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index abe263834ed..a6a09afc489 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -36,7 +36,7 @@ set_property(GLOBAL PROPERTY USE_FOLDERS ON) # Check that submodules are present if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/sysroot/README.md") - message (FATAL_ERROR "Submodules are not initialized. Run\n\tgit submodule update --init --recursive") + message (FATAL_ERROR "Submodules are not initialized. Run\n\tgit submodule update --init") endif () # Take care to add prlimit in command line before ccache, or else ccache thinks that From b62e4cec65389f1938bd2280041855ea7b744544 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 1 Jun 2022 12:39:16 +0000 Subject: [PATCH 079/101] Fix crash of FunctionHashID --- src/Functions/FunctionHashID.h | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Functions/FunctionHashID.h b/src/Functions/FunctionHashID.h index fbfb368bec7..30f08c96eca 100644 --- a/src/Functions/FunctionHashID.h +++ b/src/Functions/FunctionHashID.h @@ -51,9 +51,11 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + bool useDefaultImplementationForConstants() const override { return true; } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() < 1) + if (arguments.empty()) throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION, "Function {} expects at least one argument", getName()); const auto & id_col = arguments[0]; @@ -114,18 +116,16 @@ public: const auto & numcolumn = arguments[0].column; if (checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get()) - || checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get()) - || checkAndGetColumnConst(numcolumn.get()) || checkAndGetColumnConst(numcolumn.get()) - || checkAndGetColumnConst(numcolumn.get()) || checkAndGetColumnConst(numcolumn.get())) + || checkAndGetColumn(numcolumn.get()) || checkAndGetColumn(numcolumn.get())) { std::string salt; - UInt8 minLength = 0; + UInt8 min_length = 0; std::string alphabet; if (arguments.size() >= 4) { const auto & alphabetcolumn = arguments[3].column; - if (auto alpha_col = checkAndGetColumnConst(alphabetcolumn.get())) + if (const auto * alpha_col = checkAndGetColumnConst(alphabetcolumn.get())) { alphabet = alpha_col->getValue(); if (alphabet.find('\0') != std::string::npos) @@ -138,18 +138,18 @@ public: if (arguments.size() >= 3) { const auto & minlengthcolumn = arguments[2].column; - if (auto min_length_col = checkAndGetColumnConst(minlengthcolumn.get())) - minLength = min_length_col->getValue(); + if (const auto * min_length_col = checkAndGetColumnConst(minlengthcolumn.get())) + min_length = min_length_col->getValue(); } if (arguments.size() >= 2) { const auto & saltcolumn = arguments[1].column; - if (auto salt_col = checkAndGetColumnConst(saltcolumn.get())) + if (const auto * salt_col = checkAndGetColumnConst(saltcolumn.get())) salt = salt_col->getValue(); } - hashidsxx::Hashids hash(salt, minLength, alphabet); + hashidsxx::Hashids hash(salt, min_length, alphabet); auto col_res = ColumnString::create(); From ac10a6dc28e1211e213deed7e77d1688f8194a32 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 1 Jun 2022 12:41:36 +0000 Subject: [PATCH 080/101] update test --- tests/queries/0_stateless/02293_hashid.reference | 3 ++- tests/queries/0_stateless/02293_hashid.sql | 2 ++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02293_hashid.reference b/tests/queries/0_stateless/02293_hashid.reference index 9ae4cce3944..f36b1500288 100644 --- a/tests/queries/0_stateless/02293_hashid.reference +++ b/tests/queries/0_stateless/02293_hashid.reference @@ -8,4 +8,5 @@ 2 obmgndljgajpkeao 3 dldokmpjpgjgeanb 4 nkdlpgajngjnobme -YQrvD5XGvbx +xkOpDGxQpVB +jR diff --git a/tests/queries/0_stateless/02293_hashid.sql b/tests/queries/0_stateless/02293_hashid.sql index 145bd76ccbf..45aaefe7356 100644 --- a/tests/queries/0_stateless/02293_hashid.sql +++ b/tests/queries/0_stateless/02293_hashid.sql @@ -3,3 +3,5 @@ SET allow_experimental_hash_functions = 1; select number, hashid(number) from system.numbers limit 5; select number, hashid(number, 's3cr3t', 16, 'abcdefghijklmnop') from system.numbers limit 5; select hashid(1234567890123456, 's3cr3t'); + +SELECT hashid(1, hashid(2)); From 503d94f73be98b63318d82cfc9de0bb276cabb6d Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 1 Jun 2022 14:44:46 +0200 Subject: [PATCH 081/101] Update test.py --- tests/integration/test_mysql_database_engine/test.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 500d9176f4b..8626980a768 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -931,8 +931,10 @@ def test_predefined_connection_configuration(started_cluster): ) result = clickhouse_node.query("show create table test_database.test_table") - print(result) - assert(result.strip() == "CREATE TABLE test_database.test_table\\n(\\n `id` Int32\\n)\\nENGINE = MySQL(mysql1, table = \\'test_table\\')") + assert ( + result.strip() + == "CREATE TABLE test_database.test_table\\n(\\n `id` Int32\\n)\\nENGINE = MySQL(mysql1, table = \\'test_table\\')" + ) clickhouse_node.query("DROP DATABASE test_database") clickhouse_node.query_and_get_error( From 4413f0c7c12cd7a80c75c28e28ed2dac964e6686 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 1 Jun 2022 14:52:01 +0200 Subject: [PATCH 082/101] Fixed tests --- .../0_stateless/02315_readonly_create_function.reference | 2 +- tests/queries/0_stateless/02315_readonly_create_function.sh | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02315_readonly_create_function.reference b/tests/queries/0_stateless/02315_readonly_create_function.reference index d00491fd7e5..573541ac970 100644 --- a/tests/queries/0_stateless/02315_readonly_create_function.reference +++ b/tests/queries/0_stateless/02315_readonly_create_function.reference @@ -1 +1 @@ -1 +0 diff --git a/tests/queries/0_stateless/02315_readonly_create_function.sh b/tests/queries/0_stateless/02315_readonly_create_function.sh index 07e97355883..70e27e9ede9 100755 --- a/tests/queries/0_stateless/02315_readonly_create_function.sh +++ b/tests/queries/0_stateless/02315_readonly_create_function.sh @@ -4,4 +4,5 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --readonly 1 --query "CREATE FUNCTION test_function AS (x) -> x + 1;" 2>&1 | grep -c -F 'Code: 164' +$CLICKHOUSE_CLIENT --readonly 1 --query "CREATE FUNCTION test_function AS (x) -> x + 1;" 2>&1 | grep -q "Code: 164" +echo $?; From 6a5f5997cae8479d19bc21e4254c3e66f8dcf58c Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 1 Jun 2022 12:55:47 +0000 Subject: [PATCH 083/101] Add test 02315_pmj_union_ubsan_35857 --- .../02315_pmj_union_ubsan_35857.reference | 2 ++ .../02315_pmj_union_ubsan_35857.sql | 22 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02315_pmj_union_ubsan_35857.reference create mode 100644 tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql diff --git a/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.reference b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.reference new file mode 100644 index 00000000000..96e34d5a44c --- /dev/null +++ b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.reference @@ -0,0 +1,2 @@ +\N +\N diff --git a/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql new file mode 100644 index 00000000000..38f1d2e1b4e --- /dev/null +++ b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql @@ -0,0 +1,22 @@ +SET join_algorithm = 'partial_merge'; + +SELECT NULL +FROM +( + SELECT + NULL, + 1 AS a, + 0 :: Nullable(UInt8) AS c + UNION ALL + SELECT + NULL, + 65536, + NULL +) AS js1 +ALL LEFT JOIN +( + SELECT 2 :: Nullable(UInt8) AS a +) AS js2 +USING (a) +ORDER BY c +; From 3a824ef9a45d7231b9243e21cb6be8f5712edb0d Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 1 Jun 2022 16:00:30 +0200 Subject: [PATCH 084/101] Add no-backward-compatibility-check to 02315_pmj_union_ubsan_35857 --- tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql index 38f1d2e1b4e..47b47101a79 100644 --- a/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql +++ b/tests/queries/0_stateless/02315_pmj_union_ubsan_35857.sql @@ -1,3 +1,5 @@ +-- Tags: no-backward-compatibility-check + SET join_algorithm = 'partial_merge'; SELECT NULL From 79576fa08cf0ce81f200acc901455c54f2622df7 Mon Sep 17 00:00:00 2001 From: bkuschel Date: Wed, 1 Jun 2022 07:41:27 -0700 Subject: [PATCH 085/101] Use new submodule sync --- .github/workflows/nightly.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index b2ddd87d173..3ebf58b858a 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -81,7 +81,6 @@ jobs: cat >> "$GITHUB_ENV" << 'EOF' BUILD_NAME=coverity CACHES_PATH=${{runner.temp}}/../ccaches - CHECK_NAME=ClickHouse build check (actions) IMAGES_PATH=${{runner.temp}}/images_path REPO_COPY=${{runner.temp}}/build_check/ClickHouse TEMP_PATH=${{runner.temp}}/build_check @@ -99,13 +98,15 @@ jobs: id: coverity-checkout uses: actions/checkout@v2 with: - submodules: 'true' + fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$CHECK_NAME" "$BUILD_NAME" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - name: Upload Coverity Analysis if: ${{ success() || failure() }} run: | From 2626a496167d42fa5953bbce51b8386ef11961ee Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 4 May 2022 20:16:42 +0300 Subject: [PATCH 086/101] FR: Expose what triggered the merge in system.part_log #26255 --- src/Interpreters/PartLog.cpp | 31 ++++++++++++++++ src/Interpreters/PartLog.h | 15 ++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 4 ++ .../02293_part_log_has_merge_reason.reference | 1 + .../02293_part_log_has_merge_reason.sql | 37 +++++++++++++++++++ 5 files changed, 88 insertions(+) create mode 100644 tests/queries/0_stateless/02293_part_log_has_merge_reason.reference create mode 100644 tests/queries/0_stateless/02293_part_log_has_merge_reason.sql diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 6d57f6b7045..274fc7384ab 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -16,6 +16,25 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType merge_type) { + switch (merge_type) + { + case MergeType::REGULAR: + return REGULAR_MERGE; + case MergeType::TTL_DELETE: + return TTL_DELETE_MERGE; + case MergeType::TTL_RECOMPRESS: + return TTL_RECOMPRESS_MERGE; + } + + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); +} + NamesAndTypesList PartLogElement::getNamesAndTypes() { auto event_type_datatype = std::make_shared( @@ -30,11 +49,22 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() } ); + auto merge_reason_datatype = std::make_shared( + DataTypeEnum8::Values + { + {"NotAMerge", static_cast(NOT_A_MERGE)}, + {"RegularMerge", static_cast(REGULAR_MERGE)}, + {"TTLDeleteMerge", static_cast(TTL_DELETE_MERGE)}, + {"TTLRecompressMerge", static_cast(TTL_RECOMPRESS_MERGE)}, + } + ); + ColumnsWithTypeAndName columns_with_type_and_name; return { {"query_id", std::make_shared()}, {"event_type", std::move(event_type_datatype)}, + {"merge_reason", std::move(merge_reason_datatype)}, {"event_date", std::make_shared()}, {"event_time", std::make_shared()}, @@ -72,6 +102,7 @@ void PartLogElement::appendToBlock(MutableColumns & columns) const columns[i++]->insert(query_id); columns[i++]->insert(event_type); + columns[i++]->insert(merge_reason); columns[i++]->insert(DateLUT::instance().toDayNum(event_time).toUnderType()); columns[i++]->insert(event_time); columns[i++]->insert(event_time_microseconds); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 470dce09fa0..16a7e37ee9d 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -21,9 +22,22 @@ struct PartLogElement MOVE_PART = 6, }; + enum MergeReasonType + { + /// merge_reason is relevant only for event_type = 'MERGE_PARTS', in other cases it is NOT_A_MERGE + NOT_A_MERGE = 1, + /// Just regular merge + REGULAR_MERGE = 2, + /// Merge assigned to delete some data from parts (with TTLMergeSelector) + TTL_DELETE_MERGE = 3, + /// Merge with recompression + TTL_RECOMPRESS_MERGE = 4, + }; + String query_id; Type event_type = NEW_PART; + MergeReasonType merge_reason = NOT_A_MERGE; time_t event_time = 0; Decimal64 event_time_microseconds = 0; @@ -57,6 +71,7 @@ struct PartLogElement static std::string name() { return "PartLog"; } + static MergeReasonType getMergeReasonType(MergeType merge_type); static NamesAndTypesList getNamesAndTypes(); static NamesAndAliases getNamesAndAliases() { return {}; } void appendToBlock(MutableColumns & columns) const; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4eb4049be60..c4c99e66873 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6176,6 +6176,10 @@ try part_log_elem.event_type = type; + if (part_log_elem.event_type == PartLogElement::MERGE_PARTS) + if (merge_entry) + part_log_elem.merge_reason = PartLogElement::getMergeReasonType((*merge_entry)->merge_type); + part_log_elem.error = static_cast(execution_status.code); part_log_elem.exception = execution_status.message; diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql new file mode 100644 index 00000000000..db1f4c26af4 --- /dev/null +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS t_part_log_has_merge_type_table; + +CREATE TABLE t_part_log_has_merge_type_table +( + event_time DateTime, + UserID UInt64, + Comment String +) +ENGINE = MergeTree() +ORDER BY tuple() +TTL event_time + toIntervalMonth(3) +SETTINGS min_bytes_for_wide_part = 0, merge_with_ttl_timeout = 1; + +INSERT INTO t_part_log_has_merge_type_table VALUES (now(), 1, 'username1'); +INSERT INTO t_part_log_has_merge_type_table VALUES (now() - INTERVAL 4 MONTH, 2, 'username2'); + +OPTIMIZE TABLE t_part_log_has_merge_type_table FINAL; + +SYSTEM FLUSH LOGS; + +SELECT count(*) +FROM +( + SELECT + metadata_modification_time, + event_time + FROM system.tables AS l + INNER JOIN system.part_log AS r + ON l.name = r.table + WHERE (l.database = currentDatabase()) AND + (l.name = 't_part_log_has_merge_type_table') AND + (r.event_type = 'MergeParts') AND + (r.merge_reason = 'TTLDeleteMerge') +) +WHERE (metadata_modification_time <= event_time); + +DROP TABLE t_part_log_has_merge_type_table; From 16dc3ed97d29a17d34d9b8090ecbb070ebab3424 Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 4 May 2022 20:16:42 +0300 Subject: [PATCH 087/101] FR: Expose what triggered the merge in system.part_log #26255 --- docs/en/operations/system-tables/part_log.md | 6 +++++ src/Interpreters/PartLog.cpp | 9 ++++--- .../02293_part_log_has_merge_reason.reference | 2 +- .../02293_part_log_has_merge_reason.sql | 26 +++++++------------ 4 files changed, 21 insertions(+), 22 deletions(-) diff --git a/docs/en/operations/system-tables/part_log.md b/docs/en/operations/system-tables/part_log.md index 00eaca23862..1b567367c97 100644 --- a/docs/en/operations/system-tables/part_log.md +++ b/docs/en/operations/system-tables/part_log.md @@ -14,6 +14,11 @@ The `system.part_log` table contains the following columns: - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../../sql-reference/statements/alter/partition.md#alter_detach-partition). - `MUTATE_PART` — Mutating of a data part. - `MOVE_PART` — Moving the data part from the one disk to another one. +- `merge_reason` ([Enum8](../../sql-reference/data-types/enum.md)) — The reason for the event with type `MERGE_PARTS`. Can have one of the following values: + - `NOT_A_MERGE` — The current event has the type other than `MERGE_PARTS`. + - `REGULAR_MERGE` — Some regular merge. + - `TTL_DELETE_MERGE` — Cleaning up expired data. + - `TTL_RECOMPRESS_MERGE` — Recompressing data part with the. - `event_date` ([Date](../../sql-reference/data-types/date.md)) — Event date. - `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Event time. - `event_time_microseconds` ([DateTime64](../../sql-reference/data-types/datetime64.md)) — Event time with microseconds precision. @@ -46,6 +51,7 @@ Row 1: ────── query_id: 983ad9c7-28d5-4ae1-844e-603116b7de31 event_type: NewPart +merge_reason: NotAMerge event_date: 2021-02-02 event_time: 2021-02-02 11:14:28 event_time_microseconds: 2021-02-02 11:14:28.861919 diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 274fc7384ab..13b74f3d00a 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -21,14 +21,15 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType merge_type) { +PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType merge_type) +{ switch (merge_type) { - case MergeType::REGULAR: + case MergeType::Regular: return REGULAR_MERGE; - case MergeType::TTL_DELETE: + case MergeType::TTLDelete: return TTL_DELETE_MERGE; - case MergeType::TTL_RECOMPRESS: + case MergeType::TTLRecompress: return TTL_RECOMPRESS_MERGE; } diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference index d00491fd7e5..220107cf15b 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference @@ -1 +1 @@ -1 +MergeParts TTLDeleteMerge diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql index db1f4c26af4..7ef86354e71 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql @@ -8,30 +8,22 @@ CREATE TABLE t_part_log_has_merge_type_table ) ENGINE = MergeTree() ORDER BY tuple() -TTL event_time + toIntervalMonth(3) -SETTINGS min_bytes_for_wide_part = 0, merge_with_ttl_timeout = 1; +SETTINGS min_bytes_for_wide_part = 0, materialize_ttl_recalculate_only = true; INSERT INTO t_part_log_has_merge_type_table VALUES (now(), 1, 'username1'); INSERT INTO t_part_log_has_merge_type_table VALUES (now() - INTERVAL 4 MONTH, 2, 'username2'); +ALTER TABLE t_part_log_has_merge_type_table + MODIFY TTL event_time + INTERVAL 3 MONTH; + OPTIMIZE TABLE t_part_log_has_merge_type_table FINAL; SYSTEM FLUSH LOGS; -SELECT count(*) -FROM -( - SELECT - metadata_modification_time, - event_time - FROM system.tables AS l - INNER JOIN system.part_log AS r - ON l.name = r.table - WHERE (l.database = currentDatabase()) AND - (l.name = 't_part_log_has_merge_type_table') AND - (r.event_type = 'MergeParts') AND - (r.merge_reason = 'TTLDeleteMerge') -) -WHERE (metadata_modification_time <= event_time); +SELECT + event_type, + merge_reason +FROM system.part_log +WHERE (table = 't_part_log_has_merge_type_table') AND (merge_reason = 'TTLDeleteMerge'); DROP TABLE t_part_log_has_merge_type_table; From b3b3d7a45950d04d630e9930b7da2c331f604e4a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 1 Jun 2022 16:57:28 +0200 Subject: [PATCH 088/101] Fix test --- .../02293_part_log_has_merge_reason.sql | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql index 7ef86354e71..002bc1f37dd 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.sql @@ -8,14 +8,12 @@ CREATE TABLE t_part_log_has_merge_type_table ) ENGINE = MergeTree() ORDER BY tuple() +TTL event_time + INTERVAL 3 MONTH SETTINGS min_bytes_for_wide_part = 0, materialize_ttl_recalculate_only = true; INSERT INTO t_part_log_has_merge_type_table VALUES (now(), 1, 'username1'); INSERT INTO t_part_log_has_merge_type_table VALUES (now() - INTERVAL 4 MONTH, 2, 'username2'); -ALTER TABLE t_part_log_has_merge_type_table - MODIFY TTL event_time + INTERVAL 3 MONTH; - OPTIMIZE TABLE t_part_log_has_merge_type_table FINAL; SYSTEM FLUSH LOGS; @@ -23,7 +21,13 @@ SYSTEM FLUSH LOGS; SELECT event_type, merge_reason -FROM system.part_log -WHERE (table = 't_part_log_has_merge_type_table') AND (merge_reason = 'TTLDeleteMerge'); +FROM + system.part_log +WHERE + table = 't_part_log_has_merge_type_table' + AND + merge_reason = 'TTLDeleteMerge' + AND + database = currentDatabase(); DROP TABLE t_part_log_has_merge_type_table; From 7ef02a2e440b8b6db53745a327d4414e7a68f9ff Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 1 Jun 2022 15:32:33 +0000 Subject: [PATCH 089/101] Fix possible logical error in values table function --- src/Interpreters/evaluateConstantExpression.cpp | 14 ++++++++++++-- .../02316_values_table_func_bug.reference | 2 ++ .../0_stateless/02316_values_table_func_bug.sql | 2 ++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02316_values_table_func_bug.reference create mode 100644 tests/queries/0_stateless/02316_values_table_func_bug.sql diff --git a/src/Interpreters/evaluateConstantExpression.cpp b/src/Interpreters/evaluateConstantExpression.cpp index f5ad0337629..e9110d00128 100644 --- a/src/Interpreters/evaluateConstantExpression.cpp +++ b/src/Interpreters/evaluateConstantExpression.cpp @@ -31,11 +31,21 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +static std::pair> getFieldAndDataTypeFromLiteral(ASTLiteral * literal) +{ + auto type = applyVisitor(FieldToDataType(), literal->value); + /// In case of Array field nested fields can have different types. + /// Example: Array [1, 2.3] will have 2 fields with types UInt64 and Float64 + /// when result type is Array(Float64). + /// So, we need to convert this field to the result type. + Field res = convertFieldToType(literal->value, *type); + return {res, type}; +} std::pair> evaluateConstantExpression(const ASTPtr & node, ContextPtr context) { if (ASTLiteral * literal = node->as()) - return std::make_pair(literal->value, applyVisitor(FieldToDataType(), literal->value)); + return getFieldAndDataTypeFromLiteral(literal); NamesAndTypesList source_columns = {{ "_dummy", std::make_shared() }}; @@ -63,7 +73,7 @@ std::pair> evaluateConstantExpression(co /// AST potentially could be transformed to literal during TreeRewriter analyze. /// For example if we have SQL user defined function that return literal AS subquery. if (ASTLiteral * literal = ast->as()) - return std::make_pair(literal->value, applyVisitor(FieldToDataType(), literal->value)); + return getFieldAndDataTypeFromLiteral(literal); ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(ast, syntax_result, context).getConstActions(); diff --git a/tests/queries/0_stateless/02316_values_table_func_bug.reference b/tests/queries/0_stateless/02316_values_table_func_bug.reference new file mode 100644 index 00000000000..63f5d8d96c8 --- /dev/null +++ b/tests/queries/0_stateless/02316_values_table_func_bug.reference @@ -0,0 +1,2 @@ +[1,2.2] +[[1,2,3],[1.1,2.2,3.3]] diff --git a/tests/queries/0_stateless/02316_values_table_func_bug.sql b/tests/queries/0_stateless/02316_values_table_func_bug.sql new file mode 100644 index 00000000000..7c66cf125e1 --- /dev/null +++ b/tests/queries/0_stateless/02316_values_table_func_bug.sql @@ -0,0 +1,2 @@ +select * from values([1, 2.2]); +select * from values([[1, 2, 3], [1.1, 2.2, 3.3]]); From 1ef48c3a4afe1c57ad14e678efc5ab3958dc010e Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 1 Jun 2022 15:42:12 +0000 Subject: [PATCH 090/101] turn on setting output_format_json_named_tuples_as_objects by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c241955b268..320386c9bfe 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -695,7 +695,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ \ M(Bool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ - M(Bool, output_format_json_named_tuples_as_objects, false, "Serialize named tuple columns as JSON objects.", 0) \ + M(Bool, output_format_json_named_tuples_as_objects, true, "Serialize named tuple columns as JSON objects.", 0) \ M(Bool, output_format_json_array_of_rows, false, "Output a JSON array of all rows in JSONEachRow(Compact) format.", 0) \ \ M(UInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ From 895a96de95e5b6346cce16f18e7915bd9234ad1b Mon Sep 17 00:00:00 2001 From: bkuschel Date: Wed, 1 Jun 2022 09:44:37 -0700 Subject: [PATCH 091/101] Remove recursive --- .github/workflows/nightly.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 3ebf58b858a..e712ada1551 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -101,8 +101,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" From 4abfd54dd62f062ab9593a2f297a30a7d8b174cf Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 1 Jun 2022 16:53:37 +0000 Subject: [PATCH 092/101] Fix possible segfault in schema inference --- src/Formats/ReadSchemaUtils.cpp | 21 +++++++++++-------- ...18_template_schema_inference_bug.reference | 0 .../02318_template_schema_inference_bug.sql | 2 ++ 3 files changed, 14 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02318_template_schema_inference_bug.reference create mode 100755 tests/queries/0_stateless/02318_template_schema_inference_bug.sql diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 035546031d8..11a91bd50dc 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -100,18 +100,21 @@ ColumnsDescription readSchemaFromFormat( catch (...) { auto exception_message = getCurrentExceptionMessage(false); - size_t rows_read = schema_reader->getNumRowsRead(); - assert(rows_read <= max_rows_to_read); - max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) + if (schema_reader) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; - if (iterations > 1) + size_t rows_read = schema_reader->getNumRowsRead(); + assert(rows_read <= max_rows_to_read); + max_rows_to_read -= schema_reader->getNumRowsRead(); + if (rows_read != 0 && max_rows_to_read == 0) { - exception_messages += "\n" + exception_message; - break; + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + if (iterations > 1) + { + exception_messages += "\n" + exception_message; + break; + } + retry = false; } - retry = false; } if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) diff --git a/tests/queries/0_stateless/02318_template_schema_inference_bug.reference b/tests/queries/0_stateless/02318_template_schema_inference_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02318_template_schema_inference_bug.sql b/tests/queries/0_stateless/02318_template_schema_inference_bug.sql new file mode 100755 index 00000000000..42646013dd5 --- /dev/null +++ b/tests/queries/0_stateless/02318_template_schema_inference_bug.sql @@ -0,0 +1,2 @@ +insert into function file(data_02318.tsv) select * from numbers(10); +desc file('data_02318.tsv', 'Template') SETTINGS format_template_row='nonexist', format_template_resultset='nonexist'; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} From 663261673381d392a6a00753c165de84f5629827 Mon Sep 17 00:00:00 2001 From: lthaooo Date: Thu, 2 Jun 2022 03:09:53 +0800 Subject: [PATCH 093/101] Fix TTL merge scheduling bug (#36387) --- src/Storages/MergeTree/BackgroundJobsAssignee.cpp | 3 ++- src/Storages/MergeTree/BackgroundJobsAssignee.h | 2 +- src/Storages/MergeTree/MergeList.h | 5 +++++ src/Storages/StorageMergeTree.cpp | 8 ++++++-- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp index 4dc15d6e794..81445f40ed6 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.cpp +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.cpp @@ -50,10 +50,11 @@ void BackgroundJobsAssignee::postpone() } -void BackgroundJobsAssignee::scheduleMergeMutateTask(ExecutableTaskPtr merge_task) +bool BackgroundJobsAssignee::scheduleMergeMutateTask(ExecutableTaskPtr merge_task) { bool res = getContext()->getMergeMutateExecutor()->trySchedule(merge_task); res ? trigger() : postpone(); + return res; } diff --git a/src/Storages/MergeTree/BackgroundJobsAssignee.h b/src/Storages/MergeTree/BackgroundJobsAssignee.h index e6c5845c657..db93b5f710b 100644 --- a/src/Storages/MergeTree/BackgroundJobsAssignee.h +++ b/src/Storages/MergeTree/BackgroundJobsAssignee.h @@ -66,7 +66,7 @@ public: void postpone(); void finish(); - void scheduleMergeMutateTask(ExecutableTaskPtr merge_task); + bool scheduleMergeMutateTask(ExecutableTaskPtr merge_task); void scheduleFetchTask(ExecutableTaskPtr fetch_task); void scheduleMoveTask(ExecutableTaskPtr move_task); void scheduleCommonTask(ExecutableTaskPtr common_task, bool need_trigger); diff --git a/src/Storages/MergeTree/MergeList.h b/src/Storages/MergeTree/MergeList.h index a944779ad44..ac1db503d9b 100644 --- a/src/Storages/MergeTree/MergeList.h +++ b/src/Storages/MergeTree/MergeList.h @@ -197,6 +197,11 @@ public: ++merges_with_ttl_counter; } + void cancelMergeWithTTL() + { + --merges_with_ttl_counter; + } + size_t getMergesWithTTLCount() const { return merges_with_ttl_counter; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 2cb62801fd5..81b61909228 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -1168,8 +1168,12 @@ bool StorageMergeTree::scheduleDataProcessingJob(BackgroundJobsAssignee & assign { auto task = std::make_shared(*this, metadata_snapshot, false, Names{}, merge_entry, share_lock, common_assignee_trigger); task->setCurrentTransaction(std::move(transaction_for_merge), std::move(txn)); - assignee.scheduleMergeMutateTask(task); - return true; + bool scheduled = assignee.scheduleMergeMutateTask(task); + /// The problem that we already booked a slot for TTL merge, but a merge list entry will be created only in a prepare method + /// in MergePlainMergeTreeTask. So, this slot will never be freed. + if (!scheduled && isTTLMergeType(merge_entry->future_part->merge_type)) + getContext()->getMergeList().cancelMergeWithTTL(); + return scheduled; } if (mutate_entry) { From ec6e413f0bae58b2dd0cebb5babd1182a0f9d60e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 1 Jun 2022 23:00:49 +0200 Subject: [PATCH 094/101] Fixed runtime check for AVX512F --- src/Common/CpuId.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CpuId.h b/src/Common/CpuId.h index 5037c687943..167fa22faf6 100644 --- a/src/Common/CpuId.h +++ b/src/Common/CpuId.h @@ -221,7 +221,7 @@ bool haveAVX512F() noexcept && (our_xgetbv(0) & 6u) == 6u // XMM state and YMM state are enabled by OS && ((our_xgetbv(0) >> 5) & 7u) == 7u // ZMM state is enabled by OS && CpuInfo(0x0).registers.eax >= 0x7 // leaf 7 is present - && ((CpuInfo(0x7).registers.ebx >> 16) & 1u); // AVX512F bit + && ((CpuInfo(0x7, 0).registers.ebx >> 16) & 1u); // AVX512F bit #else return false; #endif From 5fcf8401562bc67cf0a4ed6fd197c5283fe07c34 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 2 Jun 2022 08:43:44 +0300 Subject: [PATCH 095/101] Typo. --- src/IO/S3Common.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 98471f5b81f..327730d9740 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -45,7 +45,7 @@ public: const String & force_region, const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, - bool enable_s3_requestrs_logging); + bool enable_s3_requests_logging); private: ClientFactory(); From a857bc2ccff315aeb16f1a4106789164cab8034e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 2 Jun 2022 08:46:41 +0300 Subject: [PATCH 096/101] Update S3Common.cpp --- src/IO/S3Common.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index c283afb21e4..fc4ab707026 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -765,9 +765,9 @@ namespace S3 const String & force_region, const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, - bool enable_s3_requestrs_logging) + bool enable_s3_requests_logging) { - return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requestrs_logging); + return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging); } URI::URI(const Poco::URI & uri_) From eef6a5ec9684b057708bdb23e690e7da5fffd8fc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 2 Jun 2022 09:41:12 +0300 Subject: [PATCH 097/101] Revert "Remove resursive submodules" --- .github/workflows/master.yml | 64 +++++++++++++++--------------- .github/workflows/pull_request.yml | 64 +++++++++++++++--------------- .gitmodules | 6 +-- contrib/arrow | 2 +- contrib/brotli | 2 +- contrib/cppkafka | 2 +- contrib/msgpack-c | 2 +- contrib/rapidjson | 2 +- contrib/snappy | 2 +- utils/check-style/check-style | 3 -- 10 files changed, 73 insertions(+), 76 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index e0954aab236..c890488ea80 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -215,8 +215,8 @@ jobs: fetch-depth: 0 # For a proper version and performance artifacts - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -259,8 +259,8 @@ jobs: fetch-depth: 0 # For a proper version and performance artifacts - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -305,8 +305,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -350,8 +350,8 @@ jobs: # uses: actions/checkout@v2 # - name: Build # run: | - # git -C "$GITHUB_WORKSPACE" submodule sync - # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + # git -C "$GITHUB_WORKSPACE" submodule sync --recursive + # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 # sudo rm -fr "$TEMP_PATH" # mkdir -p "$TEMP_PATH" # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -395,8 +395,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -440,8 +440,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -485,8 +485,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -530,8 +530,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -575,8 +575,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -623,8 +623,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -668,8 +668,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -715,8 +715,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -762,8 +762,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -809,8 +809,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -856,8 +856,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -903,8 +903,8 @@ jobs: fetch-depth: 0 # otherwise we will have no info about contributors - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index f6e9880d088..76a26d685c5 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -277,8 +277,8 @@ jobs: fetch-depth: 0 # for performance artifact - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -322,8 +322,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -367,8 +367,8 @@ jobs: # uses: actions/checkout@v2 # - name: Build # run: | - # git -C "$GITHUB_WORKSPACE" submodule sync - # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + # git -C "$GITHUB_WORKSPACE" submodule sync --recursive + # git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 # sudo rm -fr "$TEMP_PATH" # mkdir -p "$TEMP_PATH" # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -414,8 +414,8 @@ jobs: fetch-depth: 0 # for performance artifact - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -459,8 +459,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -504,8 +504,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -549,8 +549,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -594,8 +594,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -639,8 +639,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -687,8 +687,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -732,8 +732,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -777,8 +777,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -822,8 +822,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -867,8 +867,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -912,8 +912,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -957,8 +957,8 @@ jobs: uses: actions/checkout@v2 - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --init --jobs=10 + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" diff --git a/.gitmodules b/.gitmodules index aa68aa218b5..55fd684fddb 100644 --- a/.gitmodules +++ b/.gitmodules @@ -79,10 +79,10 @@ url = https://github.com/ClickHouse/snappy.git [submodule "contrib/cppkafka"] path = contrib/cppkafka - url = https://github.com/ClickHouse/cppkafka.git + url = https://github.com/mfontanini/cppkafka.git [submodule "contrib/brotli"] path = contrib/brotli - url = https://github.com/ClickHouse/brotli.git + url = https://github.com/google/brotli.git [submodule "contrib/h3"] path = contrib/h3 url = https://github.com/ClickHouse/h3 @@ -144,7 +144,7 @@ ignore = untracked [submodule "contrib/msgpack-c"] path = contrib/msgpack-c - url = https://github.com/ClickHouse/msgpack-c + url = https://github.com/msgpack/msgpack-c [submodule "contrib/libcpuid"] path = contrib/libcpuid url = https://github.com/ClickHouse/libcpuid.git diff --git a/contrib/arrow b/contrib/arrow index 6f274b737c6..efdcd015cfd 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 6f274b737c66a6c39bab0d3bdf6cf7d139ef06f5 +Subproject commit efdcd015cfdee1b6aa349c9ca227ca12c3d697f5 diff --git a/contrib/brotli b/contrib/brotli index 5bd78768449..63be8a99401 160000 --- a/contrib/brotli +++ b/contrib/brotli @@ -1 +1 @@ -Subproject commit 5bd78768449751a78d4b4c646b0612917986f5b1 +Subproject commit 63be8a99401992075c23e99f7c84de1c653e39e2 diff --git a/contrib/cppkafka b/contrib/cppkafka index 64bd67db12b..5a119f689f8 160000 --- a/contrib/cppkafka +++ b/contrib/cppkafka @@ -1 +1 @@ -Subproject commit 64bd67db12b9c705e9127439a5b05b351d9df7da +Subproject commit 5a119f689f8a4d90d10a9635e7ee2bee5c127de1 diff --git a/contrib/msgpack-c b/contrib/msgpack-c index 790b3fe58eb..46684265d50 160000 --- a/contrib/msgpack-c +++ b/contrib/msgpack-c @@ -1 +1 @@ -Subproject commit 790b3fe58ebded7a8bd130782ef28bec5784c248 +Subproject commit 46684265d50b5d1b062d4c5c428ba08462844b1d diff --git a/contrib/rapidjson b/contrib/rapidjson index b571bd5c1a3..c4ef90ccdbc 160000 --- a/contrib/rapidjson +++ b/contrib/rapidjson @@ -1 +1 @@ -Subproject commit b571bd5c1a3b1fc931d77ae36932537a3c9018c3 +Subproject commit c4ef90ccdbc21d5d5a628d08316bfd301e32d6fa diff --git a/contrib/snappy b/contrib/snappy index 3786173af20..fb057edfed8 160000 --- a/contrib/snappy +++ b/contrib/snappy @@ -1 +1 @@ -Subproject commit 3786173af204d21da97180977ad6ab4321138b3d +Subproject commit fb057edfed820212076239fd32cb2ff23e9016bf diff --git a/utils/check-style/check-style b/utils/check-style/check-style index 406b36e9251..84ce7ae5742 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -340,6 +340,3 @@ fi # Forbid files that differ only by character case find $ROOT_PATH | sort -f | uniq -i -c | awk '{ if ($1 > 1) print }' - -# Forbid recursive submodules -find $ROOT_PATH/contrib -name '.gitmodules' -size +0 | xargs cat | grep -P '.' && echo "Recursive submodules are forbidden." From 2b2232c264e655df20d92d29ad98447a182858ac Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 2 Jun 2022 06:42:29 -0300 Subject: [PATCH 098/101] test for #36995 (#37668) --- .../02312_is_not_null_prewhere.reference | 3 +++ .../02312_is_not_null_prewhere.sql | 21 +++++++++++++++++++ 2 files changed, 24 insertions(+) create mode 100644 tests/queries/0_stateless/02312_is_not_null_prewhere.reference create mode 100644 tests/queries/0_stateless/02312_is_not_null_prewhere.sql diff --git a/tests/queries/0_stateless/02312_is_not_null_prewhere.reference b/tests/queries/0_stateless/02312_is_not_null_prewhere.reference new file mode 100644 index 00000000000..bdaa7374c1b --- /dev/null +++ b/tests/queries/0_stateless/02312_is_not_null_prewhere.reference @@ -0,0 +1,3 @@ +2022-01-01 00:00:00 1 +2022-01-01 00:00:00 1 +2022-01-01 00:00:00 1 diff --git a/tests/queries/0_stateless/02312_is_not_null_prewhere.sql b/tests/queries/0_stateless/02312_is_not_null_prewhere.sql new file mode 100644 index 00000000000..56371d0ec6c --- /dev/null +++ b/tests/queries/0_stateless/02312_is_not_null_prewhere.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS bug_36995; + +CREATE TABLE bug_36995( + `time` DateTime, + `product` String) +ENGINE = MergeTree +ORDER BY time AS +SELECT '2022-01-01 00:00:00','1'; + +SELECT * FROM bug_36995 +WHERE (time IS NOT NULL) AND (product IN (SELECT '1')) +SETTINGS optimize_move_to_prewhere = 1; + +SELECT * FROM bug_36995 +WHERE (time IS NOT NULL) AND (product IN (SELECT '1')) +SETTINGS optimize_move_to_prewhere = 0; + +SELECT * FROM bug_36995 +PREWHERE (time IS NOT NULL) WHERE (product IN (SELECT '1')); + +DROP TABLE bug_36995; From d34e051c69d3acaefc8828d00b925af142ecef40 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 2 Jun 2022 11:46:33 +0200 Subject: [PATCH 099/101] Support for simultaneous read from local and remote parallel replica (#37204) --- src/Interpreters/Cluster.cpp | 10 + src/Interpreters/Cluster.h | 1 + .../ClusterProxy/IStreamFactory.h | 65 ------ .../ClusterProxy/SelectStreamFactory.cpp | 104 +++++++++- .../ClusterProxy/SelectStreamFactory.h | 59 +++++- .../ClusterProxy/executeQuery.cpp | 92 ++++++++- src/Interpreters/ClusterProxy/executeQuery.h | 15 +- src/Interpreters/InterpreterSelectQuery.cpp | 7 +- src/Interpreters/InterpreterSelectQuery.h | 2 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 4 +- .../QueryPlan/DistributedCreateLocalPlan.h | 2 + src/Processors/QueryPlan/ReadFromRemote.cpp | 192 +++++++++++------- src/Processors/QueryPlan/ReadFromRemote.h | 60 ++++-- .../ParallelReplicasReadingCoordinator.h | 2 + src/Storages/StorageDistributed.cpp | 27 ++- ..._parallel_processing_on_replicas_part_1.sh | 2 +- 16 files changed, 468 insertions(+), 176 deletions(-) delete mode 100644 src/Interpreters/ClusterProxy/IStreamFactory.h diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp index d817988e7b6..54f55c7b1f6 100644 --- a/src/Interpreters/Cluster.cpp +++ b/src/Interpreters/Cluster.cpp @@ -418,6 +418,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, if (address.is_local) info.local_addresses.push_back(address); + info.all_addresses.push_back(address); + auto pool = ConnectionPoolFactory::instance().get( settings.distributed_connections_pool_size, address.host_name, address.port, @@ -485,6 +487,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, } Addresses shard_local_addresses; + Addresses shard_all_addresses; ConnectionPoolPtrs all_replicas_pools; all_replicas_pools.reserve(replica_addresses.size()); @@ -502,6 +505,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, all_replicas_pools.emplace_back(replica_pool); if (replica.is_local) shard_local_addresses.push_back(replica); + shard_all_addresses.push_back(replica); } ConnectionPoolWithFailoverPtr shard_pool = std::make_shared( @@ -516,6 +520,7 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, current_shard_num, weight, std::move(shard_local_addresses), + std::move(shard_all_addresses), std::move(shard_pool), std::move(all_replicas_pools), internal_replication @@ -571,6 +576,7 @@ Cluster::Cluster( addresses_with_failover.emplace_back(current); Addresses shard_local_addresses; + Addresses all_addresses; ConnectionPoolPtrs all_replicas; all_replicas.reserve(current.size()); @@ -585,6 +591,7 @@ Cluster::Cluster( all_replicas.emplace_back(replica_pool); if (replica.is_local && !treat_local_as_remote) shard_local_addresses.push_back(replica); + all_addresses.push_back(replica); } ConnectionPoolWithFailoverPtr shard_pool = std::make_shared( @@ -597,6 +604,7 @@ Cluster::Cluster( current_shard_num, default_weight, std::move(shard_local_addresses), + std::move(all_addresses), std::move(shard_pool), std::move(all_replicas), false // has_internal_replication @@ -680,6 +688,8 @@ Cluster::Cluster(Cluster::ReplicasAsShardsTag, const Cluster & from, const Setti if (address.is_local) info.local_addresses.push_back(address); + info.all_addresses.push_back(address); + auto pool = ConnectionPoolFactory::instance().get( settings.distributed_connections_pool_size, address.host_name, diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index 7c8d15d0350..5ce011782fc 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -202,6 +202,7 @@ public: UInt32 shard_num = 0; UInt32 weight = 1; Addresses local_addresses; + Addresses all_addresses; /// nullptr if there are no remote addresses ConnectionPoolWithFailoverPtr pool; /// Connection pool for each replica, contains nullptr for local replicas diff --git a/src/Interpreters/ClusterProxy/IStreamFactory.h b/src/Interpreters/ClusterProxy/IStreamFactory.h deleted file mode 100644 index 483ce9dcab9..00000000000 --- a/src/Interpreters/ClusterProxy/IStreamFactory.h +++ /dev/null @@ -1,65 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -struct Settings; -class Cluster; -class Throttler; -struct SelectQueryInfo; - -class Pipe; -using Pipes = std::vector; - -class QueryPlan; -using QueryPlanPtr = std::unique_ptr; - -struct StorageID; - -namespace ClusterProxy -{ - -/// Base class for the implementation of the details of distributed query -/// execution that are specific to the query type. -class IStreamFactory -{ -public: - virtual ~IStreamFactory() = default; - - struct Shard - { - /// Query and header may be changed depending on shard. - ASTPtr query; - Block header; - - size_t shard_num = 0; - size_t num_replicas = 0; - ConnectionPoolWithFailoverPtr pool; - ConnectionPoolPtrs per_replica_pools; - - /// If we connect to replicas lazily. - /// (When there is a local replica with big delay). - bool lazy = false; - UInt32 local_delay = 0; - }; - - using Shards = std::vector; - - virtual void createForShard( - const Cluster::ShardInfo & shard_info, - const ASTPtr & query_ast, - const StorageID & main_table, - const ASTPtr & table_func_ptr, - ContextPtr context, - std::vector & local_plans, - Shards & remote_shards, - UInt32 shard_count) = 0; -}; - -} - -} diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index a2afed3759f..89123cda531 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -10,14 +11,15 @@ #include #include +#include #include #include +#include #include #include #include #include - namespace ProfileEvents { extern const Event DistributedConnectionMissingTable; @@ -63,7 +65,8 @@ void SelectStreamFactory::createForShard( auto emplace_local_stream = [&]() { - local_plans.emplace_back(createLocalPlan(query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*coordinator=*/nullptr)); + local_plans.emplace_back(createLocalPlan( + query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, /*replica_num=*/0, /*replica_count=*/0, /*coordinator=*/nullptr)); }; auto emplace_remote_stream = [&](bool lazy = false, UInt32 local_delay = 0) @@ -71,10 +74,7 @@ void SelectStreamFactory::createForShard( remote_shards.emplace_back(Shard{ .query = query_ast, .header = header, - .shard_num = shard_info.shard_num, - .num_replicas = shard_info.getAllNodeCount(), - .pool = shard_info.pool, - .per_replica_pools = shard_info.per_replica_pools, + .shard_info = shard_info, .lazy = lazy, .local_delay = local_delay, }); @@ -173,5 +173,97 @@ void SelectStreamFactory::createForShard( emplace_remote_stream(); } + +SelectStreamFactory::ShardPlans SelectStreamFactory::createForShardWithParallelReplicas( + const Cluster::ShardInfo & shard_info, + const ASTPtr & query_ast, + const StorageID & main_table, + const ASTPtr & table_function_ptr, + const ThrottlerPtr & throttler, + ContextPtr context, + UInt32 shard_count) +{ + SelectStreamFactory::ShardPlans result; + + if (auto it = objects_by_shard.find(shard_info.shard_num); it != objects_by_shard.end()) + replaceMissedSubcolumnsByConstants(storage_snapshot->object_columns, it->second, query_ast); + + const auto & settings = context->getSettingsRef(); + + auto is_local_replica_obsolete = [&]() + { + auto resolved_id = context->resolveStorageID(main_table); + auto main_table_storage = DatabaseCatalog::instance().tryGetTable(resolved_id, context); + const auto * replicated_storage = dynamic_cast(main_table_storage.get()); + + if (!replicated_storage) + return false; + + UInt64 max_allowed_delay = settings.max_replica_delay_for_distributed_queries; + + if (!max_allowed_delay) + return false; + + UInt32 local_delay = replicated_storage->getAbsoluteDelay(); + return local_delay >= max_allowed_delay; + }; + + size_t next_replica_number = 0; + size_t all_replicas_count = shard_info.getRemoteNodeCount(); + + auto coordinator = std::make_shared(); + auto remote_plan = std::make_unique(); + + + if (settings.prefer_localhost_replica && shard_info.isLocal()) + { + /// We don't need more than one local replica in parallel reading + if (!is_local_replica_obsolete()) + { + ++all_replicas_count; + + result.local_plan = createLocalPlan( + query_ast, header, context, processed_stage, shard_info.shard_num, shard_count, next_replica_number, all_replicas_count, coordinator); + + ++next_replica_number; + } + } + + Scalars scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + scalars.emplace( + "_shard_count", Block{{DataTypeUInt32().createColumnConst(1, shard_count), std::make_shared(), "_shard_count"}}); + auto external_tables = context->getExternalTables(); + + auto shard = Shard{ + .query = query_ast, + .header = header, + .shard_info = shard_info, + .lazy = false, + .local_delay = 0, + }; + + if (shard_info.hasRemoteConnections()) + { + auto read_from_remote = std::make_unique( + coordinator, + shard, + header, + processed_stage, + main_table, + table_function_ptr, + context, + throttler, + std::move(scalars), + std::move(external_tables), + &Poco::Logger::get("ReadFromParallelRemoteReplicasStep"), + shard_count); + + remote_plan->addStep(std::move(read_from_remote)); + result.remote_plan = std::move(remote_plan); + } + + return result; +} + } } diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 731bf3acd10..f64e57e1316 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -1,22 +1,56 @@ #pragma once #include -#include #include #include #include +#include +#include +#include namespace DB { + +struct Settings; +class Cluster; +class Throttler; +struct SelectQueryInfo; + +class Pipe; +using Pipes = std::vector; + +class QueryPlan; +using QueryPlanPtr = std::unique_ptr; + +struct StorageID; + namespace ClusterProxy { + using ColumnsDescriptionByShardNum = std::unordered_map; -class SelectStreamFactory final : public IStreamFactory +class SelectStreamFactory { public: + + struct Shard + { + /// Query and header may be changed depending on shard. + ASTPtr query; + Block header; + + Cluster::ShardInfo shard_info; + + /// If we connect to replicas lazily. + /// (When there is a local replica with big delay). + bool lazy = false; + UInt32 local_delay = 0; + }; + + using Shards = std::vector; + SelectStreamFactory( const Block & header_, const ColumnsDescriptionByShardNum & objects_by_shard_, @@ -31,7 +65,26 @@ public: ContextPtr context, std::vector & local_plans, Shards & remote_shards, - UInt32 shard_count) override; + UInt32 shard_count); + + struct ShardPlans + { + /// If a shard has local replicas this won't be nullptr + std::unique_ptr local_plan; + + /// Contains several steps to read from all remote replicas + std::unique_ptr remote_plan; + }; + + ShardPlans createForShardWithParallelReplicas( + const Cluster::ShardInfo & shard_info, + const ASTPtr & query_ast, + const StorageID & main_table, + const ASTPtr & table_function_ptr, + const ThrottlerPtr & throttler, + ContextPtr context, + UInt32 shard_count + ); private: const Block header; diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 3f1823fb171..e7a0e24fc7b 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { extern const int TOO_LARGE_DISTRIBUTED_DEPTH; + extern const int LOGICAL_ERROR; } namespace ClusterProxy @@ -106,21 +107,19 @@ void executeQuery( QueryProcessingStage::Enum processed_stage, const StorageID & main_table, const ASTPtr & table_func_ptr, - IStreamFactory & stream_factory, Poco::Logger * log, + SelectStreamFactory & stream_factory, Poco::Logger * log, const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, const ClusterPtr & not_optimized_cluster) { - assert(log); - const Settings & settings = context->getSettingsRef(); if (settings.max_distributed_depth && context->getClientInfo().distributed_depth >= settings.max_distributed_depth) throw Exception("Maximum distributed depth exceeded", ErrorCodes::TOO_LARGE_DISTRIBUTED_DEPTH); std::vector plans; - IStreamFactory::Shards remote_shards; + SelectStreamFactory::Shards remote_shards; auto new_context = updateSettingsForCluster(*query_info.getCluster(), context, settings, log); @@ -213,6 +212,91 @@ void executeQuery( query_plan.unitePlans(std::move(union_step), std::move(plans)); } + +void executeQueryWithParallelReplicas( + QueryPlan & query_plan, + const StorageID & main_table, + const ASTPtr & table_func_ptr, + SelectStreamFactory & stream_factory, + const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, + const ExpressionActionsPtr & sharding_key_expr, + const std::string & sharding_key_column_name, + const ClusterPtr & not_optimized_cluster) +{ + const Settings & settings = context->getSettingsRef(); + + ThrottlerPtr user_level_throttler; + if (auto * process_list_element = context->getProcessListElement()) + user_level_throttler = process_list_element->getUserNetworkThrottler(); + + /// Network bandwidth limit, if needed. + ThrottlerPtr throttler; + if (settings.max_network_bandwidth || settings.max_network_bytes) + { + throttler = std::make_shared( + settings.max_network_bandwidth, + settings.max_network_bytes, + "Limit for bytes to send or receive over network exceeded.", + user_level_throttler); + } + else + throttler = user_level_throttler; + + + std::vector plans; + size_t shards = query_info.getCluster()->getShardCount(); + + for (const auto & shard_info : query_info.getCluster()->getShardsInfo()) + { + ASTPtr query_ast_for_shard; + if (query_info.optimized_cluster && settings.optimize_skip_unused_shards_rewrite_in && shards > 1) + { + query_ast_for_shard = query_ast->clone(); + + OptimizeShardingKeyRewriteInVisitor::Data visitor_data{ + sharding_key_expr, + sharding_key_expr->getSampleBlock().getByPosition(0).type, + sharding_key_column_name, + shard_info, + not_optimized_cluster->getSlotToShard(), + }; + OptimizeShardingKeyRewriteInVisitor visitor(visitor_data); + visitor.visit(query_ast_for_shard); + } + else + query_ast_for_shard = query_ast; + + auto shard_plans = stream_factory.createForShardWithParallelReplicas(shard_info, + query_ast_for_shard, main_table, table_func_ptr, throttler, context, shards); + + if (!shard_plans.local_plan && !shard_plans.remote_plan) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from shard. This is a bug"); + + if (shard_plans.local_plan) + plans.emplace_back(std::move(shard_plans.local_plan)); + + if (shard_plans.remote_plan) + plans.emplace_back(std::move(shard_plans.remote_plan)); + } + + if (plans.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No plans were generated for reading from Distributed. This is a bug"); + + if (plans.size() == 1) + { + query_plan = std::move(*plans.front()); + return; + } + + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (const auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + query_plan.unitePlans(std::move(union_step), std::move(plans)); +} + } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index d38bbe0fd5b..1a5035015a7 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -23,7 +23,7 @@ struct StorageID; namespace ClusterProxy { -class IStreamFactory; +class SelectStreamFactory; /// Update settings for Distributed query. /// @@ -46,7 +46,18 @@ void executeQuery( QueryProcessingStage::Enum processed_stage, const StorageID & main_table, const ASTPtr & table_func_ptr, - IStreamFactory & stream_factory, Poco::Logger * log, + SelectStreamFactory & stream_factory, Poco::Logger * log, + const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, + const ExpressionActionsPtr & sharding_key_expr, + const std::string & sharding_key_column_name, + const ClusterPtr & not_optimized_cluster); + + +void executeQueryWithParallelReplicas( + QueryPlan & query_plan, + const StorageID & main_table, + const ASTPtr & table_func_ptr, + SelectStreamFactory & stream_factory, const ASTPtr & query_ast, ContextPtr context, const SelectQueryInfo & query_info, const ExpressionActionsPtr & sharding_key_expr, const std::string & sharding_key_column_name, diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index fbafb98e0d8..a62533cfc8a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1728,12 +1728,11 @@ void InterpreterSelectQuery::setMergeTreeReadTaskCallbackAndClientInfo(MergeTree context->setMergeTreeReadTaskCallback(std::move(callback)); } -void InterpreterSelectQuery::setProperClientInfo() +void InterpreterSelectQuery::setProperClientInfo(size_t replica_num, size_t replica_count) { context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; - assert(options.shard_count.has_value() && options.shard_num.has_value()); - context->getClientInfo().count_participating_replicas = *options.shard_count; - context->getClientInfo().number_of_current_replica = *options.shard_num; + context->getClientInfo().count_participating_replicas = replica_count; + context->getClientInfo().number_of_current_replica = replica_num; } bool InterpreterSelectQuery::shouldMoveToPrewhere() diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 8aee3c7c273..b7807a486b5 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -125,7 +125,7 @@ public: void setMergeTreeReadTaskCallbackAndClientInfo(MergeTreeReadTaskCallback && callback); /// It will set shard_num and shard_count to the client_info - void setProperClientInfo(); + void setProperClientInfo(size_t replica_num, size_t replica_count); private: InterpreterSelectQuery( diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 854a677afb9..f91c8020509 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -41,6 +41,8 @@ std::unique_ptr createLocalPlan( QueryProcessingStage::Enum processed_stage, UInt32 shard_num, UInt32 shard_count, + size_t replica_num, + size_t replica_count, std::shared_ptr coordinator) { checkStackSize(); @@ -56,7 +58,7 @@ std::unique_ptr createLocalPlan( .setShardInfo(shard_num, shard_count) .ignoreASTOptimizations()); - interpreter.setProperClientInfo(); + interpreter.setProperClientInfo(replica_num, replica_count); if (coordinator) { interpreter.setMergeTreeReadTaskCallbackAndClientInfo([coordinator](PartitionReadRequest request) -> std::optional diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index fdfe1709833..b55cedf9871 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -15,6 +15,8 @@ std::unique_ptr createLocalPlan( QueryProcessingStage::Enum processed_stage, UInt32 shard_num, UInt32 shard_count, + size_t replica_num, + size_t replica_count, std::shared_ptr coordinator); } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 35db2b1548a..867daaff30c 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -16,6 +16,8 @@ #include #include +#include + namespace DB { @@ -62,7 +64,7 @@ static String formattedAST(const ASTPtr & ast) } ReadFromRemote::ReadFromRemote( - ClusterProxy::IStreamFactory::Shards shards_, + ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, StorageID main_table_, @@ -87,10 +89,7 @@ ReadFromRemote::ReadFromRemote( { } -void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, - std::shared_ptr coordinator, - std::shared_ptr pool, - std::optional replica_info) +void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -103,10 +102,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto } auto lazily_create_stream = [ - replica_info = replica_info, - pool = pool ? pool : shard.pool, - coordinator = coordinator, - shard_num = shard.shard_num, shard_count = shard_count, query = shard.query, header = shard.header, + shard = shard, shard_count = shard_count, query = shard.query, header = shard.header, context = context, throttler = throttler, main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables, @@ -122,15 +118,15 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto try { if (table_func_ptr) - try_results = pool->getManyForTableFunction(timeouts, ¤t_settings, PoolMode::GET_MANY); + try_results = shard.shard_info.pool->getManyForTableFunction(timeouts, ¤t_settings, PoolMode::GET_MANY); else - try_results = pool->getManyChecked(timeouts, ¤t_settings, PoolMode::GET_MANY, main_table.getQualifiedName()); + try_results = shard.shard_info.pool->getManyChecked(timeouts, ¤t_settings, PoolMode::GET_MANY, main_table.getQualifiedName()); } catch (const Exception & ex) { if (ex.code() == ErrorCodes::ALL_CONNECTION_TRIES_FAILED) LOG_WARNING(&Poco::Logger::get("ClusterProxy::SelectStreamFactory"), - "Connections to remote replicas of local shard {} failed, will use stale local replica", shard_num); + "Connections to remote replicas of local shard {} failed, will use stale local replica", shard.shard_info.shard_num); else throw; } @@ -144,7 +140,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto if (try_results.empty() || local_delay < max_remote_delay) { - auto plan = createLocalPlan(query, header, context, stage, shard_num, shard_count, coordinator); + auto plan = createLocalPlan(query, header, context, stage, shard.shard_info.shard_num, shard_count, 0, 0, /*coordinator=*/nullptr); return QueryPipelineBuilder::getPipe(std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), @@ -160,10 +156,9 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto String query_string = formattedAST(query); scalars["_shard_num"] - = Block{{DataTypeUInt32().createColumnConst(1, shard_num), std::make_shared(), "_shard_num"}}; + = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; auto remote_query_executor = std::make_shared( - pool, std::move(connections), query_string, header, context, throttler, scalars, external_tables, stage, - RemoteQueryExecutor::Extension{.parallel_reading_coordinator = std::move(coordinator), .replica_info = replica_info}); + shard.shard_info.pool, std::move(connections), query_string, header, context, throttler, scalars, external_tables, stage); return createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read); } @@ -174,10 +169,7 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto addConvertingActions(pipes.back(), output_stream->header); } -void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, - std::shared_ptr coordinator, - std::shared_ptr pool, - std::optional replica_info) +void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -192,20 +184,15 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory:: String query_string = formattedAST(shard.query); scalars["_shard_num"] - = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_num), std::make_shared(), "_shard_num"}}; + = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; std::shared_ptr remote_query_executor; remote_query_executor = std::make_shared( - pool ? pool : shard.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage, - RemoteQueryExecutor::Extension{.parallel_reading_coordinator = std::move(coordinator), .replica_info = std::move(replica_info)}); + shard.shard_info.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage); remote_query_executor->setLogger(log); - - /// In case of parallel reading from replicas we have a connection pool per replica. - /// Setting PoolMode will make no sense. - if (!pool) - remote_query_executor->setPoolMode(PoolMode::GET_MANY); + remote_query_executor->setPoolMode(PoolMode::GET_MANY); if (!table_func_ptr) remote_query_executor->setMainTable(main_table); @@ -219,52 +206,119 @@ void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const B { Pipes pipes; - const auto & settings = context->getSettingsRef(); - const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; - - /// We have to create a pipe for each replica - /// FIXME: The second condition is only for tests to work, because hedged connections enabled by default. - if (settings.max_parallel_replicas > 1 && !enable_sample_offset_parallel_processing && !context->getSettingsRef().use_hedged_requests) + for (const auto & shard : shards) { - const Settings & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - - for (const auto & shard : shards) - { - auto coordinator = std::make_shared(); - - for (size_t replica_num = 0; replica_num < shard.num_replicas; ++replica_num) - { - IConnections::ReplicaInfo replica_info - { - .all_replicas_count = shard.num_replicas, - .number_of_current_replica = replica_num - }; - - auto pool = shard.per_replica_pools[replica_num]; - auto pool_with_failover = std::make_shared( - ConnectionPoolPtrs{pool}, current_settings.load_balancing); - - if (shard.lazy) - addLazyPipe(pipes, shard, coordinator, pool_with_failover, replica_info); - else - addPipe(pipes, shard, coordinator, pool_with_failover, replica_info); - } - } - } - else - { - for (const auto & shard : shards) - { - if (shard.lazy) - addLazyPipe(pipes, shard, /*coordinator=*/nullptr, /*pool*/{}, /*replica_info*/std::nullopt); - else - addPipe(pipes, shard, /*coordinator=*/nullptr, /*pool*/{}, /*replica_info*/std::nullopt); - } + if (shard.lazy) + addLazyPipe(pipes, shard); + else + addPipe(pipes, shard); } auto pipe = Pipe::unitePipes(std::move(pipes)); pipeline.init(std::move(pipe)); } + +ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( + ParallelReplicasReadingCoordinatorPtr coordinator_, + ClusterProxy::SelectStreamFactory::Shard shard_, + Block header_, + QueryProcessingStage::Enum stage_, + StorageID main_table_, + ASTPtr table_func_ptr_, + ContextPtr context_, + ThrottlerPtr throttler_, + Scalars scalars_, + Tables external_tables_, + Poco::Logger * log_, + UInt32 shard_count_) + : ISourceStep(DataStream{.header = std::move(header_)}) + , coordinator(std::move(coordinator_)) + , shard(std::move(shard_)) + , stage(std::move(stage_)) + , main_table(std::move(main_table_)) + , table_func_ptr(table_func_ptr_) + , context(context_) + , throttler(throttler_) + , scalars(scalars_) + , external_tables{external_tables_} + , log(log_) + , shard_count(shard_count_) +{ + std::vector description; + + for (const auto & address : shard.shard_info.all_addresses) + if (!address.is_local) + description.push_back(fmt::format("Replica: {}", address.host_name)); + + setStepDescription(boost::algorithm::join(description, ", ")); +} + + +void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + Pipes pipes; + + const Settings & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + + for (size_t replica_num = 0; replica_num < shard.shard_info.getAllNodeCount(); ++replica_num) + { + if (shard.shard_info.all_addresses[replica_num].is_local) + continue; + + IConnections::ReplicaInfo replica_info + { + .all_replicas_count = shard.shard_info.getAllNodeCount(), + .number_of_current_replica = replica_num + }; + + auto pool = shard.shard_info.per_replica_pools[replica_num]; + assert(pool); + + auto pool_with_failover = std::make_shared( + ConnectionPoolPtrs{pool}, current_settings.load_balancing); + + addPipeForSingeReplica(pipes, pool_with_failover, replica_info); + } + + auto pipe = Pipe::unitePipes(std::move(pipes)); + pipeline.init(std::move(pipe)); + +} + + +void ReadFromParallelRemoteReplicasStep::addPipeForSingeReplica(Pipes & pipes, std::shared_ptr pool, IConnections::ReplicaInfo replica_info) +{ + bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; + bool add_totals = false; + bool add_extremes = false; + bool async_read = context->getSettingsRef().async_socket_for_remote; + if (stage == QueryProcessingStage::Complete) + { + add_totals = shard.query->as().group_by_with_totals; + add_extremes = context->getSettingsRef().extremes; + } + + String query_string = formattedAST(shard.query); + + scalars["_shard_num"] + = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_info.shard_num), std::make_shared(), "_shard_num"}}; + + std::shared_ptr remote_query_executor; + + remote_query_executor = std::make_shared( + pool, query_string, shard.header, context, throttler, scalars, external_tables, stage, + RemoteQueryExecutor::Extension{.parallel_reading_coordinator = coordinator, .replica_info = std::move(replica_info)}); + + remote_query_executor->setLogger(log); + + if (!table_func_ptr) + remote_query_executor->setMainTable(main_table); + + pipes.emplace_back(createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read)); + pipes.back().addInterpreterContext(context); + addConvertingActions(pipes.back(), output_stream->header); +} + } diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index f361be93b5a..e8f4ee5fd37 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include namespace DB @@ -22,7 +22,7 @@ class ReadFromRemote final : public ISourceStep { public: ReadFromRemote( - ClusterProxy::IStreamFactory::Shards shards_, + ClusterProxy::SelectStreamFactory::Shards shards_, Block header_, QueryProcessingStage::Enum stage_, StorageID main_table_, @@ -45,7 +45,7 @@ private: PerShard }; - ClusterProxy::IStreamFactory::Shards shards; + ClusterProxy::SelectStreamFactory::Shards shards; QueryProcessingStage::Enum stage; StorageID main_table; @@ -60,16 +60,52 @@ private: Poco::Logger * log; UInt32 shard_count; - void addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, - std::shared_ptr coordinator, - std::shared_ptr pool, - std::optional replica_info); - void addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, - std::shared_ptr coordinator, - std::shared_ptr pool, - std::optional replica_info); + void addLazyPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard); + void addPipe(Pipes & pipes, const ClusterProxy::SelectStreamFactory::Shard & shard); +}; - void addPipeForReplica(); + +class ReadFromParallelRemoteReplicasStep : public ISourceStep +{ +public: + ReadFromParallelRemoteReplicasStep( + ParallelReplicasReadingCoordinatorPtr coordinator_, + ClusterProxy::SelectStreamFactory::Shard shard, + Block header_, + QueryProcessingStage::Enum stage_, + StorageID main_table_, + ASTPtr table_func_ptr_, + ContextPtr context_, + ThrottlerPtr throttler_, + Scalars scalars_, + Tables external_tables_, + Poco::Logger * log_, + UInt32 shard_count_); + + String getName() const override { return "ReadFromRemoteParallelReplicas"; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + +private: + + void addPipeForSingeReplica(Pipes & pipes, std::shared_ptr pool, IConnections::ReplicaInfo replica_info); + + ParallelReplicasReadingCoordinatorPtr coordinator; + ClusterProxy::SelectStreamFactory::Shard shard; + QueryProcessingStage::Enum stage; + + StorageID main_table; + ASTPtr table_func_ptr; + + ContextPtr context; + + ThrottlerPtr throttler; + Scalars scalars; + Tables external_tables; + + Poco::Logger * log; + + UInt32 shard_count{0}; }; } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index af74e0fae49..bd2082be6c2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -17,4 +17,6 @@ private: std::unique_ptr pimpl; }; +using ParallelReplicasReadingCoordinatorPtr = std::shared_ptr; + } diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 2c3a1489943..a5f20c56c9d 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -707,13 +707,25 @@ void StorageDistributed::read( storage_snapshot, processed_stage); - ClusterProxy::executeQuery( - query_plan, header, processed_stage, - main_table, remote_table_function_ptr, - select_stream_factory, log, modified_query_ast, - local_context, query_info, - sharding_key_expr, sharding_key_column_name, - query_info.cluster); + + auto settings = local_context->getSettingsRef(); + bool parallel_replicas = settings.max_parallel_replicas > 1 && settings.allow_experimental_parallel_reading_from_replicas && !settings.use_hedged_requests; + + if (parallel_replicas) + ClusterProxy::executeQueryWithParallelReplicas( + query_plan, main_table, remote_table_function_ptr, + select_stream_factory, modified_query_ast, + local_context, query_info, + sharding_key_expr, sharding_key_column_name, + query_info.cluster); + else + ClusterProxy::executeQuery( + query_plan, header, processed_stage, + main_table, remote_table_function_ptr, + select_stream_factory, log, modified_query_ast, + local_context, query_info, + sharding_key_expr, sharding_key_column_name, + query_info.cluster); /// This is a bug, it is possible only when there is no shards to query, and this is handled earlier. if (!query_plan.isInitialized()) @@ -1523,4 +1535,3 @@ void registerStorageDistributed(StorageFactory & factory) } } - diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh index bbb5d903ea9..ecd0d281b53 100755 --- a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -10,7 +10,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # All replicas are localhost, disable `prefer_localhost_replica` option to test network interface # Currently this feature could not work with hedged requests # Enabling `enable_sample_offset_parallel_processing` feature could lead to intersecting marks, so some of them would be thrown away and it will lead to incorrect result of SELECT query -SETTINGS="--max_parallel_replicas=3 --use_hedged_requests=false --async_socket_for_remote=false --allow_experimental_parallel_reading_from_replicas=true" +SETTINGS="--max_parallel_replicas=3 --use_hedged_requests=false --allow_experimental_parallel_reading_from_replicas=true" # Prepare tables $CLICKHOUSE_CLIENT $SETTINGS -nm -q ''' From 6a516099152c8c14ffbcc6ffdc79c46077918889 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Jun 2022 13:35:23 +0300 Subject: [PATCH 100/101] Update 02316_const_string_intersact.sql --- tests/queries/0_stateless/02316_const_string_intersact.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02316_const_string_intersact.sql b/tests/queries/0_stateless/02316_const_string_intersact.sql index ace3c8d03c5..18af398aa5d 100644 --- a/tests/queries/0_stateless/02316_const_string_intersact.sql +++ b/tests/queries/0_stateless/02316_const_string_intersact.sql @@ -1 +1,3 @@ +-- Tags: no-backward-compatibility-check + SELECT 'Play ClickHouse' InterSect SELECT 'Play ClickHouse' From 4baae59252ae629f39a1e1562c3e057abb72d5eb Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Jun 2022 14:04:28 +0300 Subject: [PATCH 101/101] Revert "Fix possible segfault in schema inference" --- src/Formats/ReadSchemaUtils.cpp | 21 ++++++++----------- ...18_template_schema_inference_bug.reference | 0 .../02318_template_schema_inference_bug.sql | 2 -- 3 files changed, 9 insertions(+), 14 deletions(-) delete mode 100644 tests/queries/0_stateless/02318_template_schema_inference_bug.reference delete mode 100755 tests/queries/0_stateless/02318_template_schema_inference_bug.sql diff --git a/src/Formats/ReadSchemaUtils.cpp b/src/Formats/ReadSchemaUtils.cpp index 11a91bd50dc..035546031d8 100644 --- a/src/Formats/ReadSchemaUtils.cpp +++ b/src/Formats/ReadSchemaUtils.cpp @@ -100,21 +100,18 @@ ColumnsDescription readSchemaFromFormat( catch (...) { auto exception_message = getCurrentExceptionMessage(false); - if (schema_reader) + size_t rows_read = schema_reader->getNumRowsRead(); + assert(rows_read <= max_rows_to_read); + max_rows_to_read -= schema_reader->getNumRowsRead(); + if (rows_read != 0 && max_rows_to_read == 0) { - size_t rows_read = schema_reader->getNumRowsRead(); - assert(rows_read <= max_rows_to_read); - max_rows_to_read -= schema_reader->getNumRowsRead(); - if (rows_read != 0 && max_rows_to_read == 0) + exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; + if (iterations > 1) { - exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference"; - if (iterations > 1) - { - exception_messages += "\n" + exception_message; - break; - } - retry = false; + exception_messages += "\n" + exception_message; + break; } + retry = false; } if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode())) diff --git a/tests/queries/0_stateless/02318_template_schema_inference_bug.reference b/tests/queries/0_stateless/02318_template_schema_inference_bug.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/queries/0_stateless/02318_template_schema_inference_bug.sql b/tests/queries/0_stateless/02318_template_schema_inference_bug.sql deleted file mode 100755 index 42646013dd5..00000000000 --- a/tests/queries/0_stateless/02318_template_schema_inference_bug.sql +++ /dev/null @@ -1,2 +0,0 @@ -insert into function file(data_02318.tsv) select * from numbers(10); -desc file('data_02318.tsv', 'Template') SETTINGS format_template_row='nonexist', format_template_resultset='nonexist'; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE}