From 227af9a074b31959937fb24e11188ce0f66ddac6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Oct 2022 21:31:31 +0200 Subject: [PATCH 001/339] Optimizations around ThreadStatus --- src/Common/ThreadStatus.cpp | 9 +++------ src/Common/ThreadStatus.h | 6 +----- src/Daemon/BaseDaemon.cpp | 3 ++- src/Daemon/CMakeLists.txt | 2 +- src/Interpreters/ThreadStatusExt.cpp | 4 ++-- 5 files changed, 9 insertions(+), 15 deletions(-) diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp index b62a7af6c71..81650f107a4 100644 --- a/src/Common/ThreadStatus.cpp +++ b/src/Common/ThreadStatus.cpp @@ -188,13 +188,10 @@ void ThreadStatus::updatePerformanceCounters() } } -void ThreadStatus::assertState(const std::initializer_list & permitted_states, const char * description) const +void ThreadStatus::assertState(ThreadState permitted_state, const char * description) const { - for (auto permitted_state : permitted_states) - { - if (getCurrentState() == permitted_state) - return; - } + if (getCurrentState() == permitted_state) + return; if (description) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected thread state {}: {}", getCurrentState(), description); diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 67450d8c779..313bae5d27b 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -87,10 +87,6 @@ public: LogsLevel client_logs_level = LogsLevel::none; String query; - /// Query without new lines (see toOneLineQuery()) - /// Used to print in case of fatal error - /// (to avoid calling extra code in the fatal error handler) - String one_line_query; UInt64 normalized_query_hash = 0; std::vector finished_threads_counters_memory; @@ -286,7 +282,7 @@ protected: void logToQueryThreadLog(QueryThreadLog & thread_log, const String & current_database, std::chrono::time_point now); - void assertState(const std::initializer_list & permitted_states, const char * description = nullptr) const; + void assertState(ThreadState permitted_state, const char * description = nullptr) const; private: diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 2dddfd6874a..edd203f7df9 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -304,7 +305,7 @@ private: if (auto thread_group = thread_ptr->getThreadGroup()) { - query = thread_group->one_line_query; + query = DB::toOneLineQuery(thread_group->query); } if (auto logs_queue = thread_ptr->getInternalTextLogsQueue()) diff --git a/src/Daemon/CMakeLists.txt b/src/Daemon/CMakeLists.txt index e1a9f09003c..316b03dc535 100644 --- a/src/Daemon/CMakeLists.txt +++ b/src/Daemon/CMakeLists.txt @@ -11,7 +11,7 @@ if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES) target_link_libraries (daemon PUBLIC -Wl,-undefined,dynamic_lookup) endif() -target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_common_io clickhouse_common_config) +target_link_libraries (daemon PUBLIC loggers common PRIVATE clickhouse_parsers clickhouse_common_io clickhouse_common_config) if (TARGET ch_contrib::sentry) target_link_libraries (daemon PRIVATE ch_contrib::sentry dbms) diff --git a/src/Interpreters/ThreadStatusExt.cpp b/src/Interpreters/ThreadStatusExt.cpp index 4810174e395..35b4afb5595 100644 --- a/src/Interpreters/ThreadStatusExt.cpp +++ b/src/Interpreters/ThreadStatusExt.cpp @@ -97,7 +97,7 @@ void CurrentThread::defaultThreadDeleter() void ThreadStatus::setupState(const ThreadGroupStatusPtr & thread_group_) { - assertState({ThreadState::DetachedFromQuery}, __PRETTY_FUNCTION__); + assertState(ThreadState::DetachedFromQuery, __PRETTY_FUNCTION__); /// Attach or init current thread to thread group and copy useful information from it thread_group = thread_group_; @@ -315,7 +315,7 @@ void ThreadStatus::detachQuery(bool exit_if_already_detached, bool thread_exits) return; } - assertState({ThreadState::AttachedToQuery}, __PRETTY_FUNCTION__); + assertState(ThreadState::AttachedToQuery, __PRETTY_FUNCTION__); finalizeQueryProfiler(); finalizePerformanceCounters(); From 19310a5877f79665907c2c386a2ea11a555481df Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Oct 2022 21:50:11 +0200 Subject: [PATCH 002/339] Replace std::vector with absl inlined_vector --- src/CMakeLists.txt | 1 + src/Interpreters/ComparisonGraph.h | 8 +++--- src/Interpreters/CrossToInnerJoinVisitor.cpp | 2 +- src/Interpreters/IdentifierSemantic.cpp | 2 +- .../JoinToSubqueryTransformVisitor.cpp | 2 +- src/Interpreters/ProcessList.cpp | 1 - src/Parsers/ASTIdentifier.cpp | 8 +++--- src/Parsers/ASTIdentifier.h | 9 ++++--- src/Parsers/ExpressionElementParsers.cpp | 3 ++- src/Parsers/IAST_fwd.h | 26 +++++++++++++++++-- src/Parsers/IParser.h | 10 ++++--- src/Storages/ConstraintsDescription.cpp | 6 ++--- ...ergeTreeIndexHypothesisMergedCondition.cpp | 4 +-- .../MergeTreeIndexHypothesisMergedCondition.h | 2 +- src/Storages/PartitionedSink.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 4 +-- 16 files changed, 59 insertions(+), 31 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f33744e918d..fa24fe4e57f 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -382,6 +382,7 @@ if (TARGET ch_contrib::cpuid) endif() dbms_target_link_libraries(PUBLIC ch_contrib::abseil_swiss_tables) +target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::abseil_swiss_tables) # Make dbms depend on roaring instead of clickhouse_common_io so that roaring itself can depend on clickhouse_common_io # That way we we can redirect malloc/free functions avoiding circular dependencies diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 3891fbf51cf..996526b60df 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -17,7 +17,7 @@ class ComparisonGraph { public: /// atomic_formulas are extracted from constraints. - explicit ComparisonGraph(const std::vector & atomic_formulas); + explicit ComparisonGraph(const ASTs & atomic_formulas); enum class CompareResult { @@ -43,7 +43,7 @@ public: bool isAlwaysCompare(CompareResult expected, const ASTPtr & left, const ASTPtr & right) const; /// Returns all expressions from component to which @ast belongs if any. - std::vector getEqual(const ASTPtr & ast) const; + ASTs getEqual(const ASTPtr & ast) const; /// Returns constant expression from component to which @ast belongs if any. std::optional getEqualConst(const ASTPtr & ast) const; @@ -52,7 +52,7 @@ public: std::optional getComponentId(const ASTPtr & ast) const; /// Returns all expressions from component. - std::vector getComponent(size_t id) const; + ASTs getComponent(size_t id) const; size_t getNumOfComponents() const { return graph.vertices.size(); } @@ -72,7 +72,7 @@ private: struct EqualComponent { /// All these expressions are considered as equal. - std::vector asts; + ASTs asts; std::optional constant_index; bool hasConstant() const; diff --git a/src/Interpreters/CrossToInnerJoinVisitor.cpp b/src/Interpreters/CrossToInnerJoinVisitor.cpp index bafa63e767f..09aebf874be 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -149,7 +149,7 @@ ASTPtr makeOnExpression(const std::vector & expressions) if (expressions.size() == 1) return expressions[0]->clone(); - std::vector arguments; + ASTs arguments; arguments.reserve(expressions.size()); for (const auto & ast : expressions) arguments.emplace_back(ast->clone()); diff --git a/src/Interpreters/IdentifierSemantic.cpp b/src/Interpreters/IdentifierSemantic.cpp index d3750e98b8c..0aa70057794 100644 --- a/src/Interpreters/IdentifierSemantic.cpp +++ b/src/Interpreters/IdentifierSemantic.cpp @@ -348,7 +348,7 @@ void splitConjunctionsAst(const ASTPtr & node, ASTs & result) ASTs splitConjunctionsAst(const ASTPtr & node) { - std::vector result; + ASTs result; splitConjunctionsAst(node, result); return result; } diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 8e515caace4..d033ea46370 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -209,7 +209,7 @@ struct RewriteTablesVisitorData { if (done) return; - std::vector new_tables{left, right}; + ASTs new_tables{left, right}; ast->children.swap(new_tables); done = true; } diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index d5194a02513..e22227f0602 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -218,7 +218,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_temp_data_on_disk, settings.max_temporary_data_on_disk_size_for_query)); } thread_group->query = query_; - thread_group->one_line_query = toOneLineQuery(query_); thread_group->normalized_query_hash = normalizedQueryHash(query_); /// Set query-level memory trackers diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index ca8ac0e8e64..4364c88c571 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -24,7 +24,7 @@ ASTIdentifier::ASTIdentifier(const String & short_name, ASTPtr && name_param) children.push_back(std::move(name_param)); } -ASTIdentifier::ASTIdentifier(std::vector && name_parts_, bool special, std::vector && name_params) +ASTIdentifier::ASTIdentifier(std::vector && name_parts_, bool special, ASTs && name_params) : name_parts(name_parts_), semantic(std::make_shared()) { assert(!name_parts.empty()); @@ -163,12 +163,12 @@ void ASTIdentifier::resetFullName() full_name += '.' + name_parts[i]; } -ASTTableIdentifier::ASTTableIdentifier(const String & table_name, std::vector && name_params) +ASTTableIdentifier::ASTTableIdentifier(const String & table_name, ASTs && name_params) : ASTIdentifier({table_name}, true, std::move(name_params)) { } -ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, std::vector && name_params) +ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, ASTs && name_params) : ASTIdentifier( table_id.database_name.empty() ? std::vector{table_id.table_name} : std::vector{table_id.database_name, table_id.table_name}, @@ -177,7 +177,7 @@ ASTTableIdentifier::ASTTableIdentifier(const StorageID & table_id, std::vector && name_params) +ASTTableIdentifier::ASTTableIdentifier(const String & database_name, const String & table_name, ASTs && name_params) : ASTIdentifier({database_name, table_name}, true, std::move(name_params)) { } diff --git a/src/Parsers/ASTIdentifier.h b/src/Parsers/ASTIdentifier.h index 14e2fcef39d..463575f80c0 100644 --- a/src/Parsers/ASTIdentifier.h +++ b/src/Parsers/ASTIdentifier.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -23,7 +24,7 @@ class ASTIdentifier : public ASTWithAlias { public: explicit ASTIdentifier(const String & short_name, ASTPtr && name_param = {}); - explicit ASTIdentifier(std::vector && name_parts, bool special = false, std::vector && name_params = {}); + explicit ASTIdentifier(std::vector && name_parts, bool special = false, ASTs && name_params = {}); /** Get the text that identifies this element. */ String getID(char delim) const override { return "Identifier" + (delim + name()); } @@ -71,9 +72,9 @@ private: class ASTTableIdentifier : public ASTIdentifier { public: - explicit ASTTableIdentifier(const String & table_name, std::vector && name_params = {}); - explicit ASTTableIdentifier(const StorageID & table_id, std::vector && name_params = {}); - ASTTableIdentifier(const String & database_name, const String & table_name, std::vector && name_params = {}); + explicit ASTTableIdentifier(const String & table_name, ASTs && name_params = {}); + explicit ASTTableIdentifier(const StorageID & table_id, ASTs && name_params = {}); + ASTTableIdentifier(const String & database_name, const String & table_name, ASTs && name_params = {}); String getID(char delim) const override { return "TableIdentifier" + (delim + name()); } ASTPtr clone() const override; diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index 88784329ece..1890168ac73 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include @@ -223,7 +224,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; std::vector parts; - std::vector params; + ASTs params; const auto & list = id_list->as(); for (const auto & child : list.children) { diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 18ba79d6618..611fafa46fd 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -1,13 +1,35 @@ #pragma once +#include #include -#include namespace DB { class IAST; using ASTPtr = std::shared_ptr; -using ASTs = std::vector; +/// sizeof(absl::InlinedVector) == 8 + N * 16. +/// 7 elements take 120 Bytes which is ~128 +using ASTs = absl::InlinedVector; + +} + +namespace std +{ + +inline typename DB::ASTs::size_type erase(DB::ASTs asts, const DB::ASTPtr & element) +{ + auto old_size = asts.size(); + asts.erase(std::remove(asts.begin(), asts.end(), element), asts.end()); + return old_size - asts.size(); +} + +template +inline typename DB::ASTs::size_type erase_if(DB::ASTs asts, Predicate pred) +{ + auto old_size = asts.size(); + asts.erase(std::remove_if(asts.begin(), asts.end(), pred), asts.end()); + return old_size - asts.size(); +} } diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 4e6dbca15a6..36fff297dee 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -1,7 +1,7 @@ #pragma once +#include #include -#include #include #include @@ -25,7 +25,7 @@ namespace ErrorCodes struct Expected { const char * max_parsed_pos = nullptr; - std::vector variants; + absl::InlinedVector variants; /// 'description' should be statically allocated string. ALWAYS_INLINE void add(const char * current_pos, const char * description) @@ -38,8 +38,12 @@ struct Expected return; } - if ((current_pos == max_parsed_pos) && (find(variants.begin(), variants.end(), description) == variants.end())) + if (current_pos == max_parsed_pos) + { + for (auto it = variants.begin(); it != variants.end(); it++) + if (*it == description) break; variants.push_back(description); + } } ALWAYS_INLINE void add(TokenIterator it, const char * description) diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index f73a148ad07..96037b46e52 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -107,7 +107,7 @@ std::unique_ptr ConstraintsDescription::buildGraph() const { static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; - std::vector constraints_for_graph; + ASTs constraints_for_graph; auto atomic_formulas = getAtomicConstraintData(); for (const auto & atomic_formula : atomic_formulas) { @@ -153,7 +153,7 @@ const std::vector> & ConstraintsDescription return cnf_constraints; } -const std::vector & ConstraintsDescription::getConstraints() const +const ASTs & ConstraintsDescription::getConstraints() const { return constraints; } @@ -218,7 +218,7 @@ void ConstraintsDescription::update() { cnf_constraints.clear(); ast_to_atom_ids.clear(); - graph = std::make_unique(std::vector()); + graph = std::make_unique(ASTs()); return; } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp index 3a145c0f505..c62b5e86c75 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp @@ -88,7 +88,7 @@ void MergeTreeIndexhypothesisMergedCondition::addConstraints(const ConstraintsDe /// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const { - std::vector active_atomic_formulas(atomic_constraints); + ASTs active_atomic_formulas(atomic_constraints); for (const auto & hypothesis : index_to_compare_atomic_hypotheses) { active_atomic_formulas.insert( @@ -190,7 +190,7 @@ bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTree std::unique_ptr MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector & values) const { - std::vector active_atomic_formulas(atomic_constraints); + ASTs active_atomic_formulas(atomic_constraints); for (size_t i = 0; i < values.size(); ++i) { if (values[i]) diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h index 9ebcbe9d7dc..6153c214898 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h @@ -34,7 +34,7 @@ private: std::vector> index_to_compare_atomic_hypotheses; std::vector> index_to_atomic_hypotheses; - std::vector atomic_constraints; + ASTs atomic_constraints; }; } diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 027e4f1f306..363b4557290 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -33,7 +33,7 @@ PartitionedSink::PartitionedSink( , context(context_) , sample_block(sample_block_) { - std::vector arguments(1, partition_by); + ASTs arguments(1, partition_by); ASTPtr partition_by_string = makeASTFunction(FunctionToString::name, std::move(arguments)); auto syntax_result = TreeRewriter(context).analyze(partition_by_string, sample_block.getNamesAndTypesList()); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 43e31b8e4f4..f186ce8f624 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -51,7 +51,7 @@ bool isValidFunction(const ASTPtr & expression, const std::function & is_constant, std::vector & result) +bool extractFunctions(const ASTPtr & expression, const std::function & is_constant, ASTs & result) { const auto * function = expression->as(); if (function && (function->name == "and" || function->name == "indexHint")) @@ -173,7 +173,7 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, ContextPtr context, Block }; /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. - std::vector functions; + ASTs functions; if (select.where()) unmodified &= extractFunctions(select.where(), is_constant, functions); if (select.prewhere()) From 74c61671e607ed8bc76ef3dc0aa59e0a09c15eeb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Oct 2022 22:19:25 +0200 Subject: [PATCH 003/339] Fix bug --- src/Parsers/IParser.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 36fff297dee..507961f573d 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -24,8 +24,8 @@ namespace ErrorCodes */ struct Expected { - const char * max_parsed_pos = nullptr; absl::InlinedVector variants; + const char * max_parsed_pos = nullptr; /// 'description' should be statically allocated string. ALWAYS_INLINE void add(const char * current_pos, const char * description) @@ -41,7 +41,8 @@ struct Expected if (current_pos == max_parsed_pos) { for (auto it = variants.begin(); it != variants.end(); it++) - if (*it == description) break; + if (*it == description) + return; variants.push_back(description); } } From 7a943ef7c43213bf2607ebd38f1c1dfa25152ef4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 13 Oct 2022 23:29:27 +0200 Subject: [PATCH 004/339] Sometimes I hate C++ --- src/Parsers/IAST_fwd.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 611fafa46fd..7c5174342ed 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -17,7 +17,7 @@ using ASTs = absl::InlinedVector; namespace std { -inline typename DB::ASTs::size_type erase(DB::ASTs asts, const DB::ASTPtr & element) +inline typename DB::ASTs::size_type erase(DB::ASTs & asts, const DB::ASTPtr & element) { auto old_size = asts.size(); asts.erase(std::remove(asts.begin(), asts.end(), element), asts.end()); @@ -25,7 +25,7 @@ inline typename DB::ASTs::size_type erase(DB::ASTs asts, const DB::ASTPtr & elem } template -inline typename DB::ASTs::size_type erase_if(DB::ASTs asts, Predicate pred) +inline typename DB::ASTs::size_type erase_if(DB::ASTs & asts, Predicate pred) { auto old_size = asts.size(); asts.erase(std::remove_if(asts.begin(), asts.end(), pred), asts.end()); From 46616d341c97d358a8a4237ca6d4a7f75c28c3d6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Oct 2022 14:06:44 +0200 Subject: [PATCH 005/339] Make explain_ast even larger --- tests/performance/explain_ast.xml | 5905 ++++++++++++++++++++++++++++- 1 file changed, 5903 insertions(+), 2 deletions(-) diff --git a/tests/performance/explain_ast.xml b/tests/performance/explain_ast.xml index 0daa748de83..5bcdd96c10e 100644 --- a/tests/performance/explain_ast.xml +++ b/tests/performance/explain_ast.xml @@ -1,6 +1,6 @@ - - + + 0, c3, NULL)) AS c3_q, +quantiles(0.25, 0.5, 0.75)(if(c4 > 0, c4, NULL)) AS c4_q, +quantiles(0.25, 0.5, 0.75)(t.c17 / t.c19) AS c5_q, +quantiles(0.25, 0.5, 0.75)(c6) AS c6_q, +quantiles(0.25, 0.5, 0.75)(c7) AS c7_q, +quantiles(0.25, 0.5, 0.75)(c8) AS c8_q, +quantiles(0.25, 0.5, 0.75)(c9) AS c9_q, +quantiles(0.25, 0.5, 0.75)(c10) AS c10_q, +quantiles(0.25, 0.5, 0.75)(c11) AS c11_q, +quantiles(0.25, 0.5, 0.75)(c12) AS c12_q, +quantiles(0.25, 0.5, 0.75)(c13) AS c13_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_q, +quantiles(0.25, 0.5, 0.75)(t.c16) AS c16_q, +quantiles(0.25, 0.5, 0.75)(t.c17) AS c17_q, +quantiles(0.25, 0.5, 0.75)(if(t.c18 > 0, t.c18, NULL)) AS c18_q, +max(if(c3 > 0, c3, NULL)) AS c3_max, +min(if(c3 > 0, c3, NULL)) AS c3_min, +avg(if(c3 > 0, c3, NULL)) AS c3_avg, +max(if(c4 > 0, c4, NULL)) AS c4_max, +min(if(c4 > 0, c4, NULL)) AS c4_min, +avg(if(c4 > 0, c4, NULL)) AS c4_avg, +max(t.c17 / t.c19) AS c5_max, +min(t.c17 / t.c19) AS c5_min, +avg(t.c17 / t.c19) AS c5_avg, +max(if(c6 > 0, c6, NULL)) AS c6_max, +min(if(c6 > 0, c6, NULL)) AS c6_min, +avg(if(c6 > 0, c6, NULL)) AS c6_avg, +max(if(c7 > 0, c7, NULL)) AS c7_max, +min(if(c7 > 0, c7, NULL)) AS c7_min, +avg(if(c7 > 0, c7, NULL)) AS c7_avg, +max(if(c10 > 0, c10, NULL)) AS c10_max, +min(if(c10 > 0, c10, NULL)) AS c10_min, +avg(if(c10 > 0, c10, NULL)) AS c10_avg, +max(if(c8 > 0, c8, NULL)) AS c8_max, +min(if(c8 > 0, c8, NULL)) AS c8_min, +avg(if(c8 > 0, c8, NULL)) AS c8_avg, +max(if(c9 > 0, c9, NULL)) AS c9_max, +min(if(c9 > 0, c9, NULL)) AS c9_min, +avg(if(c9 > 0, c9, NULL)) AS c9_avg, +max(if(c11 > 0, c11, NULL)) AS c11_max, +min(if(c11 > 0, c11, NULL)) AS c11_min, +avg(if(c11 > 0, c11, NULL)) AS c11_avg, +max(if(c12 > 0, c12, NULL)) AS c12_max, +min(if(c12 > 0, c12, NULL)) AS c12_min, +avg(if(c12 > 0, c12, NULL)) AS c12_avg, +max(if(c13 > 0, c13, NULL)) AS c13_max, +min(if(c13 > 0, c13, NULL)) AS c13_min, +avg(if(c13 > 0, c13, NULL)) AS c13_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_avg, +max(t.c16) AS c16_max, +min(t.c16) AS c16_min, +avg(t.c16) AS c16_avg, +max(t.c17) AS c17_max, +min(t.c17) AS c17_min, +avg(t.c17) AS c17_avg, +max(if(t.c18 > 0, t.c18, NULL)) AS c18_max, +min(if(t.c18 > 0, t.c18, NULL)) AS c18_min, +avg(if(t.c18 > 0, t.c18, NULL)) AS c18_avg, +sum(t.c19) AS c19, +sum(if(t.c18 > 0, t.c18, NULL)) AS c18, +sum(t.c16) AS c16, +sum(c23) AS c23, +sum(t.c17) AS c17, +sum(if(t.c24 > 0, t.c24, NULL)) AS c24, +c24 / c19 AS c14, +c24 / c17 AS c15, +median(if(isNotNull(c29) AND (t.c22 > 0), c13 * (t.c22 / c29), NULL)) AS c21, +sum(c22) AS c22 +FROM +( +SELECT +c27, +c39 AS c1, +c29, +c19, +c23, +c17, +c16, +c18, +c22, +c24, +c3, +c4, +c8, +c9, +c10, +c11, +c12, +c13, +c6, +c7 +FROM +( +SELECT +c27, +uniqExact(c30, c31) AS c19, +uniqExact(c30, c31, c32) AS c23, +uniqExactIf(c30, c31, c33 IN ('c37', 'c38')) AS c17, +countIf(c33 IN ('c37', 'c38')) AS c16, +countIf(c33 = 'c39') AS c18, +coalesce(sumIf(c29, c33 = 'c39'), 0) AS c22, +coalesce(sumIf(c37, c33 = 'c39'), 0) AS c24, +if((c18 > 0) AND (c19 > 0), c18 / c19, NULL) AS c3, +if(c17 != 0, c18 / c17, NULL) AS c4, +coalesce(avgIf(c34, (c34 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c8, +coalesce(avgIf(c35, (c35 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c9, +coalesce(avgIf(c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c10, +coalesce(avgIf(c35, (c35 > 0) AND (c33 = 'c39')), NULL) AS c11, +coalesce(avgIf(c37, c33 = 'c39'), NULL) AS c12, +coalesce(avgIf(c37 / c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c13, +coalesce(avgIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c6, +coalesce(minIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38')) AND (c37 > (c36 / 2))), NULL) AS c7 +FROM +( +SELECT +c27, +c30, +c32, +c31, +NULL AS c29, +NULL AS c33, +NULL AS c37, +NULL AS c34, +NULL AS c35 +FROM +( +SELECT +c27, +c30, +c32, +c31 +FROM database.t1 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE c61 = 0 +) AS table25 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c37' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table24 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table23 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c39' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table22 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table21 +) AS table20 +ALL LEFT JOIN +( +SELECT +c27, +avgMerge(avg_c37) * joinGet('database.table18', 'c60', concat('USD', '_', 'CH')) AS c36 +FROM database.table19 +PREWHERE c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) +WHERE date > (now() - toIntervalMonth(3)) +GROUP BY c27 +) AS table17 USING (c27) +GROUP BY c27 +) AS table16 +ALL LEFT JOIN +( +SELECT +comp_c27 AS c27, +assumeNotNull(c39) AS c39, +c29 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) USING (c27) +) AS t +ALL LEFT JOIN +( +SELECT +c1, +c2 +FROM +( +SELECT +c39 AS c1, +groupArray(comp_c27) AS c49, +multiIf(c1 = 'c58', if(length(c49) <= 2, 0, 1), c1 = 'c57', 1, if(length(c49) <= 3, 0, 1)) AS c2 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +GROUP BY c39 +) AS table3 +) USING (c1) +GROUP BY +c1, +c2 +) AS table2 +ORDER BY c1 ASC +) AS table1 +UNION ALL +SELECT * +FROM +( +SELECT +c1, +c2, +c3_q[1] AS c3_q1, +c3_q[3] AS c3_q3, +c3_q[2] AS c3_median, +least(c3_max, c3_q3 + (1.5 * (c3_q3 - c3_q1))) AS c3_max, +greatest(c3_min, c3_q1 - (1.5 * (c3_q3 - c3_q1))) AS c3_min, +c3_avg, +c4_q[1] AS c4_q1, +c4_q[3] AS c4_q3, +c4_q[2] AS c4_median, +least(c4_max, c4_q3 + (1.5 * (c4_q3 - c4_q1))) AS c4_max, +greatest(c4_min, c4_q1 - (1.5 * (c4_q3 - c4_q1))) AS c4_min, +c4_avg, +c5_q[1] AS c5_q1, +c5_q[3] AS c5_q3, +c5_q[2] AS c5_median, +least(c5_max, c5_q3 + (1.5 * (c5_q3 - c5_q1))) AS c5_max, +greatest(c5_min, c5_q1 - (1.5 * (c5_q3 - c5_q1))) AS c5_min, +c5_avg, +c6_q[1] AS c6_q1, +c6_q[3] AS c6_q3, +c6_q[2] AS c6_median, +least(c6_max, c6_q3 + (1.5 * (c6_q3 - c6_q1))) AS c6_max, +greatest(c6_min, c6_q1 - (1.5 * (c6_q3 - c6_q1))) AS c6_min, +c6_avg, +c7_q[1] AS c7_q1, +c7_q[3] AS c7_q3, +c7_q[2] AS c7_median, +least(c7_max, c7_q3 + (1.5 * (c7_q3 - c7_q1))) AS c7_max, +greatest(c7_min, c7_q1 - (1.5 * (c7_q3 - c7_q1))) AS c7_min, +c7_avg, +c8_q[1] AS c8_q1, +c8_q[3] AS c8_q3, +c8_q[2] AS c8_median, +least(c8_max, c8_q3 + (1.5 * (c8_q3 - c8_q1))) AS c8_max, +greatest(c8_min, c8_q1 - (1.5 * (c8_q3 - c8_q1))) AS c8_min, +c8_avg, +c9_q[1] AS c9_q1, +c9_q[3] AS c9_q3, +c9_q[2] AS c9_median, +least(c9_max, c9_q3 + (1.5 * (c9_q3 - c9_q1))) AS c9_max, +greatest(c9_min, c9_q1 - (1.5 * (c9_q3 - c9_q1))) AS c9_min, +c9_avg, +c10_q[1] AS c10_q1, +c10_q[3] AS c10_q3, +c10_q[2] AS c10_median, +least(c10_max, c10_q3 + (1.5 * (c10_q3 - c10_q1))) AS c10_max, +greatest(c10_min, c10_q1 - (1.5 * (c10_q3 - c10_q1))) AS c10_min, +c10_avg, +c10_avg, +c11_q[1] AS c11_q1, +c11_q[3] AS c11_q3, +c11_q[2] AS c11_median, +least(c11_max, c11_q3 + (1.5 * (c11_q3 - c11_q1))) AS c11_max, +greatest(c11_min, c11_q1 - (1.5 * (c11_q3 - c11_q1))) AS c11_min, +c11_avg, +c12_q[1] AS c12_q1, +c12_q[3] AS c12_q3, +c12_q[2] AS c12_median, +least(c12_max, c12_q3 + (1.5 * (c12_q3 - c12_q1))) AS c12_max, +greatest(c12_min, c12_q1 - (1.5 * (c12_q3 - c12_q1))) AS c12_min, +c12_avg, +c13_q[1] AS c13_q1, +c13_q[3] AS c13_q3, +c13_q[2] AS c13_median, +least(c13_max, c13_q3 + (1.5 * (c13_q3 - c13_q1))) AS c13_max, +greatest(c13_min, c13_q1 - (1.5 * (c13_q3 - c13_q1))) AS c13_min, +c13_avg, +c14_q[1] AS c14_q1, +c14_q[3] AS c14_q3, +c14_q[2] AS c14_median, +least(c14_max, c14_q3 + (1.5 * (c14_q3 - c14_q1))) AS c14_max, +greatest(c14_min, c14_q1 - (1.5 * (c14_q3 - c14_q1))) AS c14_min, +c14_avg, +c15_q[1] AS c15_q1, +c15_q[3] AS c15_q3, +c15_q[2] AS c15_median, +least(c15_max, c15_q3 + (1.5 * (c15_q3 - c15_q1))) AS c15_max, +greatest(c15_min, c15_q1 - (1.5 * (c15_q3 - c15_q1))) AS c15_min, +c15_avg, +c16_q[1] AS c16_q1, +c16_q[3] AS c16_q3, +c16_q[2] AS c16_median, +least(toFloat64(c16_max), c16_q3 + (1.5 * (c16_q3 - c16_q1))) AS c16_max, +greatest(toFloat64(c16_min), c16_q1 - (1.5 * (c16_q3 - c16_q1))) AS c16_min, +c16_avg, +c17_q[1] AS c17_q1, +c17_q[3] AS c17_q3, +c17_q[2] AS c17_median, +least(toFloat64(c17_max), c17_q3 + (1.5 * (c17_q3 - c17_q1))) AS c17_max, +greatest(toFloat64(c17_min), c17_q1 - (1.5 * (c17_q3 - c17_q1))) AS c17_min, +c17_avg, +c18_q[1] AS c18_q1, +c18_q[3] AS c18_q3, +c18_q[2] AS c18_median, +least(toFloat64(c18_max), c18_q3 + (1.5 * (c18_q3 - c18_q1))) AS c18_max, +greatest(toFloat64(c18_min), c18_q1 - (1.5 * (c18_q3 - c18_q1))) AS c18_min, +c18_avg, +round(if(c19 != 0, c24 / c19, 0), 2) AS c20, +c21, +c22, +c23 AS c23, +c19 AS c19, +c16 AS c16, +c17 AS c17, +c18 AS c18, +round(c24, 2) AS c24, +round(if(c17 != 0, c24 / c17, 0), 2) AS c25, +'CH' AS c26 +FROM +( +SELECT +c1, +c2, +groupUniqArray(c27) AS c28, +groupUniqArrayIf(c27, isNotNull(c29)) AS c28_with_c29, +quantiles(0.25, 0.5, 0.75)(if(c3 > 0, c3, NULL)) AS c3_q, +quantiles(0.25, 0.5, 0.75)(if(c4 > 0, c4, NULL)) AS c4_q, +quantiles(0.25, 0.5, 0.75)(t.c17 / t.c19) AS c5_q, +quantiles(0.25, 0.5, 0.75)(c6) AS c6_q, +quantiles(0.25, 0.5, 0.75)(c7) AS c7_q, +quantiles(0.25, 0.5, 0.75)(c8) AS c8_q, +quantiles(0.25, 0.5, 0.75)(c9) AS c9_q, +quantiles(0.25, 0.5, 0.75)(c10) AS c10_q, +quantiles(0.25, 0.5, 0.75)(c11) AS c11_q, +quantiles(0.25, 0.5, 0.75)(c12) AS c12_q, +quantiles(0.25, 0.5, 0.75)(c13) AS c13_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_q, +quantiles(0.25, 0.5, 0.75)(t.c16) AS c16_q, +quantiles(0.25, 0.5, 0.75)(t.c17) AS c17_q, +quantiles(0.25, 0.5, 0.75)(if(t.c18 > 0, t.c18, NULL)) AS c18_q, +max(if(c3 > 0, c3, NULL)) AS c3_max, +min(if(c3 > 0, c3, NULL)) AS c3_min, +avg(if(c3 > 0, c3, NULL)) AS c3_avg, +max(if(c4 > 0, c4, NULL)) AS c4_max, +min(if(c4 > 0, c4, NULL)) AS c4_min, +avg(if(c4 > 0, c4, NULL)) AS c4_avg, +max(t.c17 / t.c19) AS c5_max, +min(t.c17 / t.c19) AS c5_min, +avg(t.c17 / t.c19) AS c5_avg, +max(if(c6 > 0, c6, NULL)) AS c6_max, +min(if(c6 > 0, c6, NULL)) AS c6_min, +avg(if(c6 > 0, c6, NULL)) AS c6_avg, +max(if(c7 > 0, c7, NULL)) AS c7_max, +min(if(c7 > 0, c7, NULL)) AS c7_min, +avg(if(c7 > 0, c7, NULL)) AS c7_avg, +max(if(c10 > 0, c10, NULL)) AS c10_max, +min(if(c10 > 0, c10, NULL)) AS c10_min, +avg(if(c10 > 0, c10, NULL)) AS c10_avg, +max(if(c8 > 0, c8, NULL)) AS c8_max, +min(if(c8 > 0, c8, NULL)) AS c8_min, +avg(if(c8 > 0, c8, NULL)) AS c8_avg, +max(if(c9 > 0, c9, NULL)) AS c9_max, +min(if(c9 > 0, c9, NULL)) AS c9_min, +avg(if(c9 > 0, c9, NULL)) AS c9_avg, +max(if(c11 > 0, c11, NULL)) AS c11_max, +min(if(c11 > 0, c11, NULL)) AS c11_min, +avg(if(c11 > 0, c11, NULL)) AS c11_avg, +max(if(c12 > 0, c12, NULL)) AS c12_max, +min(if(c12 > 0, c12, NULL)) AS c12_min, +avg(if(c12 > 0, c12, NULL)) AS c12_avg, +max(if(c13 > 0, c13, NULL)) AS c13_max, +min(if(c13 > 0, c13, NULL)) AS c13_min, +avg(if(c13 > 0, c13, NULL)) AS c13_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_avg, +max(t.c16) AS c16_max, +min(t.c16) AS c16_min, +avg(t.c16) AS c16_avg, +max(t.c17) AS c17_max, +min(t.c17) AS c17_min, +avg(t.c17) AS c17_avg, +max(if(t.c18 > 0, t.c18, NULL)) AS c18_max, +min(if(t.c18 > 0, t.c18, NULL)) AS c18_min, +avg(if(t.c18 > 0, t.c18, NULL)) AS c18_avg, +sum(t.c19) AS c19, +sum(if(t.c18 > 0, t.c18, NULL)) AS c18, +sum(t.c16) AS c16, +sum(c23) AS c23, +sum(t.c17) AS c17, +sum(if(t.c24 > 0, t.c24, NULL)) AS c24, +c24 / c19 AS c14, +c24 / c17 AS c15, +median(if(isNotNull(c29) AND (t.c22 > 0), c13 * (t.c22 / c29), NULL)) AS c21, +sum(c22) AS c22 +FROM +( +SELECT +c27, +c39 AS c1, +c29, +c19, +c23, +c17, +c16, +c18, +c22, +c24, +c3, +c4, +c8, +c9, +c10, +c11, +c12, +c13, +c6, +c7 +FROM +( +SELECT +c27, +uniqExact(c30, c31) AS c19, +uniqExact(c30, c31, c32) AS c23, +uniqExactIf(c30, c31, c33 IN ('c37', 'c38')) AS c17, +countIf(c33 IN ('c37', 'c38')) AS c16, +countIf(c33 = 'c39') AS c18, +coalesce(sumIf(c29, c33 = 'c39'), 0) AS c22, +coalesce(sumIf(c37, c33 = 'c39'), 0) AS c24, +if((c18 > 0) AND (c19 > 0), c18 / c19, NULL) AS c3, +if(c17 != 0, c18 / c17, NULL) AS c4, +coalesce(avgIf(c34, (c34 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c8, +coalesce(avgIf(c35, (c35 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c9, +coalesce(avgIf(c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c10, +coalesce(avgIf(c35, (c35 > 0) AND (c33 = 'c39')), NULL) AS c11, +coalesce(avgIf(c37, c33 = 'c39'), NULL) AS c12, +coalesce(avgIf(c37 / c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c13, +coalesce(avgIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c6, +coalesce(minIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38')) AND (c37 > (c36 / 2))), NULL) AS c7 +FROM +( +SELECT +c27, +c30, +c32, +c31, +NULL AS c29, +NULL AS c33, +NULL AS c37, +NULL AS c34, +NULL AS c35 +FROM +( +SELECT +c27, +c30, +c32, +c31 +FROM database.t1 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE c61 = 0 +) AS table25 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c37' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table24 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table23 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c39' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table22 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table21 +) AS table20 +ALL LEFT JOIN +( +SELECT +c27, +avgMerge(avg_c37) * joinGet('database.table18', 'c60', concat('USD', '_', 'CH')) AS c36 +FROM database.table19 +PREWHERE c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) +WHERE date > (now() - toIntervalMonth(3)) +GROUP BY c27 +) AS table17 USING (c27) +GROUP BY c27 +) AS table16 +ALL LEFT JOIN +( +SELECT +comp_c27 AS c27, +assumeNotNull(c39) AS c39, +c29 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) USING (c27) +) AS t +ALL LEFT JOIN +( +SELECT +c1, +c2 +FROM +( +SELECT +c39 AS c1, +groupArray(comp_c27) AS c49, +multiIf(c1 = 'c58', if(length(c49) <= 2, 0, 1), c1 = 'c57', 1, if(length(c49) <= 3, 0, 1)) AS c2 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +GROUP BY c39 +) AS table3 +) USING (c1) +GROUP BY +c1, +c2 +) AS table2 +ORDER BY c1 ASC +) AS table1 +UNION ALL +SELECT * +FROM +( +SELECT +c1, +c2, +c3_q[1] AS c3_q1, +c3_q[3] AS c3_q3, +c3_q[2] AS c3_median, +least(c3_max, c3_q3 + (1.5 * (c3_q3 - c3_q1))) AS c3_max, +greatest(c3_min, c3_q1 - (1.5 * (c3_q3 - c3_q1))) AS c3_min, +c3_avg, +c4_q[1] AS c4_q1, +c4_q[3] AS c4_q3, +c4_q[2] AS c4_median, +least(c4_max, c4_q3 + (1.5 * (c4_q3 - c4_q1))) AS c4_max, +greatest(c4_min, c4_q1 - (1.5 * (c4_q3 - c4_q1))) AS c4_min, +c4_avg, +c5_q[1] AS c5_q1, +c5_q[3] AS c5_q3, +c5_q[2] AS c5_median, +least(c5_max, c5_q3 + (1.5 * (c5_q3 - c5_q1))) AS c5_max, +greatest(c5_min, c5_q1 - (1.5 * (c5_q3 - c5_q1))) AS c5_min, +c5_avg, +c6_q[1] AS c6_q1, +c6_q[3] AS c6_q3, +c6_q[2] AS c6_median, +least(c6_max, c6_q3 + (1.5 * (c6_q3 - c6_q1))) AS c6_max, +greatest(c6_min, c6_q1 - (1.5 * (c6_q3 - c6_q1))) AS c6_min, +c6_avg, +c7_q[1] AS c7_q1, +c7_q[3] AS c7_q3, +c7_q[2] AS c7_median, +least(c7_max, c7_q3 + (1.5 * (c7_q3 - c7_q1))) AS c7_max, +greatest(c7_min, c7_q1 - (1.5 * (c7_q3 - c7_q1))) AS c7_min, +c7_avg, +c8_q[1] AS c8_q1, +c8_q[3] AS c8_q3, +c8_q[2] AS c8_median, +least(c8_max, c8_q3 + (1.5 * (c8_q3 - c8_q1))) AS c8_max, +greatest(c8_min, c8_q1 - (1.5 * (c8_q3 - c8_q1))) AS c8_min, +c8_avg, +c9_q[1] AS c9_q1, +c9_q[3] AS c9_q3, +c9_q[2] AS c9_median, +least(c9_max, c9_q3 + (1.5 * (c9_q3 - c9_q1))) AS c9_max, +greatest(c9_min, c9_q1 - (1.5 * (c9_q3 - c9_q1))) AS c9_min, +c9_avg, +c10_q[1] AS c10_q1, +c10_q[3] AS c10_q3, +c10_q[2] AS c10_median, +least(c10_max, c10_q3 + (1.5 * (c10_q3 - c10_q1))) AS c10_max, +greatest(c10_min, c10_q1 - (1.5 * (c10_q3 - c10_q1))) AS c10_min, +c10_avg, +c10_avg, +c11_q[1] AS c11_q1, +c11_q[3] AS c11_q3, +c11_q[2] AS c11_median, +least(c11_max, c11_q3 + (1.5 * (c11_q3 - c11_q1))) AS c11_max, +greatest(c11_min, c11_q1 - (1.5 * (c11_q3 - c11_q1))) AS c11_min, +c11_avg, +c12_q[1] AS c12_q1, +c12_q[3] AS c12_q3, +c12_q[2] AS c12_median, +least(c12_max, c12_q3 + (1.5 * (c12_q3 - c12_q1))) AS c12_max, +greatest(c12_min, c12_q1 - (1.5 * (c12_q3 - c12_q1))) AS c12_min, +c12_avg, +c13_q[1] AS c13_q1, +c13_q[3] AS c13_q3, +c13_q[2] AS c13_median, +least(c13_max, c13_q3 + (1.5 * (c13_q3 - c13_q1))) AS c13_max, +greatest(c13_min, c13_q1 - (1.5 * (c13_q3 - c13_q1))) AS c13_min, +c13_avg, +c14_q[1] AS c14_q1, +c14_q[3] AS c14_q3, +c14_q[2] AS c14_median, +least(c14_max, c14_q3 + (1.5 * (c14_q3 - c14_q1))) AS c14_max, +greatest(c14_min, c14_q1 - (1.5 * (c14_q3 - c14_q1))) AS c14_min, +c14_avg, +c15_q[1] AS c15_q1, +c15_q[3] AS c15_q3, +c15_q[2] AS c15_median, +least(c15_max, c15_q3 + (1.5 * (c15_q3 - c15_q1))) AS c15_max, +greatest(c15_min, c15_q1 - (1.5 * (c15_q3 - c15_q1))) AS c15_min, +c15_avg, +c16_q[1] AS c16_q1, +c16_q[3] AS c16_q3, +c16_q[2] AS c16_median, +least(toFloat64(c16_max), c16_q3 + (1.5 * (c16_q3 - c16_q1))) AS c16_max, +greatest(toFloat64(c16_min), c16_q1 - (1.5 * (c16_q3 - c16_q1))) AS c16_min, +c16_avg, +c17_q[1] AS c17_q1, +c17_q[3] AS c17_q3, +c17_q[2] AS c17_median, +least(toFloat64(c17_max), c17_q3 + (1.5 * (c17_q3 - c17_q1))) AS c17_max, +greatest(toFloat64(c17_min), c17_q1 - (1.5 * (c17_q3 - c17_q1))) AS c17_min, +c17_avg, +c18_q[1] AS c18_q1, +c18_q[3] AS c18_q3, +c18_q[2] AS c18_median, +least(toFloat64(c18_max), c18_q3 + (1.5 * (c18_q3 - c18_q1))) AS c18_max, +greatest(toFloat64(c18_min), c18_q1 - (1.5 * (c18_q3 - c18_q1))) AS c18_min, +c18_avg, +round(if(c19 != 0, c24 / c19, 0), 2) AS c20, +c21, +c22, +c23 AS c23, +c19 AS c19, +c16 AS c16, +c17 AS c17, +c18 AS c18, +round(c24, 2) AS c24, +round(if(c17 != 0, c24 / c17, 0), 2) AS c25, +'CH' AS c26 +FROM +( +SELECT +c1, +c2, +groupUniqArray(c27) AS c28, +groupUniqArrayIf(c27, isNotNull(c29)) AS c28_with_c29, +quantiles(0.25, 0.5, 0.75)(if(c3 > 0, c3, NULL)) AS c3_q, +quantiles(0.25, 0.5, 0.75)(if(c4 > 0, c4, NULL)) AS c4_q, +quantiles(0.25, 0.5, 0.75)(t.c17 / t.c19) AS c5_q, +quantiles(0.25, 0.5, 0.75)(c6) AS c6_q, +quantiles(0.25, 0.5, 0.75)(c7) AS c7_q, +quantiles(0.25, 0.5, 0.75)(c8) AS c8_q, +quantiles(0.25, 0.5, 0.75)(c9) AS c9_q, +quantiles(0.25, 0.5, 0.75)(c10) AS c10_q, +quantiles(0.25, 0.5, 0.75)(c11) AS c11_q, +quantiles(0.25, 0.5, 0.75)(c12) AS c12_q, +quantiles(0.25, 0.5, 0.75)(c13) AS c13_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_q, +quantiles(0.25, 0.5, 0.75)(t.c16) AS c16_q, +quantiles(0.25, 0.5, 0.75)(t.c17) AS c17_q, +quantiles(0.25, 0.5, 0.75)(if(t.c18 > 0, t.c18, NULL)) AS c18_q, +max(if(c3 > 0, c3, NULL)) AS c3_max, +min(if(c3 > 0, c3, NULL)) AS c3_min, +avg(if(c3 > 0, c3, NULL)) AS c3_avg, +max(if(c4 > 0, c4, NULL)) AS c4_max, +min(if(c4 > 0, c4, NULL)) AS c4_min, +avg(if(c4 > 0, c4, NULL)) AS c4_avg, +max(t.c17 / t.c19) AS c5_max, +min(t.c17 / t.c19) AS c5_min, +avg(t.c17 / t.c19) AS c5_avg, +max(if(c6 > 0, c6, NULL)) AS c6_max, +min(if(c6 > 0, c6, NULL)) AS c6_min, +avg(if(c6 > 0, c6, NULL)) AS c6_avg, +max(if(c7 > 0, c7, NULL)) AS c7_max, +min(if(c7 > 0, c7, NULL)) AS c7_min, +avg(if(c7 > 0, c7, NULL)) AS c7_avg, +max(if(c10 > 0, c10, NULL)) AS c10_max, +min(if(c10 > 0, c10, NULL)) AS c10_min, +avg(if(c10 > 0, c10, NULL)) AS c10_avg, +max(if(c8 > 0, c8, NULL)) AS c8_max, +min(if(c8 > 0, c8, NULL)) AS c8_min, +avg(if(c8 > 0, c8, NULL)) AS c8_avg, +max(if(c9 > 0, c9, NULL)) AS c9_max, +min(if(c9 > 0, c9, NULL)) AS c9_min, +avg(if(c9 > 0, c9, NULL)) AS c9_avg, +max(if(c11 > 0, c11, NULL)) AS c11_max, +min(if(c11 > 0, c11, NULL)) AS c11_min, +avg(if(c11 > 0, c11, NULL)) AS c11_avg, +max(if(c12 > 0, c12, NULL)) AS c12_max, +min(if(c12 > 0, c12, NULL)) AS c12_min, +avg(if(c12 > 0, c12, NULL)) AS c12_avg, +max(if(c13 > 0, c13, NULL)) AS c13_max, +min(if(c13 > 0, c13, NULL)) AS c13_min, +avg(if(c13 > 0, c13, NULL)) AS c13_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_avg, +max(t.c16) AS c16_max, +min(t.c16) AS c16_min, +avg(t.c16) AS c16_avg, +max(t.c17) AS c17_max, +min(t.c17) AS c17_min, +avg(t.c17) AS c17_avg, +max(if(t.c18 > 0, t.c18, NULL)) AS c18_max, +min(if(t.c18 > 0, t.c18, NULL)) AS c18_min, +avg(if(t.c18 > 0, t.c18, NULL)) AS c18_avg, +sum(t.c19) AS c19, +sum(if(t.c18 > 0, t.c18, NULL)) AS c18, +sum(t.c16) AS c16, +sum(c23) AS c23, +sum(t.c17) AS c17, +sum(if(t.c24 > 0, t.c24, NULL)) AS c24, +c24 / c19 AS c14, +c24 / c17 AS c15, +median(if(isNotNull(c29) AND (t.c22 > 0), c13 * (t.c22 / c29), NULL)) AS c21, +sum(c22) AS c22 +FROM +( +SELECT +c27, +c39 AS c1, +c29, +c19, +c23, +c17, +c16, +c18, +c22, +c24, +c3, +c4, +c8, +c9, +c10, +c11, +c12, +c13, +c6, +c7 +FROM +( +SELECT +c27, +uniqExact(c30, c31) AS c19, +uniqExact(c30, c31, c32) AS c23, +uniqExactIf(c30, c31, c33 IN ('c37', 'c38')) AS c17, +countIf(c33 IN ('c37', 'c38')) AS c16, +countIf(c33 = 'c39') AS c18, +coalesce(sumIf(c29, c33 = 'c39'), 0) AS c22, +coalesce(sumIf(c37, c33 = 'c39'), 0) AS c24, +if((c18 > 0) AND (c19 > 0), c18 / c19, NULL) AS c3, +if(c17 != 0, c18 / c17, NULL) AS c4, +coalesce(avgIf(c34, (c34 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c8, +coalesce(avgIf(c35, (c35 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c9, +coalesce(avgIf(c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c10, +coalesce(avgIf(c35, (c35 > 0) AND (c33 = 'c39')), NULL) AS c11, +coalesce(avgIf(c37, c33 = 'c39'), NULL) AS c12, +coalesce(avgIf(c37 / c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c13, +coalesce(avgIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c6, +coalesce(minIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38')) AND (c37 > (c36 / 2))), NULL) AS c7 +FROM +( +SELECT +c27, +c30, +c32, +c31, +NULL AS c29, +NULL AS c33, +NULL AS c37, +NULL AS c34, +NULL AS c35 +FROM +( +SELECT +c27, +c30, +c32, +c31 +FROM database.t1 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE c61 = 0 +) AS table25 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c37' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table24 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table23 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c39' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table22 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table21 +) AS table20 +ALL LEFT JOIN +( +SELECT +c27, +avgMerge(avg_c37) * joinGet('database.table18', 'c60', concat('USD', '_', 'CH')) AS c36 +FROM database.table19 +PREWHERE c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) +WHERE date > (now() - toIntervalMonth(3)) +GROUP BY c27 +) AS table17 USING (c27) +GROUP BY c27 +) AS table16 +ALL LEFT JOIN +( +SELECT +comp_c27 AS c27, +assumeNotNull(c39) AS c39, +c29 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) USING (c27) +) AS t +ALL LEFT JOIN +( +SELECT +c1, +c2 +FROM +( +SELECT +c39 AS c1, +groupArray(comp_c27) AS c49, +multiIf(c1 = 'c58', if(length(c49) <= 2, 0, 1), c1 = 'c57', 1, if(length(c49) <= 3, 0, 1)) AS c2 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +GROUP BY c39 +) AS table3 +) USING (c1) +GROUP BY +c1, +c2 +) AS table2 +ORDER BY c1 ASC +) AS table1 +UNION ALL +SELECT * +FROM +( +SELECT +c1, +c2, +c3_q[1] AS c3_q1, +c3_q[3] AS c3_q3, +c3_q[2] AS c3_median, +least(c3_max, c3_q3 + (1.5 * (c3_q3 - c3_q1))) AS c3_max, +greatest(c3_min, c3_q1 - (1.5 * (c3_q3 - c3_q1))) AS c3_min, +c3_avg, +c4_q[1] AS c4_q1, +c4_q[3] AS c4_q3, +c4_q[2] AS c4_median, +least(c4_max, c4_q3 + (1.5 * (c4_q3 - c4_q1))) AS c4_max, +greatest(c4_min, c4_q1 - (1.5 * (c4_q3 - c4_q1))) AS c4_min, +c4_avg, +c5_q[1] AS c5_q1, +c5_q[3] AS c5_q3, +c5_q[2] AS c5_median, +least(c5_max, c5_q3 + (1.5 * (c5_q3 - c5_q1))) AS c5_max, +greatest(c5_min, c5_q1 - (1.5 * (c5_q3 - c5_q1))) AS c5_min, +c5_avg, +c6_q[1] AS c6_q1, +c6_q[3] AS c6_q3, +c6_q[2] AS c6_median, +least(c6_max, c6_q3 + (1.5 * (c6_q3 - c6_q1))) AS c6_max, +greatest(c6_min, c6_q1 - (1.5 * (c6_q3 - c6_q1))) AS c6_min, +c6_avg, +c7_q[1] AS c7_q1, +c7_q[3] AS c7_q3, +c7_q[2] AS c7_median, +least(c7_max, c7_q3 + (1.5 * (c7_q3 - c7_q1))) AS c7_max, +greatest(c7_min, c7_q1 - (1.5 * (c7_q3 - c7_q1))) AS c7_min, +c7_avg, +c8_q[1] AS c8_q1, +c8_q[3] AS c8_q3, +c8_q[2] AS c8_median, +least(c8_max, c8_q3 + (1.5 * (c8_q3 - c8_q1))) AS c8_max, +greatest(c8_min, c8_q1 - (1.5 * (c8_q3 - c8_q1))) AS c8_min, +c8_avg, +c9_q[1] AS c9_q1, +c9_q[3] AS c9_q3, +c9_q[2] AS c9_median, +least(c9_max, c9_q3 + (1.5 * (c9_q3 - c9_q1))) AS c9_max, +greatest(c9_min, c9_q1 - (1.5 * (c9_q3 - c9_q1))) AS c9_min, +c9_avg, +c10_q[1] AS c10_q1, +c10_q[3] AS c10_q3, +c10_q[2] AS c10_median, +least(c10_max, c10_q3 + (1.5 * (c10_q3 - c10_q1))) AS c10_max, +greatest(c10_min, c10_q1 - (1.5 * (c10_q3 - c10_q1))) AS c10_min, +c10_avg, +c10_avg, +c11_q[1] AS c11_q1, +c11_q[3] AS c11_q3, +c11_q[2] AS c11_median, +least(c11_max, c11_q3 + (1.5 * (c11_q3 - c11_q1))) AS c11_max, +greatest(c11_min, c11_q1 - (1.5 * (c11_q3 - c11_q1))) AS c11_min, +c11_avg, +c12_q[1] AS c12_q1, +c12_q[3] AS c12_q3, +c12_q[2] AS c12_median, +least(c12_max, c12_q3 + (1.5 * (c12_q3 - c12_q1))) AS c12_max, +greatest(c12_min, c12_q1 - (1.5 * (c12_q3 - c12_q1))) AS c12_min, +c12_avg, +c13_q[1] AS c13_q1, +c13_q[3] AS c13_q3, +c13_q[2] AS c13_median, +least(c13_max, c13_q3 + (1.5 * (c13_q3 - c13_q1))) AS c13_max, +greatest(c13_min, c13_q1 - (1.5 * (c13_q3 - c13_q1))) AS c13_min, +c13_avg, +c14_q[1] AS c14_q1, +c14_q[3] AS c14_q3, +c14_q[2] AS c14_median, +least(c14_max, c14_q3 + (1.5 * (c14_q3 - c14_q1))) AS c14_max, +greatest(c14_min, c14_q1 - (1.5 * (c14_q3 - c14_q1))) AS c14_min, +c14_avg, +c15_q[1] AS c15_q1, +c15_q[3] AS c15_q3, +c15_q[2] AS c15_median, +least(c15_max, c15_q3 + (1.5 * (c15_q3 - c15_q1))) AS c15_max, +greatest(c15_min, c15_q1 - (1.5 * (c15_q3 - c15_q1))) AS c15_min, +c15_avg, +c16_q[1] AS c16_q1, +c16_q[3] AS c16_q3, +c16_q[2] AS c16_median, +least(toFloat64(c16_max), c16_q3 + (1.5 * (c16_q3 - c16_q1))) AS c16_max, +greatest(toFloat64(c16_min), c16_q1 - (1.5 * (c16_q3 - c16_q1))) AS c16_min, +c16_avg, +c17_q[1] AS c17_q1, +c17_q[3] AS c17_q3, +c17_q[2] AS c17_median, +least(toFloat64(c17_max), c17_q3 + (1.5 * (c17_q3 - c17_q1))) AS c17_max, +greatest(toFloat64(c17_min), c17_q1 - (1.5 * (c17_q3 - c17_q1))) AS c17_min, +c17_avg, +c18_q[1] AS c18_q1, +c18_q[3] AS c18_q3, +c18_q[2] AS c18_median, +least(toFloat64(c18_max), c18_q3 + (1.5 * (c18_q3 - c18_q1))) AS c18_max, +greatest(toFloat64(c18_min), c18_q1 - (1.5 * (c18_q3 - c18_q1))) AS c18_min, +c18_avg, +round(if(c19 != 0, c24 / c19, 0), 2) AS c20, +c21, +c22, +c23 AS c23, +c19 AS c19, +c16 AS c16, +c17 AS c17, +c18 AS c18, +round(c24, 2) AS c24, +round(if(c17 != 0, c24 / c17, 0), 2) AS c25, +'CH' AS c26 +FROM +( +SELECT +c1, +c2, +groupUniqArray(c27) AS c28, +groupUniqArrayIf(c27, isNotNull(c29)) AS c28_with_c29, +quantiles(0.25, 0.5, 0.75)(if(c3 > 0, c3, NULL)) AS c3_q, +quantiles(0.25, 0.5, 0.75)(if(c4 > 0, c4, NULL)) AS c4_q, +quantiles(0.25, 0.5, 0.75)(t.c17 / t.c19) AS c5_q, +quantiles(0.25, 0.5, 0.75)(c6) AS c6_q, +quantiles(0.25, 0.5, 0.75)(c7) AS c7_q, +quantiles(0.25, 0.5, 0.75)(c8) AS c8_q, +quantiles(0.25, 0.5, 0.75)(c9) AS c9_q, +quantiles(0.25, 0.5, 0.75)(c10) AS c10_q, +quantiles(0.25, 0.5, 0.75)(c11) AS c11_q, +quantiles(0.25, 0.5, 0.75)(c12) AS c12_q, +quantiles(0.25, 0.5, 0.75)(c13) AS c13_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_q, +quantiles(0.25, 0.5, 0.75)(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_q, +quantiles(0.25, 0.5, 0.75)(t.c16) AS c16_q, +quantiles(0.25, 0.5, 0.75)(t.c17) AS c17_q, +quantiles(0.25, 0.5, 0.75)(if(t.c18 > 0, t.c18, NULL)) AS c18_q, +max(if(c3 > 0, c3, NULL)) AS c3_max, +min(if(c3 > 0, c3, NULL)) AS c3_min, +avg(if(c3 > 0, c3, NULL)) AS c3_avg, +max(if(c4 > 0, c4, NULL)) AS c4_max, +min(if(c4 > 0, c4, NULL)) AS c4_min, +avg(if(c4 > 0, c4, NULL)) AS c4_avg, +max(t.c17 / t.c19) AS c5_max, +min(t.c17 / t.c19) AS c5_min, +avg(t.c17 / t.c19) AS c5_avg, +max(if(c6 > 0, c6, NULL)) AS c6_max, +min(if(c6 > 0, c6, NULL)) AS c6_min, +avg(if(c6 > 0, c6, NULL)) AS c6_avg, +max(if(c7 > 0, c7, NULL)) AS c7_max, +min(if(c7 > 0, c7, NULL)) AS c7_min, +avg(if(c7 > 0, c7, NULL)) AS c7_avg, +max(if(c10 > 0, c10, NULL)) AS c10_max, +min(if(c10 > 0, c10, NULL)) AS c10_min, +avg(if(c10 > 0, c10, NULL)) AS c10_avg, +max(if(c8 > 0, c8, NULL)) AS c8_max, +min(if(c8 > 0, c8, NULL)) AS c8_min, +avg(if(c8 > 0, c8, NULL)) AS c8_avg, +max(if(c9 > 0, c9, NULL)) AS c9_max, +min(if(c9 > 0, c9, NULL)) AS c9_min, +avg(if(c9 > 0, c9, NULL)) AS c9_avg, +max(if(c11 > 0, c11, NULL)) AS c11_max, +min(if(c11 > 0, c11, NULL)) AS c11_min, +avg(if(c11 > 0, c11, NULL)) AS c11_avg, +max(if(c12 > 0, c12, NULL)) AS c12_max, +min(if(c12 > 0, c12, NULL)) AS c12_min, +avg(if(c12 > 0, c12, NULL)) AS c12_avg, +max(if(c13 > 0, c13, NULL)) AS c13_max, +min(if(c13 > 0, c13, NULL)) AS c13_min, +avg(if(c13 > 0, c13, NULL)) AS c13_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c19) AS c14_avg, +max(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_max, +min(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_min, +avg(if(t.c24 > 0, t.c24, NULL) / t.c17) AS c15_avg, +max(t.c16) AS c16_max, +min(t.c16) AS c16_min, +avg(t.c16) AS c16_avg, +max(t.c17) AS c17_max, +min(t.c17) AS c17_min, +avg(t.c17) AS c17_avg, +max(if(t.c18 > 0, t.c18, NULL)) AS c18_max, +min(if(t.c18 > 0, t.c18, NULL)) AS c18_min, +avg(if(t.c18 > 0, t.c18, NULL)) AS c18_avg, +sum(t.c19) AS c19, +sum(if(t.c18 > 0, t.c18, NULL)) AS c18, +sum(t.c16) AS c16, +sum(c23) AS c23, +sum(t.c17) AS c17, +sum(if(t.c24 > 0, t.c24, NULL)) AS c24, +c24 / c19 AS c14, +c24 / c17 AS c15, +median(if(isNotNull(c29) AND (t.c22 > 0), c13 * (t.c22 / c29), NULL)) AS c21, +sum(c22) AS c22 +FROM +( +SELECT +c27, +c39 AS c1, +c29, +c19, +c23, +c17, +c16, +c18, +c22, +c24, +c3, +c4, +c8, +c9, +c10, +c11, +c12, +c13, +c6, +c7 +FROM +( +SELECT +c27, +uniqExact(c30, c31) AS c19, +uniqExact(c30, c31, c32) AS c23, +uniqExactIf(c30, c31, c33 IN ('c37', 'c38')) AS c17, +countIf(c33 IN ('c37', 'c38')) AS c16, +countIf(c33 = 'c39') AS c18, +coalesce(sumIf(c29, c33 = 'c39'), 0) AS c22, +coalesce(sumIf(c37, c33 = 'c39'), 0) AS c24, +if((c18 > 0) AND (c19 > 0), c18 / c19, NULL) AS c3, +if(c17 != 0, c18 / c17, NULL) AS c4, +coalesce(avgIf(c34, (c34 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c8, +coalesce(avgIf(c35, (c35 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c9, +coalesce(avgIf(c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c10, +coalesce(avgIf(c35, (c35 > 0) AND (c33 = 'c39')), NULL) AS c11, +coalesce(avgIf(c37, c33 = 'c39'), NULL) AS c12, +coalesce(avgIf(c37 / c34, (c34 > 0) AND (c33 = 'c39')), NULL) AS c13, +coalesce(avgIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38'))), NULL) AS c6, +coalesce(minIf(c37, (c37 > 0) AND (c33 IN ('c37', 'c38')) AND (c37 > (c36 / 2))), NULL) AS c7 +FROM +( +SELECT +c27, +c30, +c32, +c31, +NULL AS c29, +NULL AS c33, +NULL AS c37, +NULL AS c34, +NULL AS c35 +FROM +( +SELECT +c27, +c30, +c32, +c31 +FROM database.t1 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE c61 = 0 +) AS table25 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c37' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table24 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table23 +UNION ALL +SELECT +c27, +c30, +c32, +c31, +c29, +c33, +c37, +c34, +c35 +FROM +( +SELECT +c27, +c30, +c32, +'c39' AS c33, +coalesce(c37 * joinGet('database.table18', 'c60', concat(c26, '_', 'CH')), 0) AS c37, +if(c53 > 0, c53, 2) AS c53, +c54, +if(c29 > 0, c29, 1) AS c29, +c55, +c56, +datediff('day', c55, c56) AS c34, +datediff('day', c32, c55) AS c35, +c31 +FROM database.table22 +PREWHERE ((c32 >= parseDateTimeBestEffort('2020-01-01')) AND (c32 <= parseDateTimeBestEffort('2020-01-01 23:59:59'))) AND (c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +)) +WHERE (c61 = 0) AND (c37 < (666 * (1 / joinGet('database.table18', 'c60', concat(c26, '_', 'CH'))))) +) AS table21 +) AS table20 +ALL LEFT JOIN +( +SELECT +c27, +avgMerge(avg_c37) * joinGet('database.table18', 'c60', concat('USD', '_', 'CH')) AS c36 +FROM database.table19 +PREWHERE c27 IN +( +SELECT comp_c27 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) +WHERE date > (now() - toIntervalMonth(3)) +GROUP BY c27 +) AS table17 USING (c27) +GROUP BY c27 +) AS table16 +ALL LEFT JOIN +( +SELECT +comp_c27 AS c27, +assumeNotNull(c39) AS c39, +c29 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +) USING (c27) +) AS t +ALL LEFT JOIN +( +SELECT +c1, +c2 +FROM +( +SELECT +c39 AS c1, +groupArray(comp_c27) AS c49, +multiIf(c1 = 'c58', if(length(c49) <= 2, 0, 1), c1 = 'c57', 1, if(length(c49) <= 3, 0, 1)) AS c2 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +min_c32, +max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38 +FROM +( +SELECT +comp_c27, +groupArray(c39) AS c39, +any(c40) AS c40, +any(c41) AS c41, +any(c42) AS c42, +any(c29) AS c29, +any(c43) AS c43, +any(c44) AS c44, +any(min_c32) AS min_c32, +any(max_c32) AS max_c32, +any(c45) AS c45, +any(c46) AS c46, +any(c38) AS c38, +any(c47) AS c47 +FROM +( +SELECT +c27 AS comp_c27, +if(comp_c27 = 0, toDate('2010-01-01'), toDate(minMerge(min_c32))) AS min_c32, +if(comp_c27 = 0, toDate(now()), toDate(maxMerge(max_c32))) + 1 AS max_c32, +NULL AS c39, +NULL AS c40, +NULL AS c41, +NULL AS c42, +NULL AS c29, +NULL AS c43, +NULL AS c44, +NULL AS c45, +NULL AS c46, +NULL AS c38, +NULL AS c47 +FROM database.table15 +GROUP BY comp_c27 +UNION ALL +SELECT +comp_c27, +NULL AS min_c32, +NULL AS max_c32, +c39, +c40, +c41, +c42, +c29, +c43, +c44, +c45, +c46, +c38, +c47 +FROM +( +SELECT +c39, +comp_c27 AS c27, +comp_c27, +c40, +c41, +assumeNotNull(c45) AS c45, +assumeNotNull(c46) AS c46, +assumeNotNull(c38) AS c38, +joinGet('database.table14', 'c48', c40) AS c42, +joinGet('database.table14', 'c29', c40) AS c29, +joinGet('database.table14', 'c43', c40) AS c43, +joinGet('database.table14', 'property_c44', c40) AS c44, +splitByChar(',', assumeNotNull(joinGet('database.jointable13', 'prefix_c33', comp_c27))) AS c33s, +joinGet('database.jointable13', 'c47', comp_c27) AS c47 +FROM +( +SELECT +c39, +comp_c27, +joinGet('database.jointable13', 'c40', comp_c27) AS c40, +c41, +c45, +c46, +c38 +FROM +( +SELECT +c39, +arrayJoin(arrayMap(x -> toInt64(x), arrayFilter(x -> (length(x) > 0), splitByString(', ', c49)))) AS comp_c27, +c41, +c45, +c46, +c38 +FROM +( +SELECT +'c57' AS c39, +toString(c27) AS c49, +1 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE c27 IN (322) +UNION ALL +SELECT +'c58' AS c39, +arrayStringConcat(groupArray(toString(c27)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table12 +WHERE chain_id IN +( +SELECT chain_id +FROM database.table12 +WHERE c27 IN (322) +) +UNION ALL +SELECT +'c59' AS c39, +assumeNotNull(c27s_str) AS c49, +0 AS c41, +c50 AS c45, +c51 AS c46, +c52 AS c38 +FROM +( +SELECT * +FROM table11 +WHERE c27 IN (322) +) AS c1s_c59 +WHERE c27 IN (322) +UNION ALL +SELECT +'superSupercalifragilisticexpialidocious' AS c39, +arrayStringConcat(groupArray(toString(c1_id)), ', ') AS c49, +0 AS c41, +'' AS c45, +'' AS c46, +0 AS c38 +FROM database.table10 +WHERE c27 IN (322) +) AS table9 +) +) AS a +) AS table8 +) AS table7 +GROUP BY comp_c27 +) AS table6 +WHERE (parseDateTimeBestEffort('2020-01-01') >= min_c32) AND (max_c32 >= (parseDateTimeBestEffort('2021-05-02') - 2)) +) AS table5 +ARRAY JOIN c39 +WHERE isNotNull(c39) +) AS table4 +GROUP BY c39 +) AS table3 +) USING (c1) +GROUP BY +c1, +c2 +) AS table2 +ORDER BY c1 ASC ) AS table1 FORMAT Null ]]> From 801a894385593356fc9c29f456fd01caa584fc07 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Oct 2022 14:07:17 +0200 Subject: [PATCH 006/339] Final tweaks to reduce the set of changes --- src/Parsers/IAST_fwd.h | 1 + src/Parsers/IParser.h | 6 ++---- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 7c5174342ed..931aa31024e 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include namespace DB diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 507961f573d..94b2ce1a0a1 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -40,10 +40,8 @@ struct Expected if (current_pos == max_parsed_pos) { - for (auto it = variants.begin(); it != variants.end(); it++) - if (*it == description) - return; - variants.push_back(description); + if ((current_pos == max_parsed_pos) && (std::find(variants.begin(), variants.end(), description) == variants.end())) + variants.push_back(description); } } From 95bd7eec679d0745a7e1dff60ce266b6884268d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Oct 2022 14:27:11 +0200 Subject: [PATCH 007/339] style --- src/Parsers/IAST_fwd.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/IAST_fwd.h b/src/Parsers/IAST_fwd.h index 931aa31024e..53d41d42d65 100644 --- a/src/Parsers/IAST_fwd.h +++ b/src/Parsers/IAST_fwd.h @@ -1,8 +1,8 @@ #pragma once -#include #include #include +#include namespace DB { From 1aa6266c560f64dd13c6ba48babe7c4ad77a7c9a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Oct 2022 15:24:47 +0200 Subject: [PATCH 008/339] Reduce shared_ptr copies --- src/Parsers/ExpressionListParsers.cpp | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f7a016a59e4..18365c7afbd 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -46,16 +46,15 @@ bool ParserList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!elem_parser->parse(pos, element, expected)) return false; - elements.push_back(element); + elements.push_back(std::move(element)); return true; }; if (!parseUtil(pos, expected, parse_element, *separator_parser, allow_empty)) return false; - auto list = std::make_shared(result_separator); - list->children = std::move(elements); - node = list; + node = std::make_shared(result_separator); + node->children = std::move(elements); return true; } @@ -76,7 +75,7 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!elem_parser.parse(pos, element, expected)) return false; - elements.push_back(element); + elements.push_back(std::move(element)); return true; }; @@ -120,9 +119,8 @@ bool ParserUnionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!parseUtil(pos, parse_element, parse_separator)) return false; - auto list = std::make_shared(); - list->children = std::move(elements); - node = list; + node = std::make_shared(); + node->children = std::move(elements); return true; } @@ -242,7 +240,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, ASTPtr & node if (!elem_parser->parse(pos, elem, expected)) return false; - node = elem; + node = std::move(elem); first = false; } else @@ -1398,7 +1396,7 @@ public: return false; auto subquery = std::make_shared(); - subquery->children.push_back(node); + subquery->children.push_back(std::move(node)); elements = {makeASTFunction("exists", subquery)}; finished = true; From 96a1cb4a797ed8cb619a724e8974e67a52ed790b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Oct 2022 15:43:06 +0200 Subject: [PATCH 009/339] Fix changes with master --- src/Parsers/IParser.h | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 94b2ce1a0a1..40643c98227 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -38,11 +38,8 @@ struct Expected return; } - if (current_pos == max_parsed_pos) - { - if ((current_pos == max_parsed_pos) && (std::find(variants.begin(), variants.end(), description) == variants.end())) - variants.push_back(description); - } + if ((current_pos == max_parsed_pos) && (std::find(variants.begin(), variants.end(), description) == variants.end())) + variants.push_back(description); } ALWAYS_INLINE void add(TokenIterator it, const char * description) From 7b96a7b798813900504e0517cbb7ed5f93f8c3c1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Oct 2022 16:20:56 +0200 Subject: [PATCH 010/339] Move Tokens::operator and optimize the common path --- src/Parsers/TokenIterator.cpp | 19 +++++++++++++++++++ src/Parsers/TokenIterator.h | 17 +---------------- 2 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index 08877e0b2fe..ecff8b0d482 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,6 +4,25 @@ namespace DB { +const Token & Tokens::operator[](size_t index) +{ + if (likely(index < data.size())) + return data[index]; + + while (data.empty() || !data.back().isEnd()) + { + Token token = lexer.nextToken(); + + if (token.isSignificant()) + { + data.emplace_back(std::move(token)); + if (index < data.size()) + return data[index]; + } + } + return data.back(); +} + UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin) { /// We have just two kind of parentheses: () and []. diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index b84bec57817..62570f49358 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -25,22 +25,7 @@ private: public: Tokens(const char * begin, const char * end, size_t max_query_size = 0) : lexer(begin, end, max_query_size) {} - const Token & operator[] (size_t index) - { - while (true) - { - if (index < data.size()) - return data[index]; - - if (!data.empty() && data.back().isEnd()) - return data.back(); - - Token token = lexer.nextToken(); - - if (token.isSignificant()) - data.emplace_back(token); - } - } + const Token & operator[](size_t index); const Token & max() { From d60e2d39567e95856583b3bd7f98485c64e4af82 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Oct 2022 12:06:28 +0200 Subject: [PATCH 011/339] Try tokenizing the whole input at once --- src/Parsers/TokenIterator.cpp | 17 ++++++----------- src/Parsers/TokenIterator.h | 15 +++++++-------- 2 files changed, 13 insertions(+), 19 deletions(-) diff --git a/src/Parsers/TokenIterator.cpp b/src/Parsers/TokenIterator.cpp index ecff8b0d482..6633ddb9563 100644 --- a/src/Parsers/TokenIterator.cpp +++ b/src/Parsers/TokenIterator.cpp @@ -4,23 +4,18 @@ namespace DB { -const Token & Tokens::operator[](size_t index) +Tokens::Tokens(const char * begin, const char * end, size_t max_query_size) { - if (likely(index < data.size())) - return data[index]; + Lexer lexer(begin, end, max_query_size); - while (data.empty() || !data.back().isEnd()) + bool stop = false; + do { Token token = lexer.nextToken(); - + stop = token.isEnd() || token.type == TokenType::ErrorMaxQuerySizeExceeded; if (token.isSignificant()) - { data.emplace_back(std::move(token)); - if (index < data.size()) - return data[index]; - } - } - return data.back(); + } while (!stop); } UnmatchedParentheses checkUnmatchedParentheses(TokenIterator begin) diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 62570f49358..566b376eb8d 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -20,19 +20,18 @@ class Tokens { private: std::vector data; - Lexer lexer; + std::size_t last_accessed_index = 0; public: - Tokens(const char * begin, const char * end, size_t max_query_size = 0) : lexer(begin, end, max_query_size) {} + Tokens(const char * begin, const char * end, size_t max_query_size = 0); - const Token & operator[](size_t index); - - const Token & max() + ALWAYS_INLINE inline const Token & operator[](size_t index) { - if (data.empty()) - return (*this)[0]; - return data.back(); + last_accessed_index = std::max(last_accessed_index, index); + return data[index]; } + + ALWAYS_INLINE inline const Token & max() { return data[last_accessed_index]; } }; From a2eb1fe9be7f186585822ca62d83bdd81f7f8ba0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 17 Oct 2022 18:39:42 +0200 Subject: [PATCH 012/339] Add assertion --- src/Parsers/TokenIterator.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Parsers/TokenIterator.h b/src/Parsers/TokenIterator.h index 566b376eb8d..c9ac61dfef9 100644 --- a/src/Parsers/TokenIterator.h +++ b/src/Parsers/TokenIterator.h @@ -3,6 +3,7 @@ #include #include +#include #include @@ -27,6 +28,7 @@ public: ALWAYS_INLINE inline const Token & operator[](size_t index) { + assert(index < data.size()); last_accessed_index = std::max(last_accessed_index, index); return data[index]; } From e60415d07da02f3fb1c8c833c7b0d44746bebc5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 18 Oct 2022 11:40:12 +0200 Subject: [PATCH 013/339] Make clang-tidy happy --- src/Client/QueryFuzzer.cpp | 8 ++--- .../getDictionaryConfigurationFromAST.cpp | 13 +++++--- src/Interpreters/ApplyWithGlobalVisitor.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- .../JoinToSubqueryTransformVisitor.cpp | 2 +- .../LogicalExpressionsOptimizer.cpp | 2 +- .../MySQL/InterpretersMySQLDDLQuery.cpp | 2 +- .../TranslateQualifiedNamesVisitor.cpp | 2 +- src/Interpreters/TreeRewriter.cpp | 2 +- src/Interpreters/applyTableOverride.cpp | 32 +++++++++---------- src/Parsers/ASTColumnsMatcher.cpp | 2 +- src/Parsers/ASTColumnsTransformers.cpp | 4 +-- src/Parsers/ASTFunction.cpp | 4 +-- src/Parsers/ASTTTLElement.cpp | 4 +-- src/Parsers/ExpressionListParsers.cpp | 2 +- src/Storages/AlterCommands.cpp | 13 +++----- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageURL.cpp | 4 +-- 18 files changed, 49 insertions(+), 53 deletions(-) diff --git a/src/Client/QueryFuzzer.cpp b/src/Client/QueryFuzzer.cpp index f0c4313e8a8..2def3adc858 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Client/QueryFuzzer.cpp @@ -327,9 +327,7 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) // Add element if (fuzz_rand() % 50 == 0) { - auto pos = list->children.empty() - ? list->children.begin() - : list->children.begin() + fuzz_rand() % list->children.size(); + auto * pos = list->children.empty() ? list->children.begin() : list->children.begin() + fuzz_rand() % list->children.size(); auto col = getRandomColumnLike(); if (col) { @@ -373,9 +371,7 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) // Add element if (fuzz_rand() % 50 == 0) { - auto pos = impl->children.empty() - ? impl->children.begin() - : impl->children.begin() + fuzz_rand() % impl->children.size(); + auto * pos = impl->children.empty() ? impl->children.begin() : impl->children.begin() + fuzz_rand() % impl->children.size(); auto col = getRandomColumnLike(); if (col) impl->children.insert(pos, col); diff --git a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp index 4868413dabd..f957f5d2f6a 100644 --- a/src/Dictionaries/getDictionaryConfigurationFromAST.cpp +++ b/src/Dictionaries/getDictionaryConfigurationFromAST.cpp @@ -344,11 +344,14 @@ void buildPrimaryKeyConfiguration( auto identifier_name = key_names.front(); - auto it = std::find_if(children.begin(), children.end(), [&](const ASTPtr & node) - { - const ASTDictionaryAttributeDeclaration * dict_attr = node->as(); - return dict_attr->name == identifier_name; - }); + const auto * it = std::find_if( + children.begin(), + children.end(), + [&](const ASTPtr & node) + { + const ASTDictionaryAttributeDeclaration * dict_attr = node->as(); + return dict_attr->name == identifier_name; + }); if (it == children.end()) { diff --git a/src/Interpreters/ApplyWithGlobalVisitor.cpp b/src/Interpreters/ApplyWithGlobalVisitor.cpp index a0f256fca83..1d36b4ab203 100644 --- a/src/Interpreters/ApplyWithGlobalVisitor.cpp +++ b/src/Interpreters/ApplyWithGlobalVisitor.cpp @@ -88,7 +88,7 @@ void ApplyWithGlobalVisitor::visit(ASTPtr & ast) if (auto * ast_with_alias = dynamic_cast(child.get())) exprs[ast_with_alias->alias] = child; } - for (auto it = node_union->list_of_selects->children.begin() + 1; it != node_union->list_of_selects->children.end(); ++it) + for (auto * it = node_union->list_of_selects->children.begin() + 1; it != node_union->list_of_selects->children.end(); ++it) { if (auto * union_child = (*it)->as()) visit(*union_child, exprs, with_expression_list); diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index e89aa2244fe..60238792cd4 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -562,7 +562,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( ColumnsDescription res; auto name_type_it = column_names_and_types.begin(); - for (auto ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it) + for (const auto * ast_it = columns_ast.children.begin(); ast_it != columns_ast.children.end(); ++ast_it, ++name_type_it) { ColumnDescription column; diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index d033ea46370..6b4d98ad037 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -153,7 +153,7 @@ private: data.addTableColumns(identifier.name(), columns); // QualifiedAsterisk's transformers start to appear at child 1 - for (auto it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) + for (auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) { IASTColumnsTransformer::transform(*it, columns); } diff --git a/src/Interpreters/LogicalExpressionsOptimizer.cpp b/src/Interpreters/LogicalExpressionsOptimizer.cpp index 9e30cac2e19..35989f0dfba 100644 --- a/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -313,7 +313,7 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions() for (const auto & entry : garbage_map) { const auto * function = entry.first; - auto first_erased = entry.second; + auto * first_erased = entry.second; auto & operands = getFunctionOperands(function); operands.erase(first_erased, operands.end()); diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp index 4ed22b34e26..d3ce1ac7084 100644 --- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp +++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp @@ -154,7 +154,7 @@ static ColumnsDescription createColumnsDescription(const NamesAndTypesList & col /// but this produce endless recursion in gcc-11, and leads to SIGSEGV /// (see git blame for details). auto column_name_and_type = columns_name_and_type.begin(); - auto declare_column_ast = columns_definition->children.begin(); + const auto * declare_column_ast = columns_definition->children.begin(); for (; column_name_and_type != columns_name_and_type.end(); column_name_and_type++, declare_column_ast++) { const auto & declare_column = (*declare_column_ast)->as(); diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index bc862ed7b38..2ca1174f704 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -299,7 +299,7 @@ void TranslateQualifiedNamesMatcher::visit(ASTExpressionList & node, const ASTPt } // QualifiedAsterisk's transformers start to appear at child 1 - for (auto it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) + for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) { IASTColumnsTransformer::transform(*it, columns); } diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ac49d79c6ba..d0e7a32d876 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -541,7 +541,7 @@ void removeUnneededColumnsFromSelectClause(ASTSelectQuery * select_query, const auto & children = select_query->interpolate()->children; if (!children.empty()) { - for (auto it = children.begin(); it != children.end();) + for (auto * it = children.begin(); it != children.end();) { if (remove_columns.contains((*it)->as()->column)) it = select_query->interpolate()->children.erase(it); diff --git a/src/Interpreters/applyTableOverride.cpp b/src/Interpreters/applyTableOverride.cpp index e614e58b06b..8e88047c13c 100644 --- a/src/Interpreters/applyTableOverride.cpp +++ b/src/Interpreters/applyTableOverride.cpp @@ -26,10 +26,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat if (!create_query->columns_list->columns) create_query->columns_list->set(create_query->columns_list->columns, std::make_shared()); auto & dest_children = create_query->columns_list->columns->children; - auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool - { - return node->as()->name == override_column->name; - }); + auto * exists = std::find_if( + dest_children.begin(), + dest_children.end(), + [&](ASTPtr node) -> bool { return node->as()->name == override_column->name; }); /// For columns, only allow adding ALIAS (non-physical) for now. /// TODO: This logic should instead be handled by validation that is /// executed from InterpreterCreateQuery / InterpreterAlterQuery. @@ -52,10 +52,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat if (!create_query->columns_list->indices) create_query->columns_list->set(create_query->columns_list->indices, std::make_shared()); auto & dest_children = create_query->columns_list->indices->children; - auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool - { - return node->as()->name == override_index->name; - }); + auto * exists = std::find_if( + dest_children.begin(), + dest_children.end(), + [&](ASTPtr node) -> bool { return node->as()->name == override_index->name; }); if (exists == dest_children.end()) dest_children.emplace_back(override_index_ast); else @@ -72,10 +72,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat if (!create_query->columns_list->constraints) create_query->columns_list->set(create_query->columns_list->constraints, std::make_shared()); auto & dest_children = create_query->columns_list->constraints->children; - auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool - { - return node->as()->name == override_constraint->name; - }); + auto * exists = std::find_if( + dest_children.begin(), + dest_children.end(), + [&](ASTPtr node) -> bool { return node->as()->name == override_constraint->name; }); if (exists == dest_children.end()) dest_children.emplace_back(override_constraint_ast); else @@ -92,10 +92,10 @@ void applyTableOverrideToCreateQuery(const ASTTableOverride & override, ASTCreat if (!create_query->columns_list->projections) create_query->columns_list->set(create_query->columns_list->projections, std::make_shared()); auto & dest_children = create_query->columns_list->projections->children; - auto exists = std::find_if(dest_children.begin(), dest_children.end(), [&](ASTPtr node) -> bool - { - return node->as()->name == override_projection->name; - }); + auto * exists = std::find_if( + dest_children.begin(), + dest_children.end(), + [&](ASTPtr node) -> bool { return node->as()->name == override_projection->name; }); if (exists == dest_children.end()) dest_children.emplace_back(override_projection_ast); else diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index 8f167f99b37..ca7f6c6f5a1 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -82,7 +82,7 @@ void ASTColumnsListMatcher::updateTreeHashImpl(SipHash & hash_state) const void ASTColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const { writeCString("COLUMNS(", ostr); - for (auto it = column_list->children.begin(); it != column_list->children.end(); ++it) + for (auto * it = column_list->children.begin(); it != column_list->children.end(); ++it) { if (it != column_list->children.begin()) writeCString(", ", ostr); diff --git a/src/Parsers/ASTColumnsTransformers.cpp b/src/Parsers/ASTColumnsTransformers.cpp index 71207724a89..d7902698db3 100644 --- a/src/Parsers/ASTColumnsTransformers.cpp +++ b/src/Parsers/ASTColumnsTransformers.cpp @@ -217,7 +217,7 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const for (const auto & child : children) expected_columns.insert(child->as().name()); - for (auto it = nodes.begin(); it != nodes.end();) + for (auto * it = nodes.begin(); it != nodes.end();) { if (const auto * id = it->get()->as()) { @@ -234,7 +234,7 @@ void ASTColumnsExceptTransformer::transform(ASTs & nodes) const } else { - for (auto it = nodes.begin(); it != nodes.end();) + for (auto * it = nodes.begin(); it != nodes.end();) { if (const auto * id = it->get()->as()) { diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 63dc9f6b3ac..500815916b7 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -46,7 +46,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const if (parameters) { writeChar('(', ostr); - for (auto it = parameters->children.begin(); it != parameters->children.end(); ++it) + for (auto * it = parameters->children.begin(); it != parameters->children.end(); ++it) { if (it != parameters->children.begin()) writeCString(", ", ostr); @@ -59,7 +59,7 @@ void ASTFunction::appendColumnNameImpl(WriteBuffer & ostr) const writeChar('(', ostr); if (arguments) { - for (auto it = arguments->children.begin(); it != arguments->children.end(); ++it) + for (auto * it = arguments->children.begin(); it != arguments->children.end(); ++it) { if (it != arguments->children.begin()) writeCString(", ", ostr); diff --git a/src/Parsers/ASTTTLElement.cpp b/src/Parsers/ASTTTLElement.cpp index 90278e27c0c..30d8f1b8d94 100644 --- a/src/Parsers/ASTTTLElement.cpp +++ b/src/Parsers/ASTTTLElement.cpp @@ -52,7 +52,7 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st else if (mode == TTLMode::GROUP_BY) { settings.ostr << " GROUP BY "; - for (auto it = group_by_key.begin(); it != group_by_key.end(); ++it) + for (const auto * it = group_by_key.begin(); it != group_by_key.end(); ++it) { if (it != group_by_key.begin()) settings.ostr << ", "; @@ -62,7 +62,7 @@ void ASTTTLElement::formatImpl(const FormatSettings & settings, FormatState & st if (!group_by_assignments.empty()) { settings.ostr << " SET "; - for (auto it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it) + for (const auto * it = group_by_assignments.begin(); it != group_by_assignments.end(); ++it) { if (it != group_by_assignments.begin()) settings.ostr << ", "; diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 18365c7afbd..c7f76749ab3 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -594,7 +594,7 @@ public: asts.reserve(asts.size() + n); - auto start = operands.begin() + operands.size() - n; + auto * start = operands.begin() + operands.size() - n; asts.insert(asts.end(), std::make_move_iterator(start), std::make_move_iterator(operands.end())); operands.erase(start, operands.end()); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index dcd7abae68a..4acc9356bae 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -573,20 +573,17 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) ErrorCodes::ILLEGAL_COLUMN); } - auto insert_it = constraints.end(); + auto * insert_it = constraints.end(); constraints.emplace(insert_it, constraint_decl); metadata.constraints = ConstraintsDescription(constraints); } else if (type == DROP_CONSTRAINT) { auto constraints = metadata.constraints.getConstraints(); - auto erase_it = std::find_if( - constraints.begin(), - constraints.end(), - [this](const ASTPtr & constraint_ast) - { - return constraint_ast->as().name == constraint_name; - }); + auto * erase_it = std::find_if( + constraints.begin(), + constraints.end(), + [this](const ASTPtr & constraint_ast) { return constraint_ast->as().name == constraint_name; }); if (erase_it == constraints.end()) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 675dd548088..e5a52c0c757 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1152,7 +1152,7 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt "Storage S3 requires 1 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - auto header_it = StorageURL::collectHeaders(engine_args, configuration, local_context); + auto * header_it = StorageURL::collectHeaders(engine_args, configuration, local_context); if (header_it != engine_args.end()) engine_args.erase(header_it); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index f1c924a3448..4ee1f2d69e6 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -979,7 +979,7 @@ ASTs::iterator StorageURL::collectHeaders( { ASTs::iterator headers_it = url_function_args.end(); - for (auto arg_it = url_function_args.begin(); arg_it != url_function_args.end(); ++arg_it) + for (auto * arg_it = url_function_args.begin(); arg_it != url_function_args.end(); ++arg_it) { const auto * headers_ast_function = (*arg_it)->as(); if (headers_ast_function && headers_ast_function->name == "headers") @@ -1065,7 +1065,7 @@ URLBasedDataSourceConfiguration StorageURL::getConfiguration(ASTs & args, Contex if (args.empty() || args.size() > 3) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, bad_arguments_error_message); - auto header_it = collectHeaders(args, configuration, local_context); + auto * header_it = collectHeaders(args, configuration, local_context); if (header_it != args.end()) args.erase(header_it); From af019932bc46d299b791d2eed504bceac2c3caa0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 19 Oct 2022 17:20:21 +0200 Subject: [PATCH 014/339] Style --- src/Interpreters/JoinToSubqueryTransformVisitor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index 6b4d98ad037..1b3cf5ad463 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -153,7 +153,7 @@ private: data.addTableColumns(identifier.name(), columns); // QualifiedAsterisk's transformers start to appear at child 1 - for (auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) + for (const auto * it = qualified_asterisk->children.begin() + 1; it != qualified_asterisk->children.end(); ++it) { IASTColumnsTransformer::transform(*it, columns); } From 22da0420affcfd7854e9b5037c8de22bbe81fc2c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 19 Oct 2022 20:26:16 +0000 Subject: [PATCH 015/339] fix possible restart errors after failed quorum insert --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 10 +++++++++- ..._manual_write_to_replicas_quorum.reference | 20 +++++++++---------- .../01459_manual_write_to_replicas_quorum.sh | 20 +++++++++++++++++-- 3 files changed, 37 insertions(+), 13 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 0abea5977c3..feb3d9c3e9e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -576,7 +576,15 @@ void ReplicatedMergeTreeSink::commitPart( else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { storage.unlockSharedData(*part); - transaction.rollback(); + + /// Part was not committed to keeper. + /// So make it temporary and remove immediately to avoid its resurrection after restart. + transaction.rollbackPartsToTemporaryState(); + + part->is_temp = true; + part->renameTo(temporary_part_relative_path, false, builder); + builder->commit(); + throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } else diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference index 52dea650ebc..812fa4477cb 100644 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.reference @@ -1,10 +1,10 @@ -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 -100 0 99 4950 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 +200 0 199 19900 diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 6eabc9ae1b5..5a666454531 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -19,7 +19,7 @@ done valid_exceptions_to_retry='Quorum for previous write has not been satisfied yet|Another quorum insert has been already started|Unexpected logical error while adding block' -function thread { +function thread1 { for x in {0..99}; do while true; do $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break @@ -28,7 +28,23 @@ function thread { } for i in $(seq 1 $NUM_REPLICAS); do - thread $i & + thread1 $i & +done + +wait + +function thread2 { + for x in {100..199}; do + while true; do + $CLICKHOUSE_CLIENT --query "DETACH TABLE r$1" + $CLICKHOUSE_CLIENT --query "ATTACH TABLE r$1" + $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break + done + done +} + +for i in $(seq 1 $NUM_REPLICAS); do + thread2 $i & done wait From a6aad33e8912c756149392144391acbba27f39d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 20 Oct 2022 11:43:55 +0200 Subject: [PATCH 016/339] More style --- src/TableFunctions/TableFunctionMySQL.cpp | 2 +- src/TableFunctions/TableFunctionS3.cpp | 2 +- src/TableFunctions/TableFunctionURL.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/TableFunctions/TableFunctionMySQL.cpp b/src/TableFunctions/TableFunctionMySQL.cpp index ab1c23afa7a..a34e87ee313 100644 --- a/src/TableFunctions/TableFunctionMySQL.cpp +++ b/src/TableFunctions/TableFunctionMySQL.cpp @@ -45,7 +45,7 @@ void TableFunctionMySQL::parseArguments(const ASTPtr & ast_function, ContextPtr mysql_settings.connect_timeout = settings.external_storage_connect_timeout_sec; mysql_settings.read_write_timeout = settings.external_storage_rw_timeout_sec; - for (auto it = args.begin(); it != args.end(); ++it) + for (auto * it = args.begin(); it != args.end(); ++it) { const ASTSetQuery * settings_ast = (*it)->as(); if (settings_ast) diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index b8e4fcb67fa..05e11feac8e 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -41,7 +41,7 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar if (args.empty() || args.size() > 6) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); - auto header_it = StorageURL::collectHeaders(args, s3_configuration, context); + auto * header_it = StorageURL::collectHeaders(args, s3_configuration, context); if (header_it != args.end()) args.erase(header_it); diff --git a/src/TableFunctions/TableFunctionURL.cpp b/src/TableFunctions/TableFunctionURL.cpp index 888f3e7b93d..46e09b02901 100644 --- a/src/TableFunctions/TableFunctionURL.cpp +++ b/src/TableFunctions/TableFunctionURL.cpp @@ -71,7 +71,7 @@ void TableFunctionURL::parseArguments(const ASTPtr & ast_function, ContextPtr co auto * url_function_args_expr = assert_cast(args[0].get()); auto & url_function_args = url_function_args_expr->children; - auto headers_it = StorageURL::collectHeaders(url_function_args, configuration, context); + auto * headers_it = StorageURL::collectHeaders(url_function_args, configuration, context); /// ITableFunctionFileLike cannot parse headers argument, so remove it. if (headers_it != url_function_args.end()) url_function_args.erase(headers_it); From 2fa3c54caabb5f875ba473536f0ce2eb4f0ec1ca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 25 Oct 2022 13:33:07 +0200 Subject: [PATCH 017/339] ValuesBlockInputFormat: Adapt to the full tokenizer --- src/Parsers/IParser.h | 2 + .../Impl/ConstantExpressionTemplate.cpp | 23 ++- .../Formats/Impl/ConstantExpressionTemplate.h | 13 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 172 +++++++++++------- .../Formats/Impl/ValuesBlockInputFormat.h | 3 + 5 files changed, 137 insertions(+), 76 deletions(-) diff --git a/src/Parsers/IParser.h b/src/Parsers/IParser.h index 40643c98227..466cdf7a4b1 100644 --- a/src/Parsers/IParser.h +++ b/src/Parsers/IParser.h @@ -64,6 +64,8 @@ public: { } + Pos(TokenIterator token_iterator_, uint32_t max_depth_) : TokenIterator(token_iterator_), max_depth(max_depth_) { } + ALWAYS_INLINE void increaseDepth() { ++depth; diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 739fa8735b2..4cd16313c83 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -418,12 +418,13 @@ ConstantExpressionTemplate::Cache::getFromCacheOrConstruct(const DataTypePtr & r return res; } -bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const FormatSettings & format_settings, const Settings & settings) +bool ConstantExpressionTemplate::parseExpression( + ReadBuffer & istr, const TokenIterator & token_iterator, const FormatSettings & format_settings, const Settings & settings) { size_t cur_column = 0; try { - if (tryParseExpression(istr, format_settings, cur_column, settings)) + if (tryParseExpression(istr, token_iterator, format_settings, cur_column, settings)) { ++rows_count; return true; @@ -445,7 +446,12 @@ bool ConstantExpressionTemplate::parseExpression(ReadBuffer & istr, const Format return false; } -bool ConstantExpressionTemplate::tryParseExpression(ReadBuffer & istr, const FormatSettings & format_settings, size_t & cur_column, const Settings & settings) +bool ConstantExpressionTemplate::tryParseExpression( + ReadBuffer & istr, + const TokenIterator & token_iterator, + const FormatSettings & format_settings, + size_t & cur_column, + const Settings & settings) { size_t cur_token = 0; size_t num_columns = structure->literals.columns(); @@ -464,7 +470,7 @@ bool ConstantExpressionTemplate::tryParseExpression(ReadBuffer & istr, const For const DataTypePtr & type = structure->literals.getByPosition(cur_column).type; if (format_settings.values.accurate_types_of_literals && !structure->special_parser[cur_column].useDefaultParser()) { - if (!parseLiteralAndAssertType(istr, type.get(), cur_column, settings)) + if (!parseLiteralAndAssertType(istr, token_iterator, type.get(), cur_column, settings)) return false; } else @@ -482,7 +488,8 @@ bool ConstantExpressionTemplate::tryParseExpression(ReadBuffer & istr, const For return true; } -bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, const IDataType * complex_type, size_t column_idx, const Settings & settings) +bool ConstantExpressionTemplate::parseLiteralAndAssertType( + ReadBuffer & istr, const TokenIterator & token_iterator, const IDataType * complex_type, size_t column_idx, const Settings & settings) { using Type = Field::Types::Which; @@ -497,12 +504,12 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co if (type_info.is_array || type_info.is_tuple || type_info.is_map) { - /// TODO faster way to check types without using Parsers ParserArrayOfLiterals parser_array; ParserTupleOfLiterals parser_tuple; - Tokens tokens_number(istr.position(), istr.buffer().end()); - IParser::Pos iterator(tokens_number, settings.max_parser_depth); + IParser::Pos iterator(token_iterator, settings.max_parser_depth); + while (iterator->begin < istr.position()) + ++iterator; Expected expected; ASTPtr ast; if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected)) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index c5d4f033258..fbb3cbcd22a 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -71,7 +71,8 @@ public: /// Read expression from istr, assert it has the same structure and the same types of literals (template matches) /// and parse literals into temporary columns - bool parseExpression(ReadBuffer & istr, const FormatSettings & format_settings, const Settings & settings); + bool parseExpression( + ReadBuffer & istr, const TokenIterator & token_iterator, const FormatSettings & format_settings, const Settings & settings); /// Evaluate batch of expressions were parsed using template. /// If template was deduced with null_as_default == true, set bits in nulls for NULL values in column_idx, starting from offset. @@ -80,8 +81,14 @@ public: size_t rowsCount() const { return rows_count; } private: - bool tryParseExpression(ReadBuffer & istr, const FormatSettings & format_settings, size_t & cur_column, const Settings & settings); - bool parseLiteralAndAssertType(ReadBuffer & istr, const IDataType * type, size_t column_idx, const Settings & settings); + bool tryParseExpression( + ReadBuffer & istr, + const TokenIterator & token_iterator, + const FormatSettings & format_settings, + size_t & cur_column, + const Settings & settings); + bool parseLiteralAndAssertType( + ReadBuffer & istr, const TokenIterator & token_iterator, const IDataType * type, size_t column_idx, const Settings & settings); private: TemplateStructurePtr structure; diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index aff4557a4b7..597d86f5802 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -106,26 +106,105 @@ Chunk ValuesBlockInputFormat::generate() return Chunk{std::move(columns), rows_in_block}; } +/// Can be used in fileSegmentationEngine for parallel parsing of Values +static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance) +{ + skipWhitespaceIfAny(*buf); + if (buf->eof() || *buf->position() == ';') + return false; + bool quoted = false; + + size_t chunk_begin_buf_count = buf->count(); + while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes)) + { + buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end()); + if (buf->position() == buf->buffer().end()) + continue; + if (*buf->position() == '\\') + { + ++buf->position(); + if (!buf->eof()) + ++buf->position(); + } + else if (*buf->position() == '\'') + { + quoted ^= true; + ++buf->position(); + } + else if (*buf->position() == ')') + { + ++buf->position(); + if (!quoted) + --balance; + } + else if (*buf->position() == '(') + { + ++buf->position(); + if (!quoted) + ++balance; + } + } + + if (!buf->eof() && *buf->position() == ',') + ++buf->position(); + return true; +} + +/// We need continuous memory containing the expression to use Lexer +/// Note that this is both reading and tokenizing until the end of the row +/// This is doing unnecessary work if the rest of the columns can be read with tryReadValue (which doesn't require tokens) +/// and it's more efficient if they don't (as everything is already tokenized) +void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize() +{ + if (!skipToNextRow(buf.get(), 0, 1)) + return; + + buf->makeContinuousMemoryFromCheckpointToPos(); + auto rowEnd = buf->position(); + buf->rollbackToCheckpoint(); + tokens.emplace(buf->position(), rowEnd); + token_iterator.emplace(*tokens, static_cast(context->getSettingsRef().max_parser_depth)); +} + void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) { + bool line_has_been_tokenized = false; assertChar('(', *buf); for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) { skipWhitespaceIfAny(*buf); PeekableReadBufferCheckpoint checkpoint{*buf}; - bool read; + bool read = false; /// Parse value using fast streaming parser for literals and slow SQL parser for expressions. /// If there is SQL expression in some row, template of this expression will be deduced, /// so it makes possible to parse the following rows much faster /// if expressions in the following rows have the same structure if (parser_type_for_column[column_idx] == ParserType::Streaming) + { read = tryReadValue(*columns[column_idx], column_idx); + } else if (parser_type_for_column[column_idx] == ParserType::BatchTemplate) + { + if (!line_has_been_tokenized) + { + readUntilTheEndOfRowAndReTokenize(); + line_has_been_tokenized = true; + } read = tryParseExpressionUsingTemplate(columns[column_idx], column_idx); - else /// if (parser_type_for_column[column_idx] == ParserType::SingleExpressionEvaluation) + } + + /// For SingleExpressionEvaluation or it the previous ones failed + if (!read) + { + if (!line_has_been_tokenized) + { + readUntilTheEndOfRowAndReTokenize(); + line_has_been_tokenized = true; + } read = parseExpression(*columns[column_idx], column_idx); + } if (!read) block_missing_values.setBit(column_idx, row_num); @@ -141,9 +220,15 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx) { + chassert(token_iterator.has_value()); + chassert((*token_iterator)->begin <= buf->position()); + IParser::Pos start = *token_iterator; + while (token_iterator.value()->begin < buf->position()) + ++(*token_iterator); + /// Try to parse expression using template if one was successfully deduced while parsing the first row - auto settings = context->getSettingsRef(); - if (templates[column_idx]->parseExpression(*buf, format_settings, settings)) + const auto & settings = context->getSettingsRef(); + if (templates[column_idx]->parseExpression(*buf, *token_iterator, format_settings, settings)) { ++rows_parsed_using_template[column_idx]; return true; @@ -164,6 +249,7 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & /// Do not use this template anymore templates[column_idx].reset(); buf->rollbackToCheckpoint(); + *token_iterator = start; /// It will deduce new template or fallback to slow SQL parser return parseExpression(*column, column_idx); @@ -208,7 +294,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) /// Switch to SQL parser and don't try to use streaming parser for complex expressions /// Note: Throwing exceptions for each expression may be very slow because of stacktraces buf->rollbackToCheckpoint(); - return parseExpression(column, column_idx); + return false; } } @@ -293,79 +379,35 @@ namespace } } -/// Can be used in fileSegmentationEngine for parallel parsing of Values -static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance) -{ - skipWhitespaceIfAny(*buf); - if (buf->eof() || *buf->position() == ';') - return false; - bool quoted = false; - - size_t chunk_begin_buf_count = buf->count(); - while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes)) - { - buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end()); - if (buf->position() == buf->buffer().end()) - continue; - if (*buf->position() == '\\') - { - ++buf->position(); - if (!buf->eof()) - ++buf->position(); - } - else if (*buf->position() == '\'') - { - quoted ^= true; - ++buf->position(); - } - else if (*buf->position() == ')') - { - ++buf->position(); - if (!quoted) - --balance; - } - else if (*buf->position() == '(') - { - ++buf->position(); - if (!quoted) - ++balance; - } - } - - if (!buf->eof() && *buf->position() == ',') - ++buf->position(); - return true; -} - bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) { const Block & header = getPort().getHeader(); const IDataType & type = *header.getByPosition(column_idx).type; auto settings = context->getSettingsRef(); - /// We need continuous memory containing the expression to use Lexer - skipToNextRow(buf.get(), 0, 1); - buf->makeContinuousMemoryFromCheckpointToPos(); - buf->rollbackToCheckpoint(); + /// Advance the token iterator until the start of the column expression + chassert(token_iterator.has_value()); + chassert((*token_iterator)->begin <= buf->position()); + while ((*token_iterator)->begin < buf->position()) + ++(*token_iterator); Expected expected; - Tokens tokens(buf->position(), buf->buffer().end()); - IParser::Pos token_iterator(tokens, settings.max_parser_depth); + /// Keep a copy to the start of the column tokens to use if later if necessary + IParser::Pos ti_start(*token_iterator, settings.max_parser_depth); ASTPtr ast; - - bool parsed = parser.parse(token_iterator, ast, expected); + bool parsed = parser.parse(*token_iterator, ast, expected); /// Consider delimiter after value (',' or ')') as part of expression if (column_idx + 1 != num_columns) - parsed &= token_iterator->type == TokenType::Comma; + parsed &= (*token_iterator)->type == TokenType::Comma; else - parsed &= token_iterator->type == TokenType::ClosingRoundBracket; + parsed &= (*token_iterator)->type == TokenType::ClosingRoundBracket; if (!parsed) throw Exception("Cannot parse expression of type " + type.getName() + " here: " + String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())), ErrorCodes::SYNTAX_ERROR); - ++token_iterator; + ++(*token_iterator); if (parser_type_for_column[column_idx] != ParserType::Streaming && dynamic_cast(ast.get())) { @@ -415,8 +457,8 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx auto structure = templates_cache.getFromCacheOrConstruct( result_type, !result_type->isNullable() && format_settings.null_as_default, - TokenIterator(tokens), - token_iterator, + ti_start, + *token_iterator, ast, context, &found_in_cache, @@ -428,7 +470,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx ++attempts_to_deduce_template[column_idx]; buf->rollbackToCheckpoint(); - if (templates[column_idx]->parseExpression(*buf, format_settings, settings)) + if (templates[column_idx]->parseExpression(*buf, ti_start, format_settings, settings)) { ++rows_parsed_using_template[column_idx]; parser_type_for_column[column_idx] = ParserType::BatchTemplate; @@ -446,7 +488,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx else { buf->rollbackToCheckpoint(); - size_t len = const_cast(token_iterator->begin) - buf->position(); + size_t len = const_cast((*token_iterator)->begin) - buf->position(); throw Exception("Cannot deduce template of expression: " + std::string(buf->position(), len), ErrorCodes::SYNTAX_ERROR); } } @@ -458,7 +500,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx throw Exception("Interpreting expressions is disabled", ErrorCodes::SUPPORT_IS_DISABLED); /// Try to evaluate single expression if other parsers don't work - buf->position() = const_cast(token_iterator->begin); + buf->position() = const_cast((*token_iterator)->begin); std::pair value_raw = evaluateConstantExpression(ast, context); diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index bf243c54bd7..ded3d217e34 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -56,6 +56,7 @@ private: Chunk generate() override; void readRow(MutableColumns & columns, size_t row_num); + void readUntilTheEndOfRowAndReTokenize(); bool tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx); ALWAYS_INLINE inline bool tryReadValue(IColumn & column, size_t column_idx); @@ -70,6 +71,8 @@ private: void readSuffix(); std::unique_ptr buf; + std::optional token_iterator{}; + std::optional tokens{}; const RowInputFormatParams params; From 56a802188bdf071c47e54222a97dafd08f001dbe Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 26 Oct 2022 18:05:05 +0200 Subject: [PATCH 018/339] Fix bugs introduced when changing the logic --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 31 +++++-------------- 1 file changed, 8 insertions(+), 23 deletions(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 399453e6204..c4794e1ec59 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -156,9 +156,9 @@ static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int /// and it's more efficient if they don't (as everything is already tokenized) void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize() { - if (!skipToNextRow(buf.get(), 0, 1)) + if (tokens && token_iterator) return; - + skipToNextRow(buf.get(), 0, 1); buf->makeContinuousMemoryFromCheckpointToPos(); auto rowEnd = buf->position(); buf->rollbackToCheckpoint(); @@ -168,7 +168,8 @@ void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize() void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) { - bool line_has_been_tokenized = false; + tokens.reset(); + token_iterator.reset(); assertChar('(', *buf); for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) @@ -182,29 +183,11 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) /// so it makes possible to parse the following rows much faster /// if expressions in the following rows have the same structure if (parser_type_for_column[column_idx] == ParserType::Streaming) - { read = tryReadValue(*columns[column_idx], column_idx); - } else if (parser_type_for_column[column_idx] == ParserType::BatchTemplate) - { - if (!line_has_been_tokenized) - { - readUntilTheEndOfRowAndReTokenize(); - line_has_been_tokenized = true; - } read = tryParseExpressionUsingTemplate(columns[column_idx], column_idx); - } - - /// For SingleExpressionEvaluation or it the previous ones failed - if (!read) - { - if (!line_has_been_tokenized) - { - readUntilTheEndOfRowAndReTokenize(); - line_has_been_tokenized = true; - } + else read = parseExpression(*columns[column_idx], column_idx); - } if (!read) block_missing_values.setBit(column_idx, row_num); @@ -220,6 +203,7 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx) { + readUntilTheEndOfRowAndReTokenize(); chassert(token_iterator.has_value()); chassert((*token_iterator)->begin <= buf->position()); IParser::Pos start = *token_iterator; @@ -294,7 +278,7 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) /// Switch to SQL parser and don't try to use streaming parser for complex expressions /// Note: Throwing exceptions for each expression may be very slow because of stacktraces buf->rollbackToCheckpoint(); - return false; + return parseExpression(column, column_idx); } } @@ -386,6 +370,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx auto settings = context->getSettingsRef(); /// Advance the token iterator until the start of the column expression + readUntilTheEndOfRowAndReTokenize(); chassert(token_iterator.has_value()); chassert((*token_iterator)->begin <= buf->position()); while ((*token_iterator)->begin < buf->position()) From e77fcb0a99f2385191d04dce222f2e646d8bf51b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 27 Oct 2022 13:22:44 +0200 Subject: [PATCH 019/339] More style --- src/Parsers/ASTColumnsMatcher.cpp | 2 +- src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Parsers/ASTColumnsMatcher.cpp b/src/Parsers/ASTColumnsMatcher.cpp index 07f1b88a360..124206043cf 100644 --- a/src/Parsers/ASTColumnsMatcher.cpp +++ b/src/Parsers/ASTColumnsMatcher.cpp @@ -198,7 +198,7 @@ void ASTQualifiedColumnsListMatcher::appendColumnName(WriteBuffer & ostr) const qualifier->appendColumnName(ostr); writeCString(".COLUMNS(", ostr); - for (auto it = column_list->children.begin(); it != column_list->children.end(); ++it) + for (auto * it = column_list->children.begin(); it != column_list->children.end(); ++it) { if (it != column_list->children.begin()) writeCString(", ", ostr); diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index c4794e1ec59..5ebbaddd57b 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -160,9 +160,9 @@ void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize() return; skipToNextRow(buf.get(), 0, 1); buf->makeContinuousMemoryFromCheckpointToPos(); - auto rowEnd = buf->position(); + auto * row_end = buf->position(); buf->rollbackToCheckpoint(); - tokens.emplace(buf->position(), rowEnd); + tokens.emplace(buf->position(), row_end); token_iterator.emplace(*tokens, static_cast(context->getSettingsRef().max_parser_depth)); } From e75e71bbcbc9fee1b41ec45b0837a171af8623d6 Mon Sep 17 00:00:00 2001 From: stan <31004541+YunlongChen@users.noreply.github.com> Date: Wed, 16 Nov 2022 22:44:54 +0800 Subject: [PATCH 020/339] Update index.md correct the error of translation --- docs/zh/engines/table-engines/log-family/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/engines/table-engines/log-family/index.md b/docs/zh/engines/table-engines/log-family/index.md index 56776522445..1b24984f75f 100644 --- a/docs/zh/engines/table-engines/log-family/index.md +++ b/docs/zh/engines/table-engines/log-family/index.md @@ -11,7 +11,7 @@ sidebar_position: 29 这系列的引擎有: - [StripeLog](stripelog.md) -- [日志](log.md) +- [Log](log.md) - [TinyLog](tinylog.md) ## 共同属性 {#table_engines-log-engine-family-common-properties} From 80403015e7f1c1a3fc183440ff7dcceaa5152cc5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 17 Nov 2022 15:01:12 +0100 Subject: [PATCH 021/339] Fix assert reached with lines without data --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 25 ++++++++++++----- .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- .../02482_value_block_parsing.reference | 2 ++ .../0_stateless/02482_value_block_parsing.sh | 27 +++++++++++++++++++ 4 files changed, 49 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02482_value_block_parsing.reference create mode 100755 tests/queries/0_stateless/02482_value_block_parsing.sh diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index ffafe836a5e..9cc043a60ea 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -156,7 +156,7 @@ static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int /// Note that this is both reading and tokenizing until the end of the row /// This is doing unnecessary work if the rest of the columns can be read with tryReadValue (which doesn't require tokens) /// and it's more efficient if they don't (as everything is already tokenized) -void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize() +void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize(size_t current_column_idx) { if (tokens && token_iterator) return; @@ -166,6 +166,17 @@ void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize() buf->rollbackToCheckpoint(); tokens.emplace(buf->position(), row_end); token_iterator.emplace(*tokens, static_cast(context->getSettingsRef().max_parser_depth)); + auto const & first = (*token_iterator).get(); + if (first.isError() || first.isEnd()) + { + const Block & header = getPort().getHeader(); + const IDataType & type = *header.getByPosition(current_column_idx).type; + throw Exception( + ErrorCodes::SYNTAX_ERROR, + "Cannot parse expression of type {} here: {}", + type.getName(), + std::string_view(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position()))); + } } void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) @@ -205,7 +216,7 @@ void ValuesBlockInputFormat::readRow(MutableColumns & columns, size_t row_num) bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx) { - readUntilTheEndOfRowAndReTokenize(); + readUntilTheEndOfRowAndReTokenize(column_idx); chassert(token_iterator.has_value()); chassert((*token_iterator)->begin <= buf->position()); IParser::Pos start = *token_iterator; @@ -372,7 +383,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx auto settings = context->getSettingsRef(); /// Advance the token iterator until the start of the column expression - readUntilTheEndOfRowAndReTokenize(); + readUntilTheEndOfRowAndReTokenize(column_idx); chassert(token_iterator.has_value()); chassert((*token_iterator)->begin <= buf->position()); while ((*token_iterator)->begin < buf->position()) @@ -391,9 +402,11 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx parsed &= (*token_iterator)->type == TokenType::ClosingRoundBracket; if (!parsed) - throw Exception("Cannot parse expression of type " + type.getName() + " here: " - + String(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position())), - ErrorCodes::SYNTAX_ERROR); + throw Exception( + ErrorCodes::SYNTAX_ERROR, + "Cannot parse expression of type {} here: {}", + type.getName(), + std::string_view(buf->position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf->buffer().end() - buf->position()))); ++(*token_iterator); if (parser_type_for_column[column_idx] != ParserType::Streaming && dynamic_cast(ast.get())) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index ded3d217e34..9abade72af1 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -56,7 +56,7 @@ private: Chunk generate() override; void readRow(MutableColumns & columns, size_t row_num); - void readUntilTheEndOfRowAndReTokenize(); + void readUntilTheEndOfRowAndReTokenize(size_t current_column_idx); bool tryParseExpressionUsingTemplate(MutableColumnPtr & column, size_t column_idx); ALWAYS_INLINE inline bool tryReadValue(IColumn & column, size_t column_idx); diff --git a/tests/queries/0_stateless/02482_value_block_parsing.reference b/tests/queries/0_stateless/02482_value_block_parsing.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02482_value_block_parsing.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02482_value_block_parsing.sh b/tests/queries/0_stateless/02482_value_block_parsing.sh new file mode 100755 index 00000000000..7aedd4bfbea --- /dev/null +++ b/tests/queries/0_stateless/02482_value_block_parsing.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash +# Tags: long + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query=" + CREATE TABLE simple_key_dictionary_source_table__fuzz_48 + ( + id Nullable(Int8), + value Array(Date), + value_nullable UUID + ) + ENGINE = TinyLog;" + +echo "INSERT INTO simple_key_dictionary_source_table__fuzz_48 FORMAT Values (null, [], '61f0c404-5cb3-11e7-907b-a6006ad3dba0') +( -- Bu " | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" --data-binary @- -v 2>&1 | grep -c 'X-ClickHouse-Exception-Code: 62' + + +echo "INSERT INTO simple_key_dictionary_source_table__fuzz_48 FORMAT Values + (!Invalid" | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" --data-binary @- -v 2>&1 | grep -c 'X-ClickHouse-Exception-Code: 62' + +echo "INSERT INTO simple_key_dictionary_source_table__fuzz_48 FORMAT Values (null, [], '61f0c404-5cb3-11e7-907b-a6006ad3dba0') + ,(null, [], '61f0c404-5cb3-11e7-907b-a6006ad3dba0'), + (!!!!!!3adas + )" | ${CLICKHOUSE_CURL} -s "${CLICKHOUSE_URL}" --data-binary @- -v 2>&1 | grep -c 'X-ClickHouse-Exception-Code: 62' From 38309052993fd79b9bfe912a2516eae8af4f4fa6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 21 Nov 2022 11:02:08 +0100 Subject: [PATCH 022/339] Update reference --- tests/queries/0_stateless/02482_value_block_parsing.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02482_value_block_parsing.reference b/tests/queries/0_stateless/02482_value_block_parsing.reference index 6ed281c757a..e8183f05f5d 100644 --- a/tests/queries/0_stateless/02482_value_block_parsing.reference +++ b/tests/queries/0_stateless/02482_value_block_parsing.reference @@ -1,2 +1,3 @@ 1 1 +1 From 24c7496d705e6335c93ed0eacddb8f8c48bb491d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 21 Nov 2022 11:05:56 +0100 Subject: [PATCH 023/339] Why am i getting this warnings? --- src/TableFunctions/TableFunctionDeltaLake.cpp | 2 +- src/TableFunctions/TableFunctionHudi.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionDeltaLake.cpp b/src/TableFunctions/TableFunctionDeltaLake.cpp index 0f5801d57ac..c14f12ab591 100644 --- a/src/TableFunctions/TableFunctionDeltaLake.cpp +++ b/src/TableFunctions/TableFunctionDeltaLake.cpp @@ -33,7 +33,7 @@ void TableFunctionDelta::parseArgumentsImpl( if (args.empty() || args.size() > 6) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); - auto header_it = StorageURL::collectHeaders(args, base_configuration, context); + auto * header_it = StorageURL::collectHeaders(args, base_configuration, context); if (header_it != args.end()) args.erase(header_it); diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index 2e27d192b58..b88cda04c31 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -33,7 +33,7 @@ void TableFunctionHudi::parseArgumentsImpl( if (args.empty() || args.size() > 6) throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); - auto header_it = StorageURL::collectHeaders(args, base_configuration, context); + auto * header_it = StorageURL::collectHeaders(args, base_configuration, context); if (header_it != args.end()) args.erase(header_it); From ad95846b2bf0a4ed546526805758f811984cda2a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 22 Nov 2022 13:10:53 +0100 Subject: [PATCH 024/339] AST fuzzer: Print data of INSERT VALUES --- programs/client/Client.cpp | 21 +++++++++++++++++-- .../0_stateless/02482_value_block_parsing.sh | 1 - 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 6e289b57845..8c8f58e65c8 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -28,9 +28,10 @@ #include #include -#include -#include #include +#include +#include +#include #include #include @@ -39,6 +40,8 @@ #include #include +#include + #include #include @@ -829,6 +832,20 @@ bool Client::processWithFuzzing(const String & full_query) WriteBufferFromOStream ast_buf(std::cout, 4096); formatAST(*query, ast_buf, false /*highlight*/); ast_buf.next(); + if (const auto * insert = query->as()) + { + /// For inserts with data it's really useful to have the data itself available in the logs, as formatAST doesn't print it + if (insert->hasInlinedData()) + { + String bytes; + { + auto read_buf = getReadBufferFromASTInsertQuery(query); + WriteBufferFromString write_buf(bytes); + copyData(*read_buf, write_buf); + } + std::cout << std::endl << bytes; + } + } std::cout << std::endl << std::endl; try diff --git a/tests/queries/0_stateless/02482_value_block_parsing.sh b/tests/queries/0_stateless/02482_value_block_parsing.sh index 7aedd4bfbea..b74d3f395f0 100755 --- a/tests/queries/0_stateless/02482_value_block_parsing.sh +++ b/tests/queries/0_stateless/02482_value_block_parsing.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: long CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From e63ba060482935053233336dc2b678ddbd559d06 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 22 Nov 2022 19:03:17 +0100 Subject: [PATCH 025/339] Better cache management --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 4 +++- .../02482_value_block_assert.reference | 0 .../0_stateless/02482_value_block_assert.sql | 24 +++++++++++++++++++ 3 files changed, 27 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02482_value_block_assert.reference create mode 100644 tests/queries/0_stateless/02482_value_block_assert.sql diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 9cc043a60ea..6e1348d5dd0 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -158,7 +158,9 @@ static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int /// and it's more efficient if they don't (as everything is already tokenized) void ValuesBlockInputFormat::readUntilTheEndOfRowAndReTokenize(size_t current_column_idx) { - if (tokens && token_iterator) + if (tokens && token_iterator && + /// Make sure the underlying memory hasn't changed because of next() calls in the buffer + (*token_iterator)->begin >= buf->buffer().begin() && (*token_iterator)->begin <= buf->buffer().end()) return; skipToNextRow(buf.get(), 0, 1); buf->makeContinuousMemoryFromCheckpointToPos(); diff --git a/tests/queries/0_stateless/02482_value_block_assert.reference b/tests/queries/0_stateless/02482_value_block_assert.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02482_value_block_assert.sql b/tests/queries/0_stateless/02482_value_block_assert.sql new file mode 100644 index 00000000000..8684776f45f --- /dev/null +++ b/tests/queries/0_stateless/02482_value_block_assert.sql @@ -0,0 +1,24 @@ +SET allow_suspicious_low_cardinality_types=1; +CREATE TABLE range_key_dictionary_source_table__fuzz_323 +( + `key` UInt256, + `start_date` Int8, + `end_date` LowCardinality(UInt256), + `value` Tuple(UInt8, Array(DateTime), Decimal(9, 1), Array(Int16), Array(UInt8)), + `value_nullable` UUID +) +ENGINE = TinyLog; +INSERT INTO range_key_dictionary_source_table__fuzz_323 FORMAT Values +(1, toDate('2019-05-20'), toDate('2019-05-20'), 'First', 'First'); -- { clientError CANNOT_PARSE_INPUT_ASSERTION_FAILED } + + +CREATE TABLE complex_key_dictionary_source_table__fuzz_267 +( + `id` Decimal(38, 30), + `id_key` Array(UUID), + `value` Array(Nullable(DateTime64(3))), + `value_nullable` Nullable(UUID) +) +ENGINE = TinyLog; +INSERT INTO complex_key_dictionary_source_table__fuzz_267 FORMAT Values +(1, 'key', 'First', 'First'); -- { clientError CANNOT_READ_ARRAY_FROM_TEXT } From c0c976b85030f019fad841d700028923ed350d6f Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Nov 2022 18:19:00 +0000 Subject: [PATCH 026/339] Better handling of NULL in aggregate combinators, fix possible segfault/logical error while using --- .../AggregateFunctionIf.cpp | 26 +++++++++++++++---- src/AggregateFunctions/AggregateFunctionIf.h | 8 ++++-- .../AggregateFunctionNull.cpp | 18 ++++++++++--- src/AggregateFunctions/IAggregateFunction.h | 8 ++++++ src/Analyzer/Passes/SumIfToCountIfPass.cpp | 5 ++-- ...4_combinators_with_null_argument.reference | 18 +++++++++++++ .../02494_combinators_with_null_argument.sql | 11 ++++++++ .../02495_sum_if_to_count_if_bug.reference | 3 +++ .../02495_sum_if_to_count_if_bug.sql | 4 +++ 9 files changed, 88 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02494_combinators_with_null_argument.reference create mode 100644 tests/queries/0_stateless/02494_combinators_with_null_argument.sql create mode 100644 tests/queries/0_stateless/02495_sum_if_to_count_if_bug.reference create mode 100644 tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index c32454b10e4..0261af0a7d9 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -23,7 +23,7 @@ public: throw Exception("Incorrect number of arguments for aggregate function with " + getName() + " suffix", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isUInt8(arguments.back())) + if (!isUInt8(arguments.back()) && !arguments.back()->onlyNull()) throw Exception("Illegal type " + arguments.back()->getName() + " of last argument for aggregate function with " + getName() + " suffix", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -52,6 +52,7 @@ class AggregateFunctionIfNullUnary final private: size_t num_arguments; bool filter_is_nullable = false; + bool filter_is_only_null = false; /// The name of the nested function, including combinators (i.e. *If) /// @@ -84,10 +85,8 @@ private: return assert_cast(*filter_column).getData()[row_num] && !filter_null_map[row_num]; } - else - { - return assert_cast(*filter_column).getData()[row_num]; - } + + return assert_cast(*filter_column).getData()[row_num]; } public: @@ -106,10 +105,14 @@ public: "Aggregate function {} require at least one argument", getName()); filter_is_nullable = arguments[num_arguments - 1]->isNullable(); + filter_is_only_null = arguments[num_arguments - 1]->onlyNull(); } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { + if (filter_is_only_null) + return; + const ColumnNullable * column = assert_cast(columns[0]); const IColumn * nested_column = &column->getNestedColumn(); if (!column->isNullAt(row_num) && singleFilter(columns, row_num)) @@ -127,6 +130,9 @@ public: Arena * arena, ssize_t) const override { + if (filter_is_only_null) + return; + const ColumnNullable * column = assert_cast(columns[0]); const UInt8 * null_map = column->getNullMapData().data(); const IColumn * columns_param[] = {&column->getNestedColumn()}; @@ -224,6 +230,9 @@ class AggregateFunctionIfNullVariadic final : public AggregateFunctionNullBase< serialize_flag, AggregateFunctionIfNullVariadic> { +private: + bool filter_is_only_null = false; + public: String getName() const override @@ -243,10 +252,14 @@ public: for (size_t i = 0; i < number_of_arguments; ++i) is_nullable[i] = arguments[i]->isNullable(); + + filter_is_only_null = arguments.back()->onlyNull(); } static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) { + if (columns[num_arguments - 1]->onlyNull()) + return false; return assert_cast(*columns[num_arguments - 1]).getData()[row_num]; } @@ -282,6 +295,9 @@ public: void addBatchSinglePlace( size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t) const final { + if (filter_is_only_null) + return; + std::unique_ptr final_null_flags = std::make_unique(row_end); const size_t filter_column_num = number_of_arguments - 1; diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index ccc4809dd06..196d1c998ed 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -42,7 +42,7 @@ public: if (num_arguments == 0) throw Exception("Aggregate function " + getName() + " require at least one argument", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isUInt8(types.back())) + if (!isUInt8(types.back()) && !types.back()->onlyNull()) throw Exception("Last argument for aggregate function " + getName() + " must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -204,12 +204,16 @@ public: AggregateFunctionPtr getNestedFunction() const override { return nested_func; } + std::unordered_set getArgumentsThatCanBeOnlyNull() const override + { + return {num_arguments - 1}; + } #if USE_EMBEDDED_COMPILER bool isCompilable() const override { - return nested_func->isCompilable(); + return canBeNativeType(*this->argument_types.back()) && nested_func->isCompilable(); } void compileCreate(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr) const override diff --git a/src/AggregateFunctions/AggregateFunctionNull.cpp b/src/AggregateFunctions/AggregateFunctionNull.cpp index 01558b56667..177b1129941 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.cpp +++ b/src/AggregateFunctions/AggregateFunctionNull.cpp @@ -29,7 +29,13 @@ public: size_t size = arguments.size(); DataTypes res(size); for (size_t i = 0; i < size; ++i) - res[i] = removeNullable(arguments[i]); + { + /// Nullable(Nothing) is processed separately, don't convert it to Nothing. + if (arguments[i]->onlyNull()) + res[i] = arguments[i]; + else + res[i] = removeNullable(arguments[i]); + } return res; } @@ -41,12 +47,16 @@ public: { bool has_nullable_types = false; bool has_null_types = false; - for (const auto & arg_type : arguments) + std::unordered_set arguments_that_can_be_only_null; + if (nested_function) + arguments_that_can_be_only_null = nested_function->getArgumentsThatCanBeOnlyNull(); + + for (size_t i = 0; i < arguments.size(); ++i) { - if (arg_type->isNullable()) + if (arguments[i]->isNullable()) { has_nullable_types = true; - if (arg_type->onlyNull()) + if (arguments[i]->onlyNull() && !arguments_that_can_be_only_null.contains(i)) { has_null_types = true; break; diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index ada00791e69..b228de3c6bc 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -343,6 +343,14 @@ public: return nullptr; } + /// For most functions if one of arguments is always NULL, we return NULL (it's implemented in combinator Null), + /// but in some functions we can want to process this argument somehow (for example condition argument in If combinator). + /// This method returns the set of argument indexes that can be always NULL, they will be skipped in combinator Null. + virtual std::unordered_set getArgumentsThatCanBeOnlyNull() const + { + return {}; + } + /** Return the nested function if this is an Aggregate Function Combinator. * Otherwise return nullptr. */ diff --git a/src/Analyzer/Passes/SumIfToCountIfPass.cpp b/src/Analyzer/Passes/SumIfToCountIfPass.cpp index 5413d1b4670..78de512f08b 100644 --- a/src/Analyzer/Passes/SumIfToCountIfPass.cpp +++ b/src/Analyzer/Passes/SumIfToCountIfPass.cpp @@ -80,6 +80,7 @@ public: if (nested_if_function_arguments_nodes.size() != 3) return; + auto & cond_argument = nested_if_function_arguments_nodes[0]; auto if_true_condition_constant_value = nested_if_function_arguments_nodes[1]->getConstantValueOrNull(); auto if_false_condition_constant_value = nested_if_function_arguments_nodes[2]->getConstantValueOrNull(); @@ -106,8 +107,8 @@ public: return; } - /// Rewrite `sum(if(cond, 0, 1))` into `countIf(not(cond))`. - if (if_true_condition_value == 0 && if_false_condition_value == 1) + /// Rewrite `sum(if(cond, 0, 1))` into `countIf(not(cond))` if condition is not Nullable (otherwise the result can be different). + if (if_true_condition_value == 0 && if_false_condition_value == 1 && !cond_argument->getResultType()->isNullable()) { DataTypePtr not_function_result_type = std::make_shared(); diff --git a/tests/queries/0_stateless/02494_combinators_with_null_argument.reference b/tests/queries/0_stateless/02494_combinators_with_null_argument.reference new file mode 100644 index 00000000000..a891c305dde --- /dev/null +++ b/tests/queries/0_stateless/02494_combinators_with_null_argument.reference @@ -0,0 +1,18 @@ +-- { echoOn } + +select sumIf(1, NULL); +0 +select sumIf(NULL, 1); +\N +select sumIf(NULL, NULL); +\N +select countIf(1, NULL); +0 +select countIf(NULL, 1); +0 +select countIf(1, NULL); +0 +select sumArray([NULL, NULL]); +\N +select countArray([NULL, NULL]); +0 diff --git a/tests/queries/0_stateless/02494_combinators_with_null_argument.sql b/tests/queries/0_stateless/02494_combinators_with_null_argument.sql new file mode 100644 index 00000000000..e18fd741aab --- /dev/null +++ b/tests/queries/0_stateless/02494_combinators_with_null_argument.sql @@ -0,0 +1,11 @@ +-- { echoOn } + +select sumIf(1, NULL); +select sumIf(NULL, 1); +select sumIf(NULL, NULL); +select countIf(1, NULL); +select countIf(NULL, 1); +select countIf(1, NULL); +select sumArray([NULL, NULL]); +select countArray([NULL, NULL]); + diff --git a/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.reference b/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.reference new file mode 100644 index 00000000000..4bda3243d2e --- /dev/null +++ b/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.reference @@ -0,0 +1,3 @@ +1024 +0 +1024 diff --git a/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql b/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql new file mode 100644 index 00000000000..0791b374668 --- /dev/null +++ b/tests/queries/0_stateless/02495_sum_if_to_count_if_bug.sql @@ -0,0 +1,4 @@ +select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=0; +select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=1, allow_experimental_analyzer=0; +select sum(if((number % NULL) = 2, 0, 1)) FROM numbers(1024) settings optimize_rewrite_sum_if_to_count_if=1, allow_experimental_analyzer=1; + From 61d864f846a1a6f1a0e455d265095ad110353404 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 29 Nov 2022 18:20:51 +0000 Subject: [PATCH 027/339] Remove unneded check --- src/AggregateFunctions/AggregateFunctionIf.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 0261af0a7d9..793ece53d23 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -258,8 +258,6 @@ public: static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) { - if (columns[num_arguments - 1]->onlyNull()) - return false; return assert_cast(*columns[num_arguments - 1]).getData()[row_num]; } From 3579f2afb6c35e14474155d424fe52fa9d992d87 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 8 Dec 2022 04:35:31 +0000 Subject: [PATCH 028/339] do not print default EPHEMERAL in column declaration format --- src/Parsers/ASTColumnDeclaration.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index dc5651d9f14..041a19c401e 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -78,7 +79,12 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta if (default_expression) { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : ""); - if (default_specifier != "EPHEMERAL" || !default_expression->as()->value.isNull()) + if (default_specifier != "EPHEMERAL" || + !( + default_expression->as()->value.isNull() || + (type && default_expression->as()->value == DataTypeFactory::instance().get(type)->getDefault()) + ) + ) { settings.ostr << ' '; default_expression->formatImpl(settings, state, frame); From 9c245072760ae8e2f4fd4c9cd897ebf7aa024530 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Dec 2022 00:40:35 +0000 Subject: [PATCH 029/339] dedicated flag for ephemeral default, make ephemeral accept expression instead of literal --- src/Interpreters/InterpreterCreateQuery.cpp | 8 ++++---- src/Parsers/ASTColumnDeclaration.cpp | 7 +------ src/Parsers/ASTColumnDeclaration.h | 1 + src/Parsers/ParserCreateQuery.h | 10 ++++++++-- src/Storages/ColumnDefault.h | 1 + src/Storages/ColumnsDescription.cpp | 1 + 6 files changed, 16 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 50536b66185..ad7e6e2db4e 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -404,6 +404,8 @@ ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns) column_declaration->children.push_back(column_declaration->default_expression); } + column_declaration->ephemeral_default = column.default_desc.ephemeral_default; + if (!column.comment.empty()) { column_declaration->comment = std::make_shared(Field(column.comment)); @@ -590,10 +592,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( visitor.visit(col_decl.default_expression); } - ASTPtr default_expr = - col_decl.default_specifier == "EPHEMERAL" && col_decl.default_expression->as()->value.isNull() ? - std::make_shared(DataTypeFactory::instance().get(col_decl.type)->getDefault()) : - col_decl.default_expression->clone(); + ASTPtr default_expr = col_decl.default_expression->clone(); if (col_decl.type) column.type = name_type_it->type; @@ -607,6 +606,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( column.default_desc.kind = columnDefaultKindFromString(col_decl.default_specifier); column.default_desc.expression = default_expr; + column.default_desc.ephemeral_default = col_decl.ephemeral_default; } else if (col_decl.type) column.type = name_type_it->type; diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 041a19c401e..c2396708a73 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -79,12 +79,7 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & settings, FormatSta if (default_expression) { settings.ostr << ' ' << (settings.hilite ? hilite_keyword : "") << default_specifier << (settings.hilite ? hilite_none : ""); - if (default_specifier != "EPHEMERAL" || - !( - default_expression->as()->value.isNull() || - (type && default_expression->as()->value == DataTypeFactory::instance().get(type)->getDefault()) - ) - ) + if (!ephemeral_default) { settings.ostr << ' '; default_expression->formatImpl(settings, state, frame); diff --git a/src/Parsers/ASTColumnDeclaration.h b/src/Parsers/ASTColumnDeclaration.h index 5ecfb859abc..2008e4f99d1 100644 --- a/src/Parsers/ASTColumnDeclaration.h +++ b/src/Parsers/ASTColumnDeclaration.h @@ -16,6 +16,7 @@ public: std::optional null_modifier; String default_specifier; ASTPtr default_expression; + bool ephemeral_default; ASTPtr comment; ASTPtr codec; ASTPtr ttl; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index e97033c51f0..6ad7b9e6dd9 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -170,6 +171,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E ASTPtr type; String default_specifier; std::optional null_modifier; + bool ephemeral_default = false; ASTPtr default_expression; ASTPtr comment_expression; ASTPtr codec_expression; @@ -235,8 +237,11 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E else if (s_ephemeral.ignore(pos, expected)) { default_specifier = s_ephemeral.getName(); - if (!literal_parser.parse(pos, default_expression, expected) && type) - default_expression = std::make_shared(Field()); + if (!expr_parser.parse(pos, default_expression, expected) && type) + { + ephemeral_default = true; + default_expression = std::make_shared(DataTypeFactory::instance().get(type)->getDefault()); + } if (!default_expression && !type) return false; @@ -302,6 +307,7 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E column_declaration->default_specifier = default_specifier; if (default_expression) { + column_declaration->ephemeral_default = ephemeral_default; column_declaration->default_expression = default_expression; column_declaration->children.push_back(std::move(default_expression)); } diff --git a/src/Storages/ColumnDefault.h b/src/Storages/ColumnDefault.h index 096a1f177ab..af1be6f3bec 100644 --- a/src/Storages/ColumnDefault.h +++ b/src/Storages/ColumnDefault.h @@ -26,6 +26,7 @@ struct ColumnDefault { ColumnDefaultKind kind = ColumnDefaultKind::Default; ASTPtr expression; + bool ephemeral_default = false; }; diff --git a/src/Storages/ColumnsDescription.cpp b/src/Storages/ColumnsDescription.cpp index 5fa267a964b..0fdb21e064f 100644 --- a/src/Storages/ColumnsDescription.cpp +++ b/src/Storages/ColumnsDescription.cpp @@ -123,6 +123,7 @@ void ColumnDescription::readText(ReadBuffer & buf) { default_desc.kind = columnDefaultKindFromString(col_ast->default_specifier); default_desc.expression = std::move(col_ast->default_expression); + default_desc.ephemeral_default = col_ast->ephemeral_default; } if (col_ast->comment) From b13e68fa5251694aa666db25b4f5f74b7df6f40a Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 11 Dec 2022 03:29:45 +0000 Subject: [PATCH 030/339] column descriptor fixed, test updated and appended --- src/Interpreters/InterpreterCreateQuery.cpp | 6 +----- .../02287_ephemeral_format_crash.reference | 2 ++ .../0_stateless/02287_ephemeral_format_crash.sql | 11 +++++++---- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ad7e6e2db4e..84b8175af32 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -542,11 +542,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( final_column_name)); default_expr_list->children.emplace_back( - setAlias( - col_decl.default_specifier == "EPHEMERAL" ? /// can be ASTLiteral::value NULL - std::make_shared(data_type_ptr->getDefault()) : - col_decl.default_expression->clone(), - tmp_column_name)); + setAlias(col_decl.default_expression->clone(), tmp_column_name)); } else default_expr_list->children.emplace_back(setAlias(col_decl.default_expression->clone(), col_decl.name)); diff --git a/tests/queries/0_stateless/02287_ephemeral_format_crash.reference b/tests/queries/0_stateless/02287_ephemeral_format_crash.reference index e69de29bb2d..39bbe7c68eb 100644 --- a/tests/queries/0_stateless/02287_ephemeral_format_crash.reference +++ b/tests/queries/0_stateless/02287_ephemeral_format_crash.reference @@ -0,0 +1,2 @@ +CREATE TABLE default.test\n(\n `a` UInt8,\n `b` String EPHEMERAL\n)\nENGINE = Memory +CREATE TABLE default.test\n(\n `a` UInt8,\n `b` String EPHEMERAL 1 + 2\n)\nENGINE = Memory diff --git a/tests/queries/0_stateless/02287_ephemeral_format_crash.sql b/tests/queries/0_stateless/02287_ephemeral_format_crash.sql index 8fd9a4b4332..efde057a070 100644 --- a/tests/queries/0_stateless/02287_ephemeral_format_crash.sql +++ b/tests/queries/0_stateless/02287_ephemeral_format_crash.sql @@ -1,10 +1,13 @@ DROP TABLE IF EXISTS test; CREATE TABLE test(a UInt8, b String EPHEMERAL) Engine=Memory(); - +SHOW CREATE TABLE test; DROP TABLE test; -CREATE TABLE test(a UInt8, b EPHEMERAL String) Engine=Memory(); -- { clientError SYNTAX_ERROR } +CREATE TABLE test(a UInt8, b EPHEMERAL String) Engine=Memory(); -- { serverError UNKNOWN_IDENTIFIER } CREATE TABLE test(a UInt8, b EPHEMERAL 'a' String) Engine=Memory(); -- { clientError SYNTAX_ERROR } -CREATE TABLE test(a UInt8, b String EPHEMERAL test) Engine=Memory(); -- { clientError SYNTAX_ERROR } -CREATE TABLE test(a UInt8, b String EPHEMERAL 1+2) Engine=Memory(); -- { clientError SYNTAX_ERROR } +CREATE TABLE test(a UInt8, b String EPHEMERAL test) Engine=Memory(); -- { serverError UNKNOWN_IDENTIFIER } + +CREATE TABLE test(a UInt8, b String EPHEMERAL 1+2) Engine=Memory(); +SHOW CREATE TABLE test; +DROP TABLE test; \ No newline at end of file From 136af1ddaa27e0d7da2dc3b25dc11de16c8b36e5 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 14 Dec 2022 12:41:00 +0000 Subject: [PATCH 031/339] Fix parsing version from compatibility setting --- src/Core/Settings.cpp | 13 ++++++------- src/Core/Settings.h | 3 ++- src/Core/SettingsChangesHistory.h | 4 +++- .../02503_bad_compatibility_setting.reference | 1 + .../0_stateless/02503_bad_compatibility_setting.sql | 3 +++ 5 files changed, 15 insertions(+), 9 deletions(-) create mode 100644 tests/queries/0_stateless/02503_bad_compatibility_setting.reference create mode 100644 tests/queries/0_stateless/02503_bad_compatibility_setting.sql diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 7bac3f04fc6..fa1a10d22f2 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -148,31 +148,30 @@ std::vector Settings::getAllRegisteredNames() const void Settings::set(std::string_view name, const Field & value) { - BaseSettings::set(name, value); - if (name == "compatibility") - applyCompatibilitySetting(); + applyCompatibilitySetting(value.get()); /// If we change setting that was changed by compatibility setting before /// we should remove it from settings_changed_by_compatibility_setting, /// otherwise the next time we will change compatibility setting /// this setting will be changed too (and we don't want it). else if (settings_changed_by_compatibility_setting.contains(name)) settings_changed_by_compatibility_setting.erase(name); + + BaseSettings::set(name, value); } -void Settings::applyCompatibilitySetting() +void Settings::applyCompatibilitySetting(const String & compatibility_value) { /// First, revert all changes applied by previous compatibility setting for (const auto & setting_name : settings_changed_by_compatibility_setting) resetToDefault(setting_name); settings_changed_by_compatibility_setting.clear(); - String compatibility = getString("compatibility"); /// If setting value is empty, we don't need to change settings - if (compatibility.empty()) + if (compatibility_value.empty()) return; - ClickHouseVersion version(compatibility); + ClickHouseVersion version(compatibility_value); /// Iterate through ClickHouse version in descending order and apply reversed /// changes for each version that is higher that version from compatibility setting for (auto it = settings_changes_history.rbegin(); it != settings_changes_history.rend(); ++it) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5c57d2082f5..d197d66258e 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -921,7 +922,7 @@ struct Settings : public BaseSettings, public IHints<2, Settings void setDefaultValue(const String & name) { resetToDefault(name); } private: - void applyCompatibilitySetting(); + void applyCompatibilitySetting(const String & compatibility); std::unordered_set settings_changed_by_compatibility_setting; }; diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index ee378b295fa..f64c20e520d 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -28,7 +29,8 @@ public: for (const auto & split_element : split) { size_t component; - if (!tryParse(component, split_element)) + ReadBufferFromString buf(split_element); + if (!tryReadIntText(component, buf) || !buf.eof()) throw Exception{ErrorCodes::BAD_ARGUMENTS, "Cannot parse ClickHouse version here: {}", version}; components.push_back(component); } diff --git a/tests/queries/0_stateless/02503_bad_compatibility_setting.reference b/tests/queries/0_stateless/02503_bad_compatibility_setting.reference new file mode 100644 index 00000000000..5b7d2a449a0 --- /dev/null +++ b/tests/queries/0_stateless/02503_bad_compatibility_setting.reference @@ -0,0 +1 @@ + 0 diff --git a/tests/queries/0_stateless/02503_bad_compatibility_setting.sql b/tests/queries/0_stateless/02503_bad_compatibility_setting.sql new file mode 100644 index 00000000000..178c6a87531 --- /dev/null +++ b/tests/queries/0_stateless/02503_bad_compatibility_setting.sql @@ -0,0 +1,3 @@ +set compatibility='a.a'; -- { serverError BAD_ARGUMENTS } +select value, changed from system.settings where name = 'compatibility' + From 5895fcc21d791c14ad4ba3d63765fb4eb1328ac3 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 14 Dec 2022 13:42:10 +0000 Subject: [PATCH 032/339] Forbid paths in timezone names --- src/DataTypes/TimezoneMixin.h | 23 ++++++++++++++++++- ...orbid_paths_in_datetime_timezone.reference | 0 ...2505_forbid_paths_in_datetime_timezone.sql | 5 ++++ 3 files changed, 27 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.reference create mode 100644 tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index 03ecde5dd0a..cfba795c409 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -2,6 +2,14 @@ #include #include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} +} + class DateLUTImpl; /** Mixin-class that manages timezone info for timezone-aware DateTime implementations @@ -15,7 +23,7 @@ public: explicit TimezoneMixin(const String & time_zone_name = "") : has_explicit_time_zone(!time_zone_name.empty()) - , time_zone(DateLUT::instance(time_zone_name)) + , time_zone(DateLUT::instance(checkTimezoneName(time_zone_name))) , utc_time_zone(DateLUT::instance("UTC")) { } @@ -29,4 +37,17 @@ protected: const DateLUTImpl & time_zone; const DateLUTImpl & utc_time_zone; + +private: + static const String & checkTimezoneName(const String & timezone_name) + { + const char * forbidden_patterns[] = {"/", "../", "./", "~/"}; + for (const auto & pattern : forbidden_patterns) + { + if (timezone_name.starts_with(pattern)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Timezone name cannot start with '{}'", pattern); + } + + return timezone_name; + } }; diff --git a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.reference b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql new file mode 100644 index 00000000000..7cea89837df --- /dev/null +++ b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql @@ -0,0 +1,5 @@ +select toDateTime(0, '/abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, './abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, '../abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, '~/abc'); -- { serverError BAD_ARGUMENTS } + From 57a315931a8978f453693537d96f6b40499fe7da Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 14 Dec 2022 22:05:40 +0000 Subject: [PATCH 033/339] link dbms to parsers examples --- src/Parsers/examples/CMakeLists.txt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 82ca7bc0688..7a422e833d8 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -4,7 +4,7 @@ clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS}) -target_link_libraries(select_parser PRIVATE clickhouse_parsers) +target_link_libraries(select_parser PRIVATE clickhouse_parsers dbms) clickhouse_add_executable(create_parser create_parser.cpp ${SRCS}) -target_link_libraries(create_parser PRIVATE clickhouse_parsers) +target_link_libraries(create_parser PRIVATE clickhouse_parsers dbms) From 0a303433fcc99ea7ef7286d34db86e2fd61792d4 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Thu, 15 Dec 2022 17:06:09 +0800 Subject: [PATCH 034/339] make no limits on the maximum size of the result for the view --- src/Storages/StorageView.cpp | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index a55d7ad3c09..a43fe96396b 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -81,6 +81,20 @@ bool hasJoin(const ASTSelectWithUnionQuery & ast) return false; } +/** There are no limits on the maximum size of the result for the view. + * Since the result of the view is not the result of the entire query. + */ +ContextPtr getViewContext(ContextPtr context) +{ + auto view_context = Context::createCopy(context); + Settings view_settings = context->getSettings(); + view_settings.max_result_rows = 0; + view_settings.max_result_bytes = 0; + view_settings.extremes = false; + view_context->setSettings(view_settings); + return view_context; +} + } StorageView::StorageView( @@ -123,7 +137,7 @@ void StorageView::read( } auto options = SelectQueryOptions(QueryProcessingStage::Complete, 0, false, query_info.settings_limit_offset_done); - InterpreterSelectWithUnionQuery interpreter(current_inner_query, context, options, column_names); + InterpreterSelectWithUnionQuery interpreter(current_inner_query, getViewContext(context), options, column_names); interpreter.addStorageLimits(*query_info.storage_limits); interpreter.buildQueryPlan(query_plan); From 60959522b6c68e78b30b988e8b7507569794ac27 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Dec 2022 17:31:38 +0000 Subject: [PATCH 035/339] Forbid '../' pattern in the middle --- src/DataTypes/TimezoneMixin.cpp | 18 ++++++++++++++++++ src/DataTypes/TimezoneMixin.h | 12 +----------- ...02505_forbid_paths_in_datetime_timezone.sql | 1 + 3 files changed, 20 insertions(+), 11 deletions(-) create mode 100644 src/DataTypes/TimezoneMixin.cpp diff --git a/src/DataTypes/TimezoneMixin.cpp b/src/DataTypes/TimezoneMixin.cpp new file mode 100644 index 00000000000..46554364fb6 --- /dev/null +++ b/src/DataTypes/TimezoneMixin.cpp @@ -0,0 +1,18 @@ +#include +#include +#include + +const String & TimezoneMixin::checkTimezoneName(const String & timezone_name) +{ + const char * forbidden_beginnings[] = {"/", "./", "~"}; + for (const auto & pattern : forbidden_beginnings) + { + if (timezone_name.starts_with(pattern)) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Timezone name cannot start with '{}'", pattern); + } + + if (timezone_name.find("../") != std::string::npos) + throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Timezone name cannot contain pattern '../'"); + + return timezone_name; +} diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index cfba795c409..002bf33bdd6 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -39,15 +39,5 @@ protected: const DateLUTImpl & utc_time_zone; private: - static const String & checkTimezoneName(const String & timezone_name) - { - const char * forbidden_patterns[] = {"/", "../", "./", "~/"}; - for (const auto & pattern : forbidden_patterns) - { - if (timezone_name.starts_with(pattern)) - throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "Timezone name cannot start with '{}'", pattern); - } - - return timezone_name; - } + static const String & checkTimezoneName(const String & timezone_name); }; diff --git a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql index 7cea89837df..63edad6c9e3 100644 --- a/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql +++ b/tests/queries/0_stateless/02505_forbid_paths_in_datetime_timezone.sql @@ -2,4 +2,5 @@ select toDateTime(0, '/abc'); -- { serverError BAD_ARGUMENTS } select toDateTime(0, './abc'); -- { serverError BAD_ARGUMENTS } select toDateTime(0, '../abc'); -- { serverError BAD_ARGUMENTS } select toDateTime(0, '~/abc'); -- { serverError BAD_ARGUMENTS } +select toDateTime(0, 'abc/../../cba'); -- { serverError BAD_ARGUMENTS } From 24e08d475dd16f5ab4541d9a811ecd23a109bf34 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 15 Dec 2022 17:33:54 +0000 Subject: [PATCH 036/339] Move error code to cpp file --- src/DataTypes/TimezoneMixin.cpp | 8 ++++++++ src/DataTypes/TimezoneMixin.h | 8 -------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/DataTypes/TimezoneMixin.cpp b/src/DataTypes/TimezoneMixin.cpp index 46554364fb6..f749c518d43 100644 --- a/src/DataTypes/TimezoneMixin.cpp +++ b/src/DataTypes/TimezoneMixin.cpp @@ -2,6 +2,14 @@ #include #include +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} +} + const String & TimezoneMixin::checkTimezoneName(const String & timezone_name) { const char * forbidden_beginnings[] = {"/", "./", "~"}; diff --git a/src/DataTypes/TimezoneMixin.h b/src/DataTypes/TimezoneMixin.h index 002bf33bdd6..d4eae41c84a 100644 --- a/src/DataTypes/TimezoneMixin.h +++ b/src/DataTypes/TimezoneMixin.h @@ -2,14 +2,6 @@ #include #include -namespace DB -{ -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} -} - class DateLUTImpl; /** Mixin-class that manages timezone info for timezone-aware DateTime implementations From 7058a5e7ed9c3bd359a1bbd623daa923d01ff201 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 15 Dec 2022 18:37:04 +0100 Subject: [PATCH 037/339] impl --- docker/test/fuzzer/run-fuzzer.sh | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index bd539ca978b..780bafe68eb 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -91,6 +91,19 @@ function configure cp -av --dereference "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d cp -av --dereference "$script_dir"/allow-nullable-key.xml db/config.d + # the following memory limits configuration is copy-pasted from docker/test/stress/run.sh + local total_mem + total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB + total_mem=$(( total_mem*1024 )) # bytes + local max_server_mem + max_server_mem=$((total_mem*75/100)) # 75% + echo "Setting max_server_memory_usage=$max_server_mem" + cat > db/config.d/max_server_memory_usage.xml < + ${max_server_mem} + +EOL + cat > db/config.d/core.xml < From 5d35b19e7e65ca96824c6ad3f3929d9bdc495170 Mon Sep 17 00:00:00 2001 From: lzydmxy <13126752315@163.com> Date: Sat, 17 Dec 2022 00:47:05 +0800 Subject: [PATCH 038/339] Add tests --- .../02501_limits_on_result_for_view.reference | 1 + .../02501_limits_on_result_for_view.sql | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02501_limits_on_result_for_view.reference create mode 100644 tests/queries/0_stateless/02501_limits_on_result_for_view.sql diff --git a/tests/queries/0_stateless/02501_limits_on_result_for_view.reference b/tests/queries/0_stateless/02501_limits_on_result_for_view.reference new file mode 100644 index 00000000000..0691f67b202 --- /dev/null +++ b/tests/queries/0_stateless/02501_limits_on_result_for_view.reference @@ -0,0 +1 @@ +52 diff --git a/tests/queries/0_stateless/02501_limits_on_result_for_view.sql b/tests/queries/0_stateless/02501_limits_on_result_for_view.sql new file mode 100644 index 00000000000..17e6024d973 --- /dev/null +++ b/tests/queries/0_stateless/02501_limits_on_result_for_view.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS 02501_test; +DROP TABLE IF EXISTS 02501_dist; +DROP VIEW IF EXISTS 02501_view; + + +-- create local table +CREATE TABLE 02501_test(`a` UInt64) ENGINE = Memory; + +-- create dist table +CREATE TABLE 02501_dist(`a` UInt64) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), 02501_test); + +-- create view +CREATE VIEW 02501_view(`a` UInt64) AS SELECT a FROM 02501_dist; + +-- insert data +insert into 02501_test values(5),(6),(7),(8); + +-- test +SELECT * from 02501_view settings max_result_rows = 1; -- { serverError 396 } +SELECT sum(a) from 02501_view settings max_result_rows = 1; + + +DROP TABLE IF EXISTS 02501_test; +DROP TABLE IF EXISTS 02501_dist; +DROP VIEW IF EXISTS 02501_view; \ No newline at end of file From 22e70fa71e4af6f293cce4d301677bf54c39b0a7 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Dec 2022 18:09:42 +0000 Subject: [PATCH 039/339] Try fix tests --- src/AggregateFunctions/AggregateFunctionFactory.cpp | 9 ++++++--- src/AggregateFunctions/IAggregateFunction.h | 3 +++ src/Processors/Transforms/WindowTransform.cpp | 3 ++- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index a8385ad8b59..38cc355b857 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -72,9 +72,12 @@ AggregateFunctionPtr AggregateFunctionFactory::get( { auto types_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types); - /// If one of the types is Nullable, we apply aggregate function combinator "Null". - - if (std::any_of(types_without_low_cardinality.begin(), types_without_low_cardinality.end(), + /// If one of the types is Nullable, we apply aggregate function combinator "Null" if it's not window function. + /// Window functions are not real aggregate functions. Applying combinators doesn't make sense for them, + /// they must handle the nullability themselves + auto properties = tryGetPropertiesImpl(name); + bool is_window_function = properties.has_value() && properties->is_window_function; + if (!is_window_function && std::any_of(types_without_low_cardinality.begin(), types_without_low_cardinality.end(), [](const auto & type) { return type->isNullable(); })) { AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null"); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index b228de3c6bc..cc32ec5eaec 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -832,6 +832,9 @@ struct AggregateFunctionProperties * Some may also name this property as "non-commutative". */ bool is_order_dependent = false; + + /// Indicates if it's actually window function. + bool is_window_function = false; }; diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 4d3eb1f0bbd..de1e3199f75 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -2339,7 +2339,8 @@ void registerWindowFunctions(AggregateFunctionFactory & factory) .returns_default_when_only_null = true, // This probably doesn't make any difference for window functions because // it is an Aggregator-specific setting. - .is_order_dependent = true }; + .is_order_dependent = true, + .is_window_function = true}; factory.registerFunction("rank", {[](const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) From 70e011ff900792fe7d824826d6b7623afb2c6c1b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 04:58:51 +0100 Subject: [PATCH 040/339] Add a test for PowerBI --- .../0_stateless/02503_mysql_compat_utc_timestamp.reference | 0 tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql | 2 ++ 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference create mode 100644 tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql diff --git a/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql new file mode 100644 index 00000000000..fa327a38f21 --- /dev/null +++ b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql @@ -0,0 +1,2 @@ +-- PowerBI is doing this query. It should work at least somehow, not necessarily in the same way as in MySQL. +SELECT TIMEDIFF(NOW(), UTC_TIMESTAMP()); From 2d3192b0830e197710273281a083bc5badce7446 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 04:59:31 +0100 Subject: [PATCH 041/339] Add a test for PowerBI --- .../0_stateless/02503_mysql_compat_utc_timestamp.reference | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference index e69de29bb2d..573541ac970 100644 --- a/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference +++ b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.reference @@ -0,0 +1 @@ +0 From 4bc76dfdb4fbedbd2f35ccb99baf89fafef78a35 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 06:20:56 +0100 Subject: [PATCH 042/339] Add a test for #36038 --- .../02504_parse_datetime_best_effort_calebeaires.reference | 1 + .../02504_parse_datetime_best_effort_calebeaires.sql | 5 +++++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.reference create mode 100644 tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.sql diff --git a/tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.reference b/tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.reference new file mode 100644 index 00000000000..f5420eeb6ba --- /dev/null +++ b/tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.reference @@ -0,0 +1 @@ +('Date','2148-06-07') ('Date32','1969-01-01') ('DateTime','2105-02-07 17:10:16') ('DateTime','2105-02-07 17:10:16') ('DateTime64(3)','1969-01-01 10:42:00.000') diff --git a/tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.sql b/tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.sql new file mode 100644 index 00000000000..74b974d7440 --- /dev/null +++ b/tests/queries/0_stateless/02504_parse_datetime_best_effort_calebeaires.sql @@ -0,0 +1,5 @@ +CREATE TEMPORARY TABLE my_table (col_date Date, col_date32 Date32, col_datetime DateTime, col_datetime32 DateTime32, col_datetime64 DateTime64); +insert into `my_table` (`col_date`, `col_date32`, `col_datetime`, `col_datetime32`, `col_datetime64`) values (parseDateTime64BestEffort('1969-01-01'), '1969-01-01', parseDateTime64BestEffort('1969-01-01 10:42:00'), parseDateTime64BestEffort('1969-01-01 10:42:00'), parseDateTime64BestEffort('1969-01-01 10:42:00')); + +-- The values for Date32 and DateTime64 will be year 1969, while the values of Date, DateTime will contain a value affected by implementation-defined overflow and can be arbitrary. +SELECT * APPLY(x -> (toTypeName(x), x)) FROM my_table; From 72d4c7637ff10344314e06fe88146edb9e9d8a51 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 06:39:57 +0100 Subject: [PATCH 043/339] Add a test for #29386 --- .../02506_date_time64_floating_point_negative_value.reference | 4 ++++ .../02506_date_time64_floating_point_negative_value.sql | 4 ++++ 2 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.reference create mode 100644 tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.sql diff --git a/tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.reference b/tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.reference new file mode 100644 index 00000000000..b5d0547dc4a --- /dev/null +++ b/tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.reference @@ -0,0 +1,4 @@ +-3600001 +-1 +1970-01-01 00:59:59.999 +1969-12-31 23:59:59.999 diff --git a/tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.sql b/tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.sql new file mode 100644 index 00000000000..dd663c7806e --- /dev/null +++ b/tests/queries/0_stateless/02506_date_time64_floating_point_negative_value.sql @@ -0,0 +1,4 @@ +select toUnixTimestamp64Milli(toDateTime64('1969-12-31 23:59:59.999', 3, 'Europe/Amsterdam')); +select toUnixTimestamp64Milli(toDateTime64('1969-12-31 23:59:59.999', 3, 'UTC')); +select fromUnixTimestamp64Milli(toInt64(-1), 'Europe/Amsterdam'); +select fromUnixTimestamp64Milli(toInt64(-1), 'UTC'); From 180b927065c2156c6291c59014877a54ae309b77 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 06:45:16 +0100 Subject: [PATCH 044/339] Add a test for #22929 --- .../0_stateless/02507_to_unix_timestamp_overflow.reference | 1 + tests/queries/0_stateless/02507_to_unix_timestamp_overflow.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02507_to_unix_timestamp_overflow.reference create mode 100644 tests/queries/0_stateless/02507_to_unix_timestamp_overflow.sql diff --git a/tests/queries/0_stateless/02507_to_unix_timestamp_overflow.reference b/tests/queries/0_stateless/02507_to_unix_timestamp_overflow.reference new file mode 100644 index 00000000000..f6e8cd50296 --- /dev/null +++ b/tests/queries/0_stateless/02507_to_unix_timestamp_overflow.reference @@ -0,0 +1 @@ +-1293882467 diff --git a/tests/queries/0_stateless/02507_to_unix_timestamp_overflow.sql b/tests/queries/0_stateless/02507_to_unix_timestamp_overflow.sql new file mode 100644 index 00000000000..42479f6dbec --- /dev/null +++ b/tests/queries/0_stateless/02507_to_unix_timestamp_overflow.sql @@ -0,0 +1,2 @@ +SELECT toUnixTimestamp(toDateTime64('1928-12-31 12:12:12.123', 3, 'UTC')); -- { serverError DECIMAL_OVERFLOW } +SELECT toInt64(toDateTime64('1928-12-31 12:12:12.123', 3, 'UTC')); From 1795c583a11098eb2b679bf13ff84092859b6b8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 08:00:32 +0100 Subject: [PATCH 045/339] Add a test for #29883 --- programs/server/config.d/graphite.xml | 1 + tests/queries/0_stateless/02508_bad_graphite.reference | 0 tests/queries/0_stateless/02508_bad_graphite.sql | 6 ++++++ 3 files changed, 7 insertions(+) create mode 120000 programs/server/config.d/graphite.xml create mode 100644 tests/queries/0_stateless/02508_bad_graphite.reference create mode 100644 tests/queries/0_stateless/02508_bad_graphite.sql diff --git a/programs/server/config.d/graphite.xml b/programs/server/config.d/graphite.xml new file mode 120000 index 00000000000..69a0411e243 --- /dev/null +++ b/programs/server/config.d/graphite.xml @@ -0,0 +1 @@ +../../../tests/config/config.d/graphite.xml \ No newline at end of file diff --git a/tests/queries/0_stateless/02508_bad_graphite.reference b/tests/queries/0_stateless/02508_bad_graphite.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02508_bad_graphite.sql b/tests/queries/0_stateless/02508_bad_graphite.sql new file mode 100644 index 00000000000..a0ca9dcf690 --- /dev/null +++ b/tests/queries/0_stateless/02508_bad_graphite.sql @@ -0,0 +1,6 @@ +DROP TABLE IF EXISTS test_graphite; +create table test_graphite (key UInt32, Path String, Time DateTime('UTC'), Value UInt8, Version UInt32, col UInt64) + engine = GraphiteMergeTree('graphite_rollup') order by key; + +INSERT INTO test_graphite (key) VALUES (0); -- { serverError BAD_ARGUMENTS } +DROP TABLE test_graphite; From 3e8dcdc155fd86cdbcf0775d5d5c3f044070c988 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 08:01:43 +0100 Subject: [PATCH 046/339] Add earlier check --- src/Processors/Merges/Algorithms/Graphite.cpp | 3 +-- .../Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp | 9 +++++++++ src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 +- 3 files changed, 11 insertions(+), 3 deletions(-) diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp index c5c611366ff..0616c4bd6e6 100644 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -332,8 +332,7 @@ std::string buildTaggedRegex(std::string regexp_str) * * */ -static const Pattern & -appendGraphitePattern( +static const Pattern & appendGraphitePattern( const Poco::Util::AbstractConfiguration & config, const String & config_element, Patterns & patterns, bool default_rule, diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index 467ded19f4d..c5937fe0bc5 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -9,6 +9,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + static GraphiteRollupSortedAlgorithm::ColumnsDefinition defineColumns( const Block & header, const Graphite::Params & params) { @@ -26,6 +32,9 @@ static GraphiteRollupSortedAlgorithm::ColumnsDefinition defineColumns( if (i != def.time_column_num && i != def.value_column_num && i != def.version_column_num) def.unmodified_column_numbers.push_back(i); + if (!WhichDataType(header.getByPosition(def.value_column_num).type).isFloat64()) + throw Exception("Only `Float64` data type is allowed for the value column of GraphiteMergeTree", ErrorCodes::BAD_ARGUMENTS); + return def; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index ae2abaf8ea5..c34d7d4dacf 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -468,7 +468,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) { String graphite_config_name; String error_msg - = "Last parameter of GraphiteMergeTree must be name (in single quotes) of element in configuration file with Graphite options"; + = "Last parameter of GraphiteMergeTree must be the name (in single quotes) of the element in configuration file with the Graphite options"; error_msg += getMergeTreeVerboseHelp(is_extended_storage_def); if (const auto * ast = engine_args[arg_cnt - 1]->as()) From 71deb8ea3199deda36b0360d2832d36765fdab2d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Dec 2022 12:56:31 +0300 Subject: [PATCH 047/339] Fix Docker Suggested by Lupsik Pupsik in Telegram. --- docker/server/entrypoint.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 52f4f67281e..5b64d22861e 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -120,8 +120,8 @@ if [ -n "$(ls /docker-entrypoint-initdb.d/)" ] || [ -n "$CLICKHOUSE_DB" ]; then pid="$!" # check if clickhouse is ready to accept connections - # will try to send ping clickhouse via http_port (max 12 retries by default, with 1 sec timeout and 1 sec delay between retries) - tries=${CLICKHOUSE_INIT_TIMEOUT:-12} + # will try to send ping clickhouse via http_port (max 1000 retries by default, with 1 sec timeout and 1 sec delay between retries) + tries=${CLICKHOUSE_INIT_TIMEOUT:-1000} while ! wget --spider --no-check-certificate -T 1 -q "$URL" 2>/dev/null; do if [ "$tries" -le "0" ]; then echo >&2 'ClickHouse init process failed.' From b2fb8f870ea993e3a1789858c371b62e59f14d4d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Dec 2022 05:48:28 +0100 Subject: [PATCH 048/339] Add a test for #22160 --- .../02508_index_analysis_to_date_timezone.reference | 11 +++++++++++ .../02508_index_analysis_to_date_timezone.sql | 10 ++++++++++ 2 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/02508_index_analysis_to_date_timezone.reference create mode 100644 tests/queries/0_stateless/02508_index_analysis_to_date_timezone.sql diff --git a/tests/queries/0_stateless/02508_index_analysis_to_date_timezone.reference b/tests/queries/0_stateless/02508_index_analysis_to_date_timezone.reference new file mode 100644 index 00000000000..28c3774e947 --- /dev/null +++ b/tests/queries/0_stateless/02508_index_analysis_to_date_timezone.reference @@ -0,0 +1,11 @@ +4c36abda-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c408902-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c5bf20a-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c61623a-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c6efab2-8bd8-11eb-a952-005056aa8bf6 2021-03-24 01:04:27 1 +--- +4c36abda-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c408902-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c5bf20a-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c61623a-8bd8-11eb-8204-005056aa8bf6 2021-03-24 01:04:27 1 +4c6efab2-8bd8-11eb-a952-005056aa8bf6 2021-03-24 01:04:27 1 diff --git a/tests/queries/0_stateless/02508_index_analysis_to_date_timezone.sql b/tests/queries/0_stateless/02508_index_analysis_to_date_timezone.sql new file mode 100644 index 00000000000..a7e4f6e7a0e --- /dev/null +++ b/tests/queries/0_stateless/02508_index_analysis_to_date_timezone.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS table; +CREATE TABLE table (uid UUID, date DateTime('Asia/Kamchatka')) ENGINE = MergeTree ORDER BY date; + +INSERT INTO `table` VALUES ('4c36abda-8bd8-11eb-8204-005056aa8bf6', '2021-03-24 01:04:27'), ('4c408902-8bd8-11eb-8204-005056aa8bf6', '2021-03-24 01:04:27'), ('4c5bf20a-8bd8-11eb-8204-005056aa8bf6', '2021-03-24 01:04:27'), ('4c61623a-8bd8-11eb-8204-005056aa8bf6', '2021-03-24 01:04:27'), ('4c6efab2-8bd8-11eb-a952-005056aa8bf6', '2021-03-24 01:04:27'); + +SELECT uid, date, toDate(date) = toDate('2021-03-24') AS res FROM table WHERE res = 1 ORDER BY uid, date; +SELECT '---'; +SELECT uid, date, toDate(date) = toDate('2021-03-24') AS res FROM table WHERE toDate(date) = toDate('2021-03-24') ORDER BY uid, date; + +DROP TABLE table; From c3c9ebc14fd33661cfb4335b6b1c2bc6fc566a87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Dec 2022 06:38:26 +0100 Subject: [PATCH 049/339] Add a test for #34708 --- .../0_stateless/02509_h3_arguments.reference | 23 +++++++++++++++++++ .../0_stateless/02509_h3_arguments.sql | 16 +++++++++++++ 2 files changed, 39 insertions(+) create mode 100644 tests/queries/0_stateless/02509_h3_arguments.reference create mode 100644 tests/queries/0_stateless/02509_h3_arguments.sql diff --git a/tests/queries/0_stateless/02509_h3_arguments.reference b/tests/queries/0_stateless/02509_h3_arguments.reference new file mode 100644 index 00000000000..3054598cf87 --- /dev/null +++ b/tests/queries/0_stateless/02509_h3_arguments.reference @@ -0,0 +1,23 @@ +583031433791012863 +583031433791012863 +587531185127686143 +614047082918969343 +614047153853038591 +614048195802038271 +614054260742553599 +614054419345965055 +614552348391374847 +614553222213795839 +614554538768072703 +614555412668088319 +614790495813500927 +614047082918969343 +614047153853038591 +614048195802038271 +614054260742553599 +614054419345965055 +614552348391374847 +614553222213795839 +614554538768072703 +614555412668088319 +614790495813500927 diff --git a/tests/queries/0_stateless/02509_h3_arguments.sql b/tests/queries/0_stateless/02509_h3_arguments.sql new file mode 100644 index 00000000000..268012ecf4f --- /dev/null +++ b/tests/queries/0_stateless/02509_h3_arguments.sql @@ -0,0 +1,16 @@ +select h3ToParent(641573946153969375, 1); +select h3ToParent(641573946153969375, arrayJoin([1,2])); + +DROP TABLE IF EXISTS data_table; + +CREATE TABLE data_table (id UInt64, longitude Float64, latitude Float64) ENGINE=MergeTree ORDER BY id; +INSERT INTO data_table SELECT number, number, number FROM numbers(10); +SELECT geoToH3(longitude, latitude, toUInt8(8)) AS h3Index FROM data_table ORDER BY 1; + +DROP TABLE data_table; + +CREATE TABLE data_table (id UInt64, longitude Float64, latitude Float64) ENGINE=MergeTree ORDER BY id; +INSERT INTO data_table SELECT number, number, number FROM numbers(10); +SELECT geoToH3(longitude, latitude, toUInt8(longitude - longitude + 8)) AS h3Index FROM data_table ORDER BY 1; + +DROP TABLE data_table; From fddbad57486f0feb61f74572b5dc71afc4828c06 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Dec 2022 06:49:42 +0100 Subject: [PATCH 050/339] Add a test for #30679 --- .../02510_group_by_prewhere_null.reference | 1 + .../02510_group_by_prewhere_null.sql | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+) create mode 100644 tests/queries/0_stateless/02510_group_by_prewhere_null.reference create mode 100644 tests/queries/0_stateless/02510_group_by_prewhere_null.sql diff --git a/tests/queries/0_stateless/02510_group_by_prewhere_null.reference b/tests/queries/0_stateless/02510_group_by_prewhere_null.reference new file mode 100644 index 00000000000..d2bd2bb4dc6 --- /dev/null +++ b/tests/queries/0_stateless/02510_group_by_prewhere_null.reference @@ -0,0 +1 @@ +1 6 diff --git a/tests/queries/0_stateless/02510_group_by_prewhere_null.sql b/tests/queries/0_stateless/02510_group_by_prewhere_null.sql new file mode 100644 index 00000000000..90a638d0b5c --- /dev/null +++ b/tests/queries/0_stateless/02510_group_by_prewhere_null.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS table1; + +create table table1 ( + col1 Int32, + col2 Int32 +) +ENGINE = MergeTree +partition by tuple() +order by col1; + +INSERT INTO table1 VALUES (1, 2), (1, 4); + +with NULL as pid +select a.col1, sum(a.col2) as summ +from table1 a +prewhere (pid is null or a.col2 = pid) +group by a.col1; + +with 123 as pid +select a.col1, sum(a.col2) as summ +from table1 a +prewhere (pid is null or a.col2 = pid) +group by a.col1; + +DROP TABLE table1; From 8b6092b32145dff5d52411016c83afa25231db62 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Dec 2022 07:26:54 +0100 Subject: [PATCH 051/339] Fix support for complex parameters of parametric aggregate functions, #30975 --- src/AggregateFunctions/AggregateFunctionSumMap.h | 6 ++++-- src/DataTypes/DataTypeAggregateFunction.cpp | 2 +- src/Parsers/ParserDataType.cpp | 3 +-- ...plex_literals_as_aggregate_function_parameters.reference | 4 ++++ ...11_complex_literals_as_aggregate_function_parameters.sql | 4 ++++ 5 files changed, 14 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.reference create mode 100644 tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.sql diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 1e32be987ff..d7dc6b19203 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -461,6 +461,7 @@ public: bool keepKey(const T &) const { return true; } }; + template class AggregateFunctionSumMapFiltered final : public AggregateFunctionMapBaseparameters = params_; + keys_to_keep.reserve(keys_to_keep_values.size()); for (const Field & f : keys_to_keep_values) keys_to_keep.emplace(f.safeGet()); } - String getName() const override - { return overflow ? "sumMapFilteredWithOverflow" : "sumMapFiltered"; } + String getName() const override { return overflow ? "sumMapFilteredWithOverflow" : "sumMapFiltered"; } bool keepKey(const T & key) const { return keys_to_keep.count(key); } }; diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 7056fcff42f..ab6d024f5d8 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -67,7 +67,7 @@ String DataTypeAggregateFunction::getNameImpl(bool with_version) const if (!parameters.empty()) { stream << '('; - for (size_t i = 0; i < parameters.size(); ++i) + for (size_t i = 0, size = parameters.size(); i < size; ++i) { if (i) stream << ", "; diff --git a/src/Parsers/ParserDataType.cpp b/src/Parsers/ParserDataType.cpp index a1a24c40ac2..e152589dc10 100644 --- a/src/Parsers/ParserDataType.cpp +++ b/src/Parsers/ParserDataType.cpp @@ -27,7 +27,7 @@ private: { ParserNestedTable nested_parser; ParserDataType data_type_parser; - ParserLiteral literal_parser; + ParserAllCollectionsOfLiterals literal_parser; const char * operators[] = {"=", "equals", nullptr}; ParserLeftAssociativeBinaryOperatorList enum_parser(operators, std::make_unique()); @@ -145,4 +145,3 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } - diff --git a/tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.reference b/tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.reference new file mode 100644 index 00000000000..ab6afce21ef --- /dev/null +++ b/tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.reference @@ -0,0 +1,4 @@ +AggregateFunction(1, sumMapFiltered([1, 2]), Array(UInt8), Array(UInt8)) +02010A00000000000000020A00000000000000 +02010A00000000000000020A00000000000000 +([1,2],[20,20]) diff --git a/tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.sql b/tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.sql new file mode 100644 index 00000000000..92b5f0143ed --- /dev/null +++ b/tests/queries/0_stateless/02511_complex_literals_as_aggregate_function_parameters.sql @@ -0,0 +1,4 @@ +SELECT toTypeName(sumMapFilteredState([1, 2])([1, 2, 3], [10, 10, 10])); +SELECT hex(sumMapFilteredState([1, 2])([1, 2, 3], [10, 10, 10])); +SELECT hex(unhex('02010A00000000000000020A00000000000000')::AggregateFunction(1, sumMapFiltered([1, 2]), Array(UInt8), Array(UInt8))); +SELECT sumMapFilteredMerge([1, 2])(*) FROM remote('127.0.0.{1,2}', view(SELECT sumMapFilteredState([1, 2])([1, 2, 3], [10, 10, 10]))); From 06652b7d1fcab53ace47a0a85f121187cd698165 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Dec 2022 08:57:26 +0100 Subject: [PATCH 052/339] Add a test for #34724 --- ...ert_without_materialized_columns.reference | 2 ++ ...13_insert_without_materialized_columns.sql | 20 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/02513_insert_without_materialized_columns.reference create mode 100644 tests/queries/0_stateless/02513_insert_without_materialized_columns.sql diff --git a/tests/queries/0_stateless/02513_insert_without_materialized_columns.reference b/tests/queries/0_stateless/02513_insert_without_materialized_columns.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02513_insert_without_materialized_columns.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02513_insert_without_materialized_columns.sql b/tests/queries/0_stateless/02513_insert_without_materialized_columns.sql new file mode 100644 index 00000000000..b16f9715b2f --- /dev/null +++ b/tests/queries/0_stateless/02513_insert_without_materialized_columns.sql @@ -0,0 +1,20 @@ +-- Tags: no-parallel + +DROP TABLE IF EXISTS test; +create table test ( + a Int64, + b Int64 materialized a +) +engine = MergeTree() +primary key tuple(); + +insert into test values (1); +SELECT * FROM test; + +select * from test into outfile 'test.native.zstd' format Native; +truncate table test; +insert into test from infile 'test.native.zstd'; + +SELECT * FROM test; + +DROP TABLE test; From e262e375dc7986637ddf9e19035d95485784ed0e Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 19 Dec 2022 20:30:54 +0000 Subject: [PATCH 053/339] Fix reading columns that are not presented in input data in Parquet/ORC formats --- .../Formats/Impl/ArrowBlockInputFormat.cpp | 2 +- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 10 +++------- .../Formats/Impl/ArrowColumnToCHColumn.h | 4 ++-- .../Formats/Impl/ORCBlockInputFormat.cpp | 9 +++++++-- .../Formats/Impl/ParquetBlockInputFormat.cpp | 2 +- ...02511_parquet_orc_missing_columns.reference | 8 ++++++++ .../02511_parquet_orc_missing_columns.sh | 18 ++++++++++++++++++ 7 files changed, 40 insertions(+), 13 deletions(-) create mode 100644 tests/queries/0_stateless/02511_parquet_orc_missing_columns.reference create mode 100755 tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp index cae4cbab0d7..20cf5e25ce5 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.cpp @@ -71,7 +71,7 @@ Chunk ArrowBlockInputFormat::generate() ++record_batch_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result); + arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, (*table_result)->num_rows()); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 8b546f48116..30fd3399c31 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -810,7 +810,7 @@ ArrowColumnToCHColumn::ArrowColumnToCHColumn( { } -void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table) +void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows) { NameToColumnPtr name_to_column_ptr; for (auto column_name : table->ColumnNames()) @@ -824,16 +824,12 @@ void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptrsecond->length(); columns_list.reserve(header.columns()); std::unordered_map>> nested_tables; bool skipped = false; diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h index 3540778940e..dd9f44eb94e 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.h @@ -28,9 +28,9 @@ public: bool allow_missing_columns_, bool case_insensitive_matching_ = false); - void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table); + void arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, size_t num_rows); - void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr); + void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr, size_t num_rows); /// Get missing columns that exists in header but not in arrow::Schema std::vector getMissingColumns(const arrow::Schema & schema) const; diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 58fd03a7a78..fa17961e6f3 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -54,14 +54,19 @@ Chunk ORCBlockInputFormat::generate() throw ParsingException( ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_result.status().ToString()); + /// We should extract the number of rows directly from the stripe, because in case when + /// record batch contains 0 columns (for example if we requested only columns that + /// are not presented in data) the number of rows in record batch will be 0. + size_t num_rows = file_reader->GetRawORCReader()->getStripe(stripe_current)->getNumberOfRows(); + auto table = table_result.ValueOrDie(); - if (!table || !table->num_rows()) + if (!table || !num_rows) return {}; ++stripe_current; Chunk res; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table); + arrow_column_to_ch_column->arrowTableToCHChunk(res, table, num_rows); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. if (format_settings.defaults_for_omitted_fields) diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index c2253fe4b20..9e5f54ba455 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -70,7 +70,7 @@ Chunk ParquetBlockInputFormat::generate() ++row_group_current; - arrow_column_to_ch_column->arrowTableToCHChunk(res, table); + arrow_column_to_ch_column->arrowTableToCHChunk(res, table, table->num_rows()); /// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields. /// Otherwise fill the missing columns with zero values of its type. diff --git a/tests/queries/0_stateless/02511_parquet_orc_missing_columns.reference b/tests/queries/0_stateless/02511_parquet_orc_missing_columns.reference new file mode 100644 index 00000000000..d5318a96f1a --- /dev/null +++ b/tests/queries/0_stateless/02511_parquet_orc_missing_columns.reference @@ -0,0 +1,8 @@ +Hello +Hello +Hello +6 6 +Hello +Hello +Hello +6 6 diff --git a/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh b/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh new file mode 100755 index 00000000000..780504b8807 --- /dev/null +++ b/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh @@ -0,0 +1,18 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL -q "select number as x from numbers(3) format Parquet" > 02511_data1.parquet +$CLICKHOUSE_LOCAL -q "select y from file(02511_data1.parquet, auto, 'x UInt64, y String default \'Hello\'') settings input_format_parquet_allow_missing_columns=1" +$CLICKHOUSE_LOCAL -q "select number as x, 'Hello' as y from numbers(3) format Parquet" > 02511_data2.parquet +$CLICKHOUSE_LOCAL -q "select count(*), count(y) from file('02511_data*.parquet', auto, 'x UInt64, y String') settings input_format_parquet_allow_missing_columns=1" + +$CLICKHOUSE_LOCAL -q "select number as x from numbers(3) format ORC" > 02511_data1.orc +$CLICKHOUSE_LOCAL -q "select y from file(02511_data1.orc, auto, 'x UInt64, y String default \'Hello\'') settings input_format_orc_allow_missing_columns=1" +$CLICKHOUSE_LOCAL -q "select number as x, 'Hello' as y from numbers(3) format ORC" > 02511_data2.orc +$CLICKHOUSE_LOCAL -q "select count(*), count(y) from file('02511_data*.orc', auto, 'x UInt64, y String') settings input_format_orc_allow_missing_columns=1" + +rm 02511_data* + From 4cd91b1ec6c7f2e3da8deb64cd48677244740ea6 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 20 Dec 2022 02:17:02 +0100 Subject: [PATCH 054/339] Test for bar() with fractional width --- .../0_stateless/02504_bar_fractions.reference | 20 +++++++++++++++++++ .../0_stateless/02504_bar_fractions.sql | 7 +++++++ 2 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/02504_bar_fractions.reference create mode 100644 tests/queries/0_stateless/02504_bar_fractions.sql diff --git a/tests/queries/0_stateless/02504_bar_fractions.reference b/tests/queries/0_stateless/02504_bar_fractions.reference new file mode 100644 index 00000000000..2a7b46069df --- /dev/null +++ b/tests/queries/0_stateless/02504_bar_fractions.reference @@ -0,0 +1,20 @@ +0 +0.125 ▏ E2968F +0.25 ▎ ▏ E2968E E2968F +0.375 ▍ ▎ E2968D E2968E +0.5 ▌ ▍ E2968C E2968D +0.625 ▋ ▌ E2968B E2968C +0.75 ▊ ▋ E2968A E2968B +0.875 ▉ ▊ E29689 E2968A +1 █ ▉ E29688 E29689 +1.125 █▏ █ E29688E2968F E29688 +1.25 █▎ █▏ E29688E2968E E29688E2968F +1.375 █▍ █▎ E29688E2968D E29688E2968E +1.5 █▌ █▍ E29688E2968C E29688E2968D +1.625 █▋ █▌ E29688E2968B E29688E2968C +1.75 █▊ █▋ E29688E2968A E29688E2968B +1.875 █▉ █▊ E29688E29689 E29688E2968A +2 ██ █▉ E29688E29688 E29688E29689 +2.125 ██▏ ██ E29688E29688E2968F E29688E29688 +2.25 ██▎ ██▏ E29688E29688E2968E E29688E29688E2968F +2.375 ██▍ ██▎ E29688E29688E2968D E29688E29688E2968E diff --git a/tests/queries/0_stateless/02504_bar_fractions.sql b/tests/queries/0_stateless/02504_bar_fractions.sql new file mode 100644 index 00000000000..d182bced55e --- /dev/null +++ b/tests/queries/0_stateless/02504_bar_fractions.sql @@ -0,0 +1,7 @@ +SELECT + number / 8 AS width, + bar(width, 0, 3, 3) AS b, + bar(width - 0.001, 0, 3, 3) AS `b_minus`, + hex(b), + hex(b_minus) +FROM numbers(20); From 2ab6b7d0334254540d6a2053e71c9c8fef8caaa0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 20 Dec 2022 02:21:23 +0100 Subject: [PATCH 055/339] Check remaining space to avoid buffer overruns; some cleanups --- src/Common/UnicodeBar.cpp | 54 +++++++++++++++++++++++++++++++-------- src/Common/UnicodeBar.h | 2 +- src/Functions/bar.cpp | 2 +- 3 files changed, 45 insertions(+), 13 deletions(-) diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp index efc85ad35e4..259bdf227b2 100644 --- a/src/Common/UnicodeBar.cpp +++ b/src/Common/UnicodeBar.cpp @@ -9,6 +9,13 @@ #include +namespace DB +{ + namespace ErrorCodes + { + extern const int LOGICAL_ERROR; + } +} namespace UnicodeBar { @@ -26,36 +33,61 @@ namespace UnicodeBar return (x - min) / (max - min) * max_width; } + /// We use the following Unicode characters to draw the bar: + /// U+2588 "█" Full block + /// U+2589 "▉" Left seven eighths block + /// U+258A "▊" Left three quarters block + /// U+258B "▋" Left five eighths block + /// U+258C "▌" Left half block + /// U+258D "▍" Left three eighths block + /// U+258E "▎" Left one quarter block + /// U+258F "▏" Left one eighth block + constexpr size_t GRADES_IN_FULL_BAR = 8; + constexpr char FULL_BAR[] = "█"; + constexpr char FRACTIONAL_BARS[] = "▏▎▍▌▋▊▉"; /// 7 elements: 1/8, 2/8, 3/8, 4/8, 5/8, 6/8, 7/8 + size_t getWidthInBytes(double width) { - return static_cast(ceil(width - 1.0 / 8) * UNICODE_BAR_CHAR_SIZE); + Int64 int_width = Int64(width * GRADES_IN_FULL_BAR); + return (int_width / GRADES_IN_FULL_BAR) * UNICODE_BAR_CHAR_SIZE + (int_width % GRADES_IN_FULL_BAR ? UNICODE_BAR_CHAR_SIZE : 0); } - void render(double width, char * dst) + static char* checkedCopy(const char * src, size_t src_size, char * dst, const char * dst_end) { - size_t floor_width = static_cast(floor(width)); + if (dst + src_size > dst_end) + throw DB::Exception( + DB::ErrorCodes::LOGICAL_ERROR, + "Not enough space in buffer for UnicodeBar::render, required: {}, got: {}", + src_size, dst_end - dst); + + memcpy(dst, src, src_size); + return dst + src_size; + } + + void render(double width, char * dst, const char * dst_end) + { + Int64 int_width = Int64(width * GRADES_IN_FULL_BAR); + size_t floor_width = (int_width / GRADES_IN_FULL_BAR); for (size_t i = 0; i < floor_width; ++i) { - memcpy(dst, "█", UNICODE_BAR_CHAR_SIZE); - dst += UNICODE_BAR_CHAR_SIZE; + dst = checkedCopy(FULL_BAR, UNICODE_BAR_CHAR_SIZE, dst, dst_end); } - size_t remainder = static_cast(floor((width - floor_width) * 8)); + size_t remainder = int_width % GRADES_IN_FULL_BAR; if (remainder) { - memcpy(dst, &"▏▎▍▌▋▋▊▉"[(remainder - 1) * UNICODE_BAR_CHAR_SIZE], UNICODE_BAR_CHAR_SIZE); - dst += UNICODE_BAR_CHAR_SIZE; + dst = checkedCopy(&FRACTIONAL_BARS[(remainder - 1) * UNICODE_BAR_CHAR_SIZE], UNICODE_BAR_CHAR_SIZE, dst, dst_end); } - *dst = 0; + dst = checkedCopy("\0", 1, dst, dst_end); } std::string render(double width) { - std::string res(getWidthInBytes(width), '\0'); - render(width, res.data()); + std::string res(getWidthInBytes(width) + 1, '\0'); + render(width, res.data(), res.data() + res.size()); return res; } } diff --git a/src/Common/UnicodeBar.h b/src/Common/UnicodeBar.h index 64705aa5022..78e925bdb3c 100644 --- a/src/Common/UnicodeBar.h +++ b/src/Common/UnicodeBar.h @@ -14,6 +14,6 @@ namespace UnicodeBar size_t getWidthInBytes(double width); /// In `dst` there must be a space for barWidthInBytes(width) characters and a trailing zero. - void render(double width, char * dst); + void render(double width, char * dst, const char * dst_end); std::string render(double width); } diff --git a/src/Functions/bar.cpp b/src/Functions/bar.cpp index 982e1ff3a25..e1f65a61175 100644 --- a/src/Functions/bar.cpp +++ b/src/Functions/bar.cpp @@ -118,7 +118,7 @@ public: size_t next_size = current_offset + UnicodeBar::getWidthInBytes(width) + 1; dst_chars.resize(next_size); - UnicodeBar::render(width, reinterpret_cast(&dst_chars[current_offset])); + UnicodeBar::render(width, reinterpret_cast(&dst_chars[current_offset]), reinterpret_cast(&dst_chars[next_size])); current_offset = next_size; dst_offsets[i] = current_offset; } From 975906361e9cbb086f1b901a77392d5930db884e Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Tue, 20 Dec 2022 12:51:42 +0800 Subject: [PATCH 056/339] fix explain ast insert with data --- src/Parsers/parseQuery.cpp | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/src/Parsers/parseQuery.cpp b/src/Parsers/parseQuery.cpp index 4a0c60da48d..da8450ac301 100644 --- a/src/Parsers/parseQuery.cpp +++ b/src/Parsers/parseQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -263,7 +264,19 @@ ASTPtr tryParseQuery( ASTInsertQuery * insert = nullptr; if (parse_res) - insert = res->as(); + { + if (auto * explain = res->as()) + { + if (auto explained_query = explain->getExplainedQuery()) + { + insert = explained_query->as(); + } + } + else + { + insert = res->as(); + } + } // If parsed query ends at data for insertion. Data for insertion could be // in any format and not necessary be lexical correct, so we can't perform From e127e3d9387876fd78839dbfcfddeb4013ae6d05 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 20 Dec 2022 09:57:34 +0100 Subject: [PATCH 057/339] Update test results, diff is caused by duplicate U+258B in the old version --- .../01044_great_circle_angle.reference | 30 +- .../02020_exponential_smoothing.reference | 260 +++++++++--------- 2 files changed, 145 insertions(+), 145 deletions(-) diff --git a/tests/queries/0_stateless/01044_great_circle_angle.reference b/tests/queries/0_stateless/01044_great_circle_angle.reference index ebdeaa10067..c247e398824 100644 --- a/tests/queries/0_stateless/01044_great_circle_angle.reference +++ b/tests/queries/0_stateless/01044_great_circle_angle.reference @@ -29,24 +29,24 @@ ██████████████████████████████████▎ ████████████████████████████████████▏ ██████████████████████████████████████ -███████████████████████████████████████▊ -█████████████████████████████████████████▋ +███████████████████████████████████████▉ +█████████████████████████████████████████▊ ███████████████████████████████████████████▌ █████████████████████████████████████████████▍ ███████████████████████████████████████████████▏ -████████████████████████████████████████████████▊ +████████████████████████████████████████████████▉ ██████████████████████████████████████████████████▌ ████████████████████████████████████████████████████▏ -█████████████████████████████████████████████████████▊ +█████████████████████████████████████████████████████▉ ███████████████████████████████████████████████████████▍ █████████████████████████████████████████████████████████ ██████████████████████████████████████████████████████████▌ ████████████████████████████████████████████████████████████ █████████████████████████████████████████████████████████████▌ -██████████████████████████████████████████████████████████████▊ +██████████████████████████████████████████████████████████████▉ ████████████████████████████████████████████████████████████████▎ █████████████████████████████████████████████████████████████████▌ -██████████████████████████████████████████████████████████████████▋ +██████████████████████████████████████████████████████████████████▊ ████████████████████████████████████████████████████████████████████ █████████████████████████████████████████████████████████████████████▏ ██████████████████████████████████████████████████████████████████████▎ @@ -59,13 +59,13 @@ ████████████████████████████████████████████████████████████████████████████▍ █████████████████████████████████████████████████████████████████████████████ █████████████████████████████████████████████████████████████████████████████▌ -█████████████████████████████████████████████████████████████████████████████▊ +█████████████████████████████████████████████████████████████████████████████▉ ██████████████████████████████████████████████████████████████████████████████▎ ██████████████████████████████████████████████████████████████████████████████▋ -██████████████████████████████████████████████████████████████████████████████▋ -██████████████████████████████████████████████████████████████████████████████▊ -██████████████████████████████████████████████████████████████████████████████▊ ██████████████████████████████████████████████████████████████████████████████▊ +██████████████████████████████████████████████████████████████████████████████▉ +██████████████████████████████████████████████████████████████████████████████▉ +██████████████████████████████████████████████████████████████████████████████▉ ██████████████████████████████████████████████████████████████████████████████▋ ██████████████████████████████████████████████████████████████████████████████▍ ██████████████████████████████████████████████████████████████████████████████ @@ -84,18 +84,18 @@ ██████████████████████████████████████████████████████████████▌ ████████████████████████████████████████████████████████████▍ ██████████████████████████████████████████████████████████▏ -███████████████████████████████████████████████████████▋ +███████████████████████████████████████████████████████▊ █████████████████████████████████████████████████████▏ ██████████████████████████████████████████████████▍ ███████████████████████████████████████████████▌ ████████████████████████████████████████████▌ █████████████████████████████████████████▎ -█████████████████████████████████████▊ +█████████████████████████████████████▉ ██████████████████████████████████▍ ██████████████████████████████▋ -██████████████████████████▋ -██████████████████████▋ +██████████████████████████▊ +██████████████████████▊ ██████████████████▌ ██████████████▏ █████████▋ -████▊ +████▉ diff --git a/tests/queries/0_stateless/02020_exponential_smoothing.reference b/tests/queries/0_stateless/02020_exponential_smoothing.reference index 95f5cb0c310..1e98bca8a23 100644 --- a/tests/queries/0_stateless/02020_exponential_smoothing.reference +++ b/tests/queries/0_stateless/02020_exponential_smoothing.reference @@ -31,14 +31,14 @@ exponentialMovingAverage 9 9 8.002 1 0 0.067 ███▎ 0 1 0.062 ███ -0 2 0.058 ██▊ +0 2 0.058 ██▉ 0 3 0.054 ██▋ 0 4 0.051 ██▌ 0 5 0.047 ██▎ 0 6 0.044 ██▏ 0 7 0.041 ██ -0 8 0.038 █▊ -0 9 0.036 █▋ +0 8 0.038 █▉ +0 9 0.036 █▊ 0 10 0.033 █▋ 0 11 0.031 █▌ 0 12 0.029 █▍ @@ -47,16 +47,16 @@ exponentialMovingAverage 0 15 0.024 █▏ 0 16 0.022 █ 0 17 0.021 █ -0 18 0.019 ▊ -0 19 0.018 ▊ -0 20 0.017 ▋ -0 21 0.016 ▋ +0 18 0.019 ▉ +0 19 0.018 ▉ +0 20 0.017 ▊ +0 21 0.016 ▊ 0 22 0.015 ▋ 0 23 0.014 ▋ 0 24 0.013 ▋ -1 25 0.079 ███▊ +1 25 0.079 ███▉ 1 26 0.14 ███████ -1 27 0.198 █████████▊ +1 27 0.198 █████████▉ 1 28 0.252 ████████████▌ 1 29 0.302 ███████████████ 1 30 0.349 █████████████████▍ @@ -64,68 +64,68 @@ exponentialMovingAverage 1 32 0.433 █████████████████████▋ 1 33 0.471 ███████████████████████▌ 1 34 0.506 █████████████████████████▎ -1 35 0.539 ██████████████████████████▊ +1 35 0.539 ██████████████████████████▉ 1 36 0.57 ████████████████████████████▌ -1 37 0.599 █████████████████████████████▊ +1 37 0.599 █████████████████████████████▉ 1 38 0.626 ███████████████████████████████▎ 1 39 0.651 ████████████████████████████████▌ 1 40 0.674 █████████████████████████████████▋ -1 41 0.696 ██████████████████████████████████▋ -1 42 0.716 ███████████████████████████████████▋ -1 43 0.735 ████████████████████████████████████▋ +1 41 0.696 ██████████████████████████████████▊ +1 42 0.716 ███████████████████████████████████▊ +1 43 0.735 ████████████████████████████████████▊ 1 44 0.753 █████████████████████████████████████▋ 1 45 0.77 ██████████████████████████████████████▍ 1 46 0.785 ███████████████████████████████████████▎ -1 47 0.8 ███████████████████████████████████████▊ +1 47 0.8 ███████████████████████████████████████▉ 1 48 0.813 ████████████████████████████████████████▋ 1 49 0.825 █████████████████████████████████████████▎ 1 0 0.5 █████████████████████████ 0 1 0.25 ████████████▌ 0 2 0.125 ██████▎ -0 3 0.062 ███ +0 3 0.062 ███▏ 0 4 0.031 █▌ -1 5 0.516 █████████████████████████▋ -0 6 0.258 ████████████▊ +1 5 0.516 █████████████████████████▊ +0 6 0.258 ████████████▉ 0 7 0.129 ██████▍ 0 8 0.064 ███▏ 0 9 0.032 █▌ -1 10 0.516 █████████████████████████▋ -0 11 0.258 ████████████▊ +1 10 0.516 █████████████████████████▊ +0 11 0.258 ████████████▉ 0 12 0.129 ██████▍ 0 13 0.065 ███▏ 0 14 0.032 █▌ -1 15 0.516 █████████████████████████▋ -0 16 0.258 ████████████▊ +1 15 0.516 █████████████████████████▊ +0 16 0.258 ████████████▉ 0 17 0.129 ██████▍ 0 18 0.065 ███▏ 0 19 0.032 █▌ -1 20 0.516 █████████████████████████▋ -0 21 0.258 ████████████▊ +1 20 0.516 █████████████████████████▊ +0 21 0.258 ████████████▉ 0 22 0.129 ██████▍ 0 23 0.065 ███▏ 0 24 0.032 █▌ -1 25 0.516 █████████████████████████▋ -0 26 0.258 ████████████▊ +1 25 0.516 █████████████████████████▊ +0 26 0.258 ████████████▉ 0 27 0.129 ██████▍ 0 28 0.065 ███▏ 0 29 0.032 █▌ -1 30 0.516 █████████████████████████▋ -0 31 0.258 ████████████▊ +1 30 0.516 █████████████████████████▊ +0 31 0.258 ████████████▉ 0 32 0.129 ██████▍ 0 33 0.065 ███▏ 0 34 0.032 █▌ -1 35 0.516 █████████████████████████▋ -0 36 0.258 ████████████▊ +1 35 0.516 █████████████████████████▊ +0 36 0.258 ████████████▉ 0 37 0.129 ██████▍ 0 38 0.065 ███▏ 0 39 0.032 █▌ -1 40 0.516 █████████████████████████▋ -0 41 0.258 ████████████▊ +1 40 0.516 █████████████████████████▊ +0 41 0.258 ████████████▉ 0 42 0.129 ██████▍ 0 43 0.065 ███▏ 0 44 0.032 █▌ -1 45 0.516 █████████████████████████▋ -0 46 0.258 ████████████▊ +1 45 0.516 █████████████████████████▊ +0 46 0.258 ████████████▉ 0 47 0.129 ██████▍ 0 48 0.065 ███▏ 0 49 0.032 █▌ @@ -170,15 +170,15 @@ exponentialTimeDecayedSum 0 7 0.497 ██▍ 0 8 0.449 ██▏ 0 9 0.407 ██ -0 10 0.368 █▋ +0 10 0.368 █▊ 0 11 0.333 █▋ 0 12 0.301 █▌ 0 13 0.273 █▎ 0 14 0.247 █▏ 0 15 0.223 █ 0 16 0.202 █ -0 17 0.183 ▊ -0 18 0.165 ▋ +0 17 0.183 ▉ +0 18 0.165 ▊ 0 19 0.15 ▋ 0 20 0.135 ▋ 0 21 0.122 ▌ @@ -186,80 +186,80 @@ exponentialTimeDecayedSum 0 23 0.1 ▌ 0 24 0.091 ▍ 1 25 1.082 █████▍ -1 26 1.979 █████████▊ -1 27 2.791 █████████████▊ +1 26 1.979 █████████▉ +1 27 2.791 █████████████▉ 1 28 3.525 █████████████████▋ -1 29 4.19 ████████████████████▊ -1 30 4.791 ███████████████████████▊ +1 29 4.19 ████████████████████▉ +1 30 4.791 ███████████████████████▉ 1 31 5.335 ██████████████████████████▋ 1 32 5.827 █████████████████████████████▏ 1 33 6.273 ███████████████████████████████▎ 1 34 6.676 █████████████████████████████████▍ 1 35 7.041 ███████████████████████████████████▏ -1 36 7.371 ████████████████████████████████████▋ +1 36 7.371 ████████████████████████████████████▊ 1 37 7.669 ██████████████████████████████████████▎ 1 38 7.939 ███████████████████████████████████████▋ -1 39 8.184 ████████████████████████████████████████▊ +1 39 8.184 ████████████████████████████████████████▉ 1 40 8.405 ██████████████████████████████████████████ 1 41 8.605 ███████████████████████████████████████████ -1 42 8.786 ███████████████████████████████████████████▊ -1 43 8.95 ████████████████████████████████████████████▋ +1 42 8.786 ███████████████████████████████████████████▉ +1 43 8.95 ████████████████████████████████████████████▊ 1 44 9.098 █████████████████████████████████████████████▍ 1 45 9.233 ██████████████████████████████████████████████▏ -1 46 9.354 ██████████████████████████████████████████████▋ +1 46 9.354 ██████████████████████████████████████████████▊ 1 47 9.464 ███████████████████████████████████████████████▎ -1 48 9.563 ███████████████████████████████████████████████▋ +1 48 9.563 ███████████████████████████████████████████████▊ 1 49 9.653 ████████████████████████████████████████████████▎ 1 0 1 ██████████████████████████████████████████████████ 0 1 0.368 ██████████████████▍ -0 2 0.135 ██████▋ +0 2 0.135 ██████▊ 0 3 0.05 ██▍ -0 4 0.018 ▊ +0 4 0.018 ▉ 1 5 1.007 ██████████████████████████████████████████████████ 0 6 0.37 ██████████████████▌ -0 7 0.136 ██████▋ +0 7 0.136 ██████▊ 0 8 0.05 ██▌ -0 9 0.018 ▊ +0 9 0.018 ▉ 1 10 1.007 ██████████████████████████████████████████████████ 0 11 0.37 ██████████████████▌ -0 12 0.136 ██████▋ +0 12 0.136 ██████▊ 0 13 0.05 ██▌ -0 14 0.018 ▊ +0 14 0.018 ▉ 1 15 1.007 ██████████████████████████████████████████████████ 0 16 0.37 ██████████████████▌ -0 17 0.136 ██████▋ +0 17 0.136 ██████▊ 0 18 0.05 ██▌ -0 19 0.018 ▊ +0 19 0.018 ▉ 1 20 1.007 ██████████████████████████████████████████████████ 0 21 0.37 ██████████████████▌ -0 22 0.136 ██████▋ +0 22 0.136 ██████▊ 0 23 0.05 ██▌ -0 24 0.018 ▊ +0 24 0.018 ▉ 1 25 1.007 ██████████████████████████████████████████████████ 0 26 0.37 ██████████████████▌ -0 27 0.136 ██████▋ +0 27 0.136 ██████▊ 0 28 0.05 ██▌ -0 29 0.018 ▊ +0 29 0.018 ▉ 1 30 1.007 ██████████████████████████████████████████████████ 0 31 0.37 ██████████████████▌ -0 32 0.136 ██████▋ +0 32 0.136 ██████▊ 0 33 0.05 ██▌ -0 34 0.018 ▊ +0 34 0.018 ▉ 1 35 1.007 ██████████████████████████████████████████████████ 0 36 0.37 ██████████████████▌ -0 37 0.136 ██████▋ +0 37 0.136 ██████▊ 0 38 0.05 ██▌ -0 39 0.018 ▊ +0 39 0.018 ▉ 1 40 1.007 ██████████████████████████████████████████████████ 0 41 0.37 ██████████████████▌ -0 42 0.136 ██████▋ +0 42 0.136 ██████▊ 0 43 0.05 ██▌ -0 44 0.018 ▊ +0 44 0.018 ▉ 1 45 1.007 ██████████████████████████████████████████████████ 0 46 0.37 ██████████████████▌ -0 47 0.136 ██████▋ +0 47 0.136 ██████▊ 0 48 0.05 ██▌ -0 49 0.018 ▊ +0 49 0.018 ▉ exponentialTimeDecayedMax 1 0 1 0 1 0.368 @@ -301,15 +301,15 @@ exponentialTimeDecayedMax 0 7 0.497 ██▍ 0 8 0.449 ██▏ 0 9 0.407 ██ -0 10 0.368 █▋ +0 10 0.368 █▊ 0 11 0.333 █▋ 0 12 0.301 █▌ 0 13 0.273 █▎ 0 14 0.247 █▏ 0 15 0.223 █ 0 16 0.202 █ -0 17 0.183 ▊ -0 18 0.165 ▋ +0 17 0.183 ▉ +0 18 0.165 ▊ 0 19 0.15 ▋ 0 20 0.135 ▋ 0 21 0.122 ▌ @@ -343,54 +343,54 @@ exponentialTimeDecayedMax 1 49 1 █████ 1 0 1 ██████████████████████████████████████████████████ 0 1 0.368 ██████████████████▍ -0 2 0.135 ██████▋ +0 2 0.135 ██████▊ 0 3 0.05 ██▍ -0 4 0.018 ▊ +0 4 0.018 ▉ 1 5 1 ██████████████████████████████████████████████████ 0 6 0.368 ██████████████████▍ -0 7 0.135 ██████▋ +0 7 0.135 ██████▊ 0 8 0.05 ██▍ -0 9 0.018 ▊ +0 9 0.018 ▉ 1 10 1 ██████████████████████████████████████████████████ 0 11 0.368 ██████████████████▍ -0 12 0.135 ██████▋ +0 12 0.135 ██████▊ 0 13 0.05 ██▍ -0 14 0.018 ▊ +0 14 0.018 ▉ 1 15 1 ██████████████████████████████████████████████████ 0 16 0.368 ██████████████████▍ -0 17 0.135 ██████▋ +0 17 0.135 ██████▊ 0 18 0.05 ██▍ -0 19 0.018 ▊ +0 19 0.018 ▉ 1 20 1 ██████████████████████████████████████████████████ 0 21 0.368 ██████████████████▍ -0 22 0.135 ██████▋ +0 22 0.135 ██████▊ 0 23 0.05 ██▍ -0 24 0.018 ▊ +0 24 0.018 ▉ 1 25 1 ██████████████████████████████████████████████████ 0 26 0.368 ██████████████████▍ -0 27 0.135 ██████▋ +0 27 0.135 ██████▊ 0 28 0.05 ██▍ -0 29 0.018 ▊ +0 29 0.018 ▉ 1 30 1 ██████████████████████████████████████████████████ 0 31 0.368 ██████████████████▍ -0 32 0.135 ██████▋ +0 32 0.135 ██████▊ 0 33 0.05 ██▍ -0 34 0.018 ▊ +0 34 0.018 ▉ 1 35 1 ██████████████████████████████████████████████████ 0 36 0.368 ██████████████████▍ -0 37 0.135 ██████▋ +0 37 0.135 ██████▊ 0 38 0.05 ██▍ -0 39 0.018 ▊ +0 39 0.018 ▉ 1 40 1 ██████████████████████████████████████████████████ 0 41 0.368 ██████████████████▍ -0 42 0.135 ██████▋ +0 42 0.135 ██████▊ 0 43 0.05 ██▍ -0 44 0.018 ▊ +0 44 0.018 ▉ 1 45 1 ██████████████████████████████████████████████████ 0 46 0.368 ██████████████████▍ -0 47 0.135 ██████▋ +0 47 0.135 ██████▊ 0 48 0.05 ██▍ -0 49 0.018 ▊ +0 49 0.018 ▉ exponentialTimeDecayedCount 1 0 1 0 1 1.368 @@ -428,19 +428,19 @@ exponentialTimeDecayedCount 0 3 3.038 ███████████████▏ 0 4 3.487 █████████████████▍ 0 5 3.855 ███████████████████▎ -0 6 4.156 ████████████████████▋ +0 6 4.156 ████████████████████▊ 0 7 4.403 ██████████████████████ 0 8 4.605 ███████████████████████ -0 9 4.77 ███████████████████████▋ +0 9 4.77 ███████████████████████▊ 0 10 4.905 ████████████████████████▌ 0 11 5.016 █████████████████████████ 0 12 5.107 █████████████████████████▌ -0 13 5.181 █████████████████████████▊ +0 13 5.181 █████████████████████████▉ 0 14 5.242 ██████████████████████████▏ 0 15 5.292 ██████████████████████████▍ 0 16 5.333 ██████████████████████████▋ -0 17 5.366 ██████████████████████████▋ -0 18 5.393 ██████████████████████████▊ +0 17 5.366 ██████████████████████████▊ +0 18 5.393 ██████████████████████████▉ 0 19 5.416 ███████████████████████████ 0 20 5.434 ███████████████████████████▏ 0 21 5.449 ███████████████████████████▏ @@ -473,11 +473,11 @@ exponentialTimeDecayedCount 1 48 5.516 ███████████████████████████▌ 1 49 5.516 ███████████████████████████▌ 1 0 1 ██▌ -0 1 1.905 ████▋ -0 2 2.724 ██████▋ +0 1 1.905 ████▊ +0 2 2.724 ██████▊ 0 3 3.464 ████████▋ 0 4 4.135 ██████████▎ -1 5 4.741 ███████████▋ +1 5 4.741 ███████████▊ 0 6 5.29 █████████████▏ 0 7 5.787 ██████████████▍ 0 8 6.236 ███████████████▌ @@ -485,23 +485,23 @@ exponentialTimeDecayedCount 1 10 7.01 █████████████████▌ 0 11 7.343 ██████████████████▎ 0 12 7.644 ███████████████████ -0 13 7.917 ███████████████████▋ +0 13 7.917 ███████████████████▊ 0 14 8.164 ████████████████████▍ -1 15 8.387 ████████████████████▊ +1 15 8.387 ████████████████████▉ 0 16 8.589 █████████████████████▍ -0 17 8.771 █████████████████████▊ +0 17 8.771 █████████████████████▉ 0 18 8.937 ██████████████████████▎ 0 19 9.086 ██████████████████████▋ 1 20 9.222 ███████████████████████ 0 21 9.344 ███████████████████████▎ 0 22 9.455 ███████████████████████▋ -0 23 9.555 ███████████████████████▊ +0 23 9.555 ███████████████████████▉ 0 24 9.646 ████████████████████████ 1 25 9.728 ████████████████████████▎ 0 26 9.802 ████████████████████████▌ 0 27 9.869 ████████████████████████▋ -0 28 9.93 ████████████████████████▋ -0 29 9.985 ████████████████████████▊ +0 28 9.93 ████████████████████████▊ +0 29 9.985 ████████████████████████▉ 1 30 10.035 █████████████████████████ 0 31 10.08 █████████████████████████▏ 0 32 10.121 █████████████████████████▎ @@ -511,12 +511,12 @@ exponentialTimeDecayedCount 0 36 10.249 █████████████████████████▌ 0 37 10.273 █████████████████████████▋ 0 38 10.296 █████████████████████████▋ -0 39 10.316 █████████████████████████▋ -1 40 10.334 █████████████████████████▋ -0 41 10.351 █████████████████████████▊ -0 42 10.366 █████████████████████████▊ -0 43 10.379 █████████████████████████▊ -0 44 10.392 █████████████████████████▊ +0 39 10.316 █████████████████████████▊ +1 40 10.334 █████████████████████████▊ +0 41 10.351 █████████████████████████▉ +0 42 10.366 █████████████████████████▉ +0 43 10.379 █████████████████████████▉ +0 44 10.392 █████████████████████████▉ 1 45 10.403 ██████████████████████████ 0 46 10.413 ██████████████████████████ 0 47 10.422 ██████████████████████████ @@ -554,13 +554,13 @@ exponentialTimeDecayedAvg 8 8 7.419 9 9 8.418 1 0 1 ██████████ -0 1 0.475 ████▋ +0 1 0.475 ████▊ 0 2 0.301 ███ 0 3 0.214 ██▏ 0 4 0.162 █▌ 0 5 0.128 █▎ 0 6 0.104 █ -0 7 0.086 ▋ +0 7 0.086 ▊ 0 8 0.072 ▋ 0 9 0.061 ▌ 0 10 0.052 ▌ @@ -580,42 +580,42 @@ exponentialTimeDecayedAvg 0 24 0.009 1 25 0.111 █ 1 26 0.202 ██ -1 27 0.283 ██▋ +1 27 0.283 ██▊ 1 28 0.355 ███▌ 1 29 0.42 ████▏ -1 30 0.477 ████▋ +1 30 0.477 ████▊ 1 31 0.529 █████▎ -1 32 0.576 █████▋ +1 32 0.576 █████▊ 1 33 0.618 ██████▏ 1 34 0.655 ██████▌ -1 35 0.689 ██████▊ +1 35 0.689 ██████▉ 1 36 0.719 ███████▏ 1 37 0.747 ███████▍ 1 38 0.771 ███████▋ -1 39 0.793 ███████▊ +1 39 0.793 ███████▉ 1 40 0.813 ████████▏ 1 41 0.831 ████████▎ 1 42 0.848 ████████▍ 1 43 0.862 ████████▌ -1 44 0.876 ████████▋ -1 45 0.888 ████████▊ -1 46 0.898 ████████▊ +1 44 0.876 ████████▊ +1 45 0.888 ████████▉ +1 46 0.898 ████████▉ 1 47 0.908 █████████ 1 48 0.917 █████████▏ 1 49 0.925 █████████▏ 1 0 1 ██████████████████████████████████████████████████ -0 1 0.498 █████████████████████████████████████████████████▋ +0 1 0.498 █████████████████████████████████████████████████▊ 0 2 0.33 █████████████████████████████████ 0 3 0.246 ████████████████████████▋ 0 4 0.196 ███████████████████▌ 1 5 0.333 █████████████████████████████████▎ 0 6 0.284 ████████████████████████████▍ -0 7 0.248 ████████████████████████▋ -0 8 0.219 █████████████████████▊ +0 7 0.248 ████████████████████████▊ +0 8 0.219 █████████████████████▉ 0 9 0.196 ███████████████████▌ 1 10 0.273 ███████████████████████████▎ -0 11 0.249 ████████████████████████▊ -0 12 0.229 ██████████████████████▋ +0 11 0.249 ████████████████████████▉ +0 12 0.229 ██████████████████████▊ 0 13 0.211 █████████████████████ 0 14 0.196 ███████████████████▌ 1 15 0.25 █████████████████████████ @@ -623,7 +623,7 @@ exponentialTimeDecayedAvg 0 17 0.22 ██████████████████████ 0 18 0.207 ████████████████████▋ 0 19 0.196 ███████████████████▌ -1 20 0.238 ███████████████████████▋ +1 20 0.238 ███████████████████████▊ 0 21 0.226 ██████████████████████▌ 0 22 0.215 █████████████████████▌ 0 23 0.205 ████████████████████▌ @@ -634,21 +634,21 @@ exponentialTimeDecayedAvg 0 28 0.204 ████████████████████▍ 0 29 0.196 ███████████████████▌ 1 30 0.226 ██████████████████████▌ -0 31 0.218 █████████████████████▋ +0 31 0.218 █████████████████████▊ 0 32 0.21 █████████████████████ 0 33 0.203 ████████████████████▎ 0 34 0.196 ███████████████████▌ 1 35 0.222 ██████████████████████▏ 0 36 0.215 █████████████████████▌ -0 37 0.209 ████████████████████▋ +0 37 0.209 ████████████████████▊ 0 38 0.202 ████████████████████▏ 0 39 0.196 ███████████████████▌ -1 40 0.22 █████████████████████▊ +1 40 0.22 █████████████████████▉ 0 41 0.213 █████████████████████▎ 0 42 0.207 ████████████████████▋ 0 43 0.202 ████████████████████▏ 0 44 0.196 ███████████████████▌ -1 45 0.218 █████████████████████▋ +1 45 0.218 █████████████████████▊ 0 46 0.212 █████████████████████▏ 0 47 0.206 ████████████████████▋ 0 48 0.201 ████████████████████ From 76f0cb244375374b465f1d86b0317bae3f29b6c0 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Dec 2022 19:08:00 +0100 Subject: [PATCH 058/339] Migrate pull_request.yml to composite checkout --- .github/workflows/pull_request.yml | 912 ++++++++++++----------------- 1 file changed, 383 insertions(+), 529 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 1ff95fc4ffd..ae6c990890d 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -26,11 +26,10 @@ jobs: # Run the first check always, even if the CI is cancelled if: ${{ always() }} steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Labels check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -39,11 +38,10 @@ jobs: needs: CheckLabels 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Python unit tests run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -52,11 +50,10 @@ jobs: needs: CheckLabels runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -70,11 +67,10 @@ jobs: needs: CheckLabels 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -88,11 +84,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64, PythonUnitTests] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v2 with: @@ -133,11 +128,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Style Check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -159,14 +153,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed images uses: actions/download-artifact@v2 with: @@ -193,11 +183,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download json reports uses: actions/download-artifact@v2 with: @@ -225,11 +214,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download json reports uses: actions/download-artifact@v2 with: @@ -267,17 +255,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # for performance artifact + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -312,15 +297,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -355,17 +338,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true fetch-depth: 0 # for performance artifact - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -400,15 +380,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -443,15 +421,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -486,15 +462,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -529,15 +503,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -572,15 +544,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -618,15 +588,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -661,15 +629,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -704,15 +670,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -747,15 +711,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -790,15 +752,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -833,15 +793,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -876,15 +834,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -919,15 +875,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -962,15 +916,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -996,12 +948,10 @@ jobs: - 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - name: Check docker clickhouse/clickhouse-server building run: | @@ -1043,11 +993,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -1090,11 +1039,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -1130,11 +1078,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1167,11 +1114,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1204,11 +1150,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1241,11 +1186,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1278,11 +1222,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1313,11 +1256,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1350,11 +1292,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1387,11 +1328,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1424,11 +1364,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1461,11 +1400,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1498,11 +1436,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1535,11 +1472,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1572,11 +1508,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1609,11 +1544,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1646,11 +1580,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1683,11 +1616,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1720,11 +1652,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1757,11 +1688,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1794,11 +1724,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1829,11 +1758,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1866,11 +1794,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1903,11 +1830,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1940,11 +1866,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1977,11 +1902,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2014,11 +1938,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2051,11 +1974,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2088,11 +2010,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2125,11 +2046,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2162,11 +2082,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2199,11 +2118,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2236,11 +2154,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2273,11 +2190,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2310,11 +2226,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2347,11 +2262,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2384,11 +2298,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2421,11 +2334,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2458,11 +2370,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2495,11 +2406,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2532,11 +2442,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2569,11 +2478,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2606,11 +2514,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2643,11 +2550,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2678,11 +2584,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2713,11 +2618,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Bugfix test run: | sudo rm -fr "$TEMP_PATH" @@ -2762,11 +2666,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2797,11 +2700,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2832,11 +2734,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2867,11 +2768,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2902,11 +2802,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2937,11 +2836,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -2972,11 +2870,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -3009,11 +2906,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -3047,11 +2943,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -3081,11 +2976,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -3115,11 +3009,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -3149,11 +3042,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -3186,11 +3078,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -3220,11 +3111,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -3254,11 +3144,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -3288,11 +3177,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -3322,11 +3210,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -3361,11 +3248,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3397,11 +3283,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3433,11 +3318,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3469,11 +3353,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3505,11 +3388,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3541,11 +3423,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3577,11 +3458,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3613,11 +3493,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3649,11 +3528,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3685,11 +3563,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3721,11 +3598,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3757,11 +3633,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3793,11 +3668,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3829,11 +3703,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3865,11 +3738,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3901,11 +3773,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3935,11 +3806,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -3972,11 +3842,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -4006,11 +3875,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -4040,11 +3908,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -4074,11 +3941,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -4108,11 +3974,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -4147,11 +4012,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4183,11 +4047,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4219,11 +4082,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4255,11 +4117,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4291,11 +4152,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4327,11 +4187,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4363,11 +4222,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4399,11 +4257,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -4436,11 +4293,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: SQLancer run: | sudo rm -fr "$TEMP_PATH" @@ -4470,11 +4326,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: SQLancer run: | sudo rm -fr "$TEMP_PATH" @@ -4600,11 +4455,10 @@ jobs: - SQLancerTestDebug 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" From e7341efb659da4902d91d21cbea6220a67990d38 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Dec 2022 21:02:35 +0100 Subject: [PATCH 059/339] Upgrade download-artifact and upload-artifact actions to v3 --- .github/workflows/backport_branches.yml | 52 ++--- .github/workflows/docs_check.yml | 14 +- .github/workflows/docs_release.yml | 12 +- .github/workflows/master.yml | 208 +++++++++--------- .github/workflows/nightly.yml | 12 +- .github/workflows/pull_request.yml | 276 ++++++++++++------------ .github/workflows/release_branches.yml | 122 +++++------ 7 files changed, 348 insertions(+), 348 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index c90df6e57b7..e719b5d52b7 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -34,7 +34,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -51,7 +51,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -65,12 +65,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -79,7 +79,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -100,7 +100,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck @@ -132,7 +132,7 @@ jobs: BUILD_NAME=package_release EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -153,7 +153,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -177,7 +177,7 @@ jobs: BUILD_NAME=package_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -198,7 +198,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -222,7 +222,7 @@ jobs: BUILD_NAME=package_asan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -241,7 +241,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -265,7 +265,7 @@ jobs: BUILD_NAME=package_tsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -284,7 +284,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -308,7 +308,7 @@ jobs: BUILD_NAME=package_debug EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -327,7 +327,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -351,7 +351,7 @@ jobs: BUILD_NAME=binary_darwin EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -372,7 +372,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -396,7 +396,7 @@ jobs: BUILD_NAME=binary_darwin_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -417,7 +417,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -477,7 +477,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -516,7 +516,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -556,7 +556,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -594,7 +594,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -635,7 +635,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -672,7 +672,7 @@ jobs: REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 5c912ebd359..61d1c2808d4 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -44,7 +44,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -62,7 +62,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -76,12 +76,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -90,7 +90,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -110,7 +110,7 @@ jobs: - 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 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.TEMP_PATH }} @@ -140,7 +140,7 @@ jobs: REPO_COPY=${{runner.temp}}/docs_check/ClickHouse EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.TEMP_PATH }} diff --git a/.github/workflows/docs_release.yml b/.github/workflows/docs_release.yml index 1b43138852b..afad5cccea0 100644 --- a/.github/workflows/docs_release.yml +++ b/.github/workflows/docs_release.yml @@ -33,7 +33,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -50,7 +50,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -64,12 +64,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -78,7 +78,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -103,7 +103,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.TEMP_PATH }} diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f3d672136ef..a88efa377c3 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -34,7 +34,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -51,7 +51,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -65,12 +65,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -79,7 +79,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -96,7 +96,7 @@ jobs: - 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 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.TEMP_PATH }} @@ -132,7 +132,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck @@ -164,7 +164,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Shared build check @@ -196,7 +196,7 @@ jobs: BUILD_NAME=package_release EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -217,7 +217,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -241,7 +241,7 @@ jobs: BUILD_NAME=package_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/images_path @@ -258,7 +258,7 @@ jobs: cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json @@ -282,7 +282,7 @@ jobs: BUILD_NAME=binary_release EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -303,7 +303,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -327,7 +327,7 @@ jobs: BUILD_NAME=package_asan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -346,7 +346,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -370,7 +370,7 @@ jobs: BUILD_NAME=package_ubsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -389,7 +389,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -413,7 +413,7 @@ jobs: BUILD_NAME=package_tsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -432,7 +432,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -456,7 +456,7 @@ jobs: BUILD_NAME=package_msan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -475,7 +475,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -499,7 +499,7 @@ jobs: BUILD_NAME=package_debug EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -518,7 +518,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -545,7 +545,7 @@ jobs: BUILD_NAME=binary_shared EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -564,7 +564,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -588,7 +588,7 @@ jobs: BUILD_NAME=binary_tidy EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -607,7 +607,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -631,7 +631,7 @@ jobs: BUILD_NAME=binary_darwin EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -652,7 +652,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -676,7 +676,7 @@ jobs: BUILD_NAME=binary_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -697,7 +697,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -721,7 +721,7 @@ jobs: BUILD_NAME=binary_freebsd EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -742,7 +742,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -766,7 +766,7 @@ jobs: BUILD_NAME=binary_darwin_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -787,7 +787,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -811,7 +811,7 @@ jobs: BUILD_NAME=binary_ppc64le EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -832,7 +832,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -856,7 +856,7 @@ jobs: BUILD_NAME=binary_amd64sse2 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -877,7 +877,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -901,7 +901,7 @@ jobs: BUILD_NAME=binary_aarch64_v80compat EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -922,7 +922,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -986,7 +986,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1033,7 +1033,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1090,7 +1090,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1125,7 +1125,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1162,7 +1162,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1199,7 +1199,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1234,7 +1234,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1269,7 +1269,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1306,7 +1306,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1343,7 +1343,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1380,7 +1380,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1417,7 +1417,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1454,7 +1454,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1489,7 +1489,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1526,7 +1526,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1563,7 +1563,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1600,7 +1600,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1637,7 +1637,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1674,7 +1674,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1711,7 +1711,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1749,7 +1749,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1784,7 +1784,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1819,7 +1819,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1854,7 +1854,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1889,7 +1889,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1924,7 +1924,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1959,7 +1959,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1996,7 +1996,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2034,7 +2034,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2068,7 +2068,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2102,7 +2102,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2136,7 +2136,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2175,7 +2175,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2211,7 +2211,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2247,7 +2247,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2283,7 +2283,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2319,7 +2319,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2355,7 +2355,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2391,7 +2391,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2427,7 +2427,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2463,7 +2463,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2500,7 +2500,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2534,7 +2534,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2568,7 +2568,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2602,7 +2602,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2636,7 +2636,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2673,7 +2673,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2707,7 +2707,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2741,7 +2741,7 @@ jobs: # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse # EOF # - name: Download json reports - # uses: actions/download-artifact@v2 + # uses: actions/download-artifact@v3 # with: # path: ${{ env.REPORTS_PATH }} # - name: Clear repository @@ -2775,7 +2775,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2809,7 +2809,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2843,7 +2843,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2882,7 +2882,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2918,7 +2918,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2954,7 +2954,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -2990,7 +2990,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -3026,7 +3026,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -3062,7 +3062,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -3098,7 +3098,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -3134,7 +3134,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -3171,7 +3171,7 @@ jobs: REPO_COPY=${{runner.temp}}/sqlancer_release/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -3205,7 +3205,7 @@ jobs: REPO_COPY=${{runner.temp}}/sqlancer_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 9ebbe4e090d..916249327c2 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -26,7 +26,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 --all - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -43,7 +43,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 --all - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -57,12 +57,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -71,7 +71,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -90,7 +90,7 @@ jobs: EOF echo "COVERITY_TOKEN=${{ secrets.COVERITY_TOKEN }}" >> "$GITHUB_ENV" - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index ae6c990890d..3a9cc5aaf38 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -59,7 +59,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -76,7 +76,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -89,12 +89,12 @@ jobs: with: clear-repository: true - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -103,7 +103,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -124,7 +124,7 @@ jobs: - 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 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.TEMP_PATH }} @@ -158,7 +158,7 @@ jobs: with: clear-repository: true - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.TEMP_PATH }} @@ -188,7 +188,7 @@ jobs: with: clear-repository: true - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck @@ -219,7 +219,7 @@ jobs: with: clear-repository: true - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Shared build check @@ -251,7 +251,7 @@ jobs: BUILD_NAME=package_release EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -269,7 +269,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -293,7 +293,7 @@ jobs: BUILD_NAME=binary_release EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -310,7 +310,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -334,7 +334,7 @@ jobs: BUILD_NAME=package_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/images_path @@ -352,7 +352,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -376,7 +376,7 @@ jobs: BUILD_NAME=package_asan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -393,7 +393,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -417,7 +417,7 @@ jobs: BUILD_NAME=package_ubsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -434,7 +434,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -458,7 +458,7 @@ jobs: BUILD_NAME=package_tsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -475,7 +475,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -499,7 +499,7 @@ jobs: BUILD_NAME=package_msan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -516,7 +516,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -540,7 +540,7 @@ jobs: BUILD_NAME=package_debug EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -557,7 +557,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -584,7 +584,7 @@ jobs: BUILD_NAME=binary_shared EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -601,7 +601,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -625,7 +625,7 @@ jobs: BUILD_NAME=binary_tidy EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -642,7 +642,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -666,7 +666,7 @@ jobs: BUILD_NAME=binary_darwin EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -683,7 +683,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -707,7 +707,7 @@ jobs: BUILD_NAME=binary_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -724,7 +724,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -748,7 +748,7 @@ jobs: BUILD_NAME=binary_freebsd EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -765,7 +765,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -789,7 +789,7 @@ jobs: BUILD_NAME=binary_darwin_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -806,7 +806,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -830,7 +830,7 @@ jobs: BUILD_NAME=binary_ppc64le EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -847,7 +847,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -871,7 +871,7 @@ jobs: BUILD_NAME=binary_amd64sse2 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -888,7 +888,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -912,7 +912,7 @@ jobs: BUILD_NAME=binary_aarch64_v80compat EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -929,7 +929,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -990,7 +990,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1036,7 +1036,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1075,7 +1075,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1111,7 +1111,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1147,7 +1147,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1183,7 +1183,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1219,7 +1219,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1253,7 +1253,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1289,7 +1289,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1325,7 +1325,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1361,7 +1361,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1397,7 +1397,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1433,7 +1433,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1469,7 +1469,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1505,7 +1505,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1541,7 +1541,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1577,7 +1577,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1613,7 +1613,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1649,7 +1649,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1685,7 +1685,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1721,7 +1721,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1755,7 +1755,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1791,7 +1791,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1827,7 +1827,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1863,7 +1863,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1899,7 +1899,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1935,7 +1935,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -1971,7 +1971,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2007,7 +2007,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2043,7 +2043,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2079,7 +2079,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2115,7 +2115,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2151,7 +2151,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2187,7 +2187,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2223,7 +2223,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2259,7 +2259,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2295,7 +2295,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2331,7 +2331,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2367,7 +2367,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2403,7 +2403,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2439,7 +2439,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2475,7 +2475,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2511,7 +2511,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2547,7 +2547,7 @@ jobs: RUN_BY_HASH_TOTAL=5 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2581,7 +2581,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2615,7 +2615,7 @@ jobs: REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2663,7 +2663,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2697,7 +2697,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2731,7 +2731,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2765,7 +2765,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2799,7 +2799,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2833,7 +2833,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2867,7 +2867,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2903,7 +2903,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2940,7 +2940,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -2973,7 +2973,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3006,7 +3006,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3039,7 +3039,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3075,7 +3075,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3108,7 +3108,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3141,7 +3141,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3174,7 +3174,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3207,7 +3207,7 @@ jobs: REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3245,7 +3245,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3280,7 +3280,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3315,7 +3315,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3350,7 +3350,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3385,7 +3385,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3420,7 +3420,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3455,7 +3455,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3490,7 +3490,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3525,7 +3525,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3560,7 +3560,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3595,7 +3595,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3630,7 +3630,7 @@ jobs: RUN_BY_HASH_TOTAL=6 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3665,7 +3665,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3700,7 +3700,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3735,7 +3735,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3770,7 +3770,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3803,7 +3803,7 @@ jobs: REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3839,7 +3839,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3872,7 +3872,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3905,7 +3905,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3938,7 +3938,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -3971,7 +3971,7 @@ jobs: REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4009,7 +4009,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4044,7 +4044,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4079,7 +4079,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4114,7 +4114,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4149,7 +4149,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4184,7 +4184,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4219,7 +4219,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4254,7 +4254,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4290,7 +4290,7 @@ jobs: REPO_COPY=${{runner.temp}}/sqlancer_release/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code @@ -4323,7 +4323,7 @@ jobs: REPO_COPY=${{runner.temp}}/sqlancer_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bf35ca76fc6..349ff1186ae 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -25,7 +25,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json @@ -42,7 +42,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_images_check.py --suffix amd64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json @@ -56,12 +56,12 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download changed aarch64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_aarch64 path: ${{ runner.temp }} - name: Download changed amd64 images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images_amd64 path: ${{ runner.temp }} @@ -70,7 +70,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/changed_images.json @@ -91,7 +91,7 @@ jobs: - name: Check out repository code uses: actions/checkout@v2 - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: CompatibilityCheck @@ -123,7 +123,7 @@ jobs: BUILD_NAME=package_release EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -144,7 +144,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -168,7 +168,7 @@ jobs: BUILD_NAME=package_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ runner.temp }}/images_path @@ -185,7 +185,7 @@ jobs: cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ runner.temp }}/build_check/${{ env.BUILD_URLS }}.json @@ -209,7 +209,7 @@ jobs: BUILD_NAME=package_asan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -228,7 +228,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -252,7 +252,7 @@ jobs: BUILD_NAME=package_ubsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -271,7 +271,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -295,7 +295,7 @@ jobs: BUILD_NAME=package_tsan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -314,7 +314,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -338,7 +338,7 @@ jobs: BUILD_NAME=package_msan EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -357,7 +357,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -381,7 +381,7 @@ jobs: BUILD_NAME=package_debug EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -400,7 +400,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -424,7 +424,7 @@ jobs: BUILD_NAME=binary_darwin EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -445,7 +445,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -469,7 +469,7 @@ jobs: BUILD_NAME=binary_darwin_aarch64 EOF - name: Download changed images - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: name: changed_images path: ${{ env.IMAGES_PATH }} @@ -490,7 +490,7 @@ jobs: 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 + uses: actions/upload-artifact@v3 with: name: ${{ env.BUILD_URLS }} path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json @@ -553,7 +553,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -592,7 +592,7 @@ jobs: NEEDS_DATA_PATH=${{runner.temp}}/needs.json EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -649,7 +649,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -684,7 +684,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -721,7 +721,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -758,7 +758,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -795,7 +795,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -832,7 +832,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -869,7 +869,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -904,7 +904,7 @@ jobs: KILL_TIMEOUT=10800 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -941,7 +941,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -978,7 +978,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1015,7 +1015,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1052,7 +1052,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1089,7 +1089,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1126,7 +1126,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1164,7 +1164,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1199,7 +1199,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1234,7 +1234,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1269,7 +1269,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1304,7 +1304,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1339,7 +1339,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1374,7 +1374,7 @@ jobs: KILL_TIMEOUT=3600 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1411,7 +1411,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1449,7 +1449,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1483,7 +1483,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1517,7 +1517,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1551,7 +1551,7 @@ jobs: REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1590,7 +1590,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1626,7 +1626,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1662,7 +1662,7 @@ jobs: RUN_BY_HASH_TOTAL=3 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1698,7 +1698,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1734,7 +1734,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1770,7 +1770,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1806,7 +1806,7 @@ jobs: RUN_BY_HASH_TOTAL=4 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1842,7 +1842,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository @@ -1878,7 +1878,7 @@ jobs: RUN_BY_HASH_TOTAL=2 EOF - name: Download json reports - uses: actions/download-artifact@v2 + uses: actions/download-artifact@v3 with: path: ${{ env.REPORTS_PATH }} - name: Clear repository From 6646950f3c641d614e3d8c6d3e1340ee5ca8789a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 19 Dec 2022 21:28:14 +0100 Subject: [PATCH 060/339] Get rid of unused set-output --- tests/ci/docker_images_check.py | 1 - tests/ci/docker_manifests_merge.py | 1 - tests/ci/docker_server.py | 1 - 3 files changed, 3 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0618969f94c..034e0110e2f 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -476,7 +476,6 @@ def main(): url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME) print(f"::notice ::Report url: {url}") - print(f'::set-output name=url_output::"{url}"') if not args.reports: return diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index 2ba5a99de0a..14585159d47 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -208,7 +208,6 @@ def main(): url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME) print(f"::notice ::Report url: {url}") - print(f'::set-output name=url_output::"{url}"') if not args.reports: return diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index e0053f09664..fd28e5a1890 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -340,7 +340,6 @@ def main(): url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME) print(f"::notice ::Report url: {url}") - print(f'::set-output name=url_output::"{url}"') if not args.reports: return From ef6868951d459e3ed2733d33a6b5adb372379bd9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 00:58:23 +0100 Subject: [PATCH 061/339] Move backport branches to ClickHouse/checkout --- .github/workflows/backport_branches.yml | 156 ++++++++++-------------- 1 file changed, 62 insertions(+), 94 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index e719b5d52b7..4157cf446a0 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -12,11 +12,10 @@ jobs: PythonUnitTests: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Python unit tests run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -24,11 +23,10 @@ jobs: DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -41,11 +39,10 @@ jobs: DockerHubPushAmd64: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -59,11 +56,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: @@ -94,11 +90,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download json reports uses: actions/download-artifact@v3 with: @@ -136,17 +131,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -181,17 +173,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -226,15 +215,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -269,15 +256,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -312,15 +297,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -355,17 +338,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -400,17 +380,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -436,12 +413,10 @@ jobs: - 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - name: Check docker clickhouse/clickhouse-server building run: | @@ -480,11 +455,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -519,11 +493,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -559,11 +532,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -597,11 +569,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -638,11 +609,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -675,11 +645,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -706,11 +675,10 @@ jobs: - CompatibilityCheck 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" From eddff099a734e73d1154ee2914973cee9a1151bb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 01:00:22 +0100 Subject: [PATCH 062/339] Migrate CherryPick to ClickHouse/checkout --- .github/workflows/cherry_pick.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/cherry_pick.yml b/.github/workflows/cherry_pick.yml index 3e6f9e76c56..c0bae7f1ca3 100644 --- a/.github/workflows/cherry_pick.yml +++ b/.github/workflows/cherry_pick.yml @@ -28,8 +28,9 @@ jobs: REPO_TEAM=core EOF - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: + clear-repository: true token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} fetch-depth: 0 - name: Cherry pick From 65673fbf724083c058ee3957386e071d56a52ca9 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 01:03:19 +0100 Subject: [PATCH 063/339] Migrate docs_check.yml to ClickHouse/checkout --- .github/workflows/docs_check.yml | 49 ++++++++++++++------------------ 1 file changed, 21 insertions(+), 28 deletions(-) diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 61d1c2808d4..ef93b4796c2 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -21,11 +21,10 @@ jobs: CheckLabels: runs-on: [self-hosted, style-checker] steps: - - name: Clear repository - run: | - sudo rm -rf "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Labels check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -34,11 +33,10 @@ jobs: needs: CheckLabels runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -52,11 +50,10 @@ jobs: needs: CheckLabels 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -70,11 +67,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: @@ -114,11 +110,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Style Check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -144,11 +139,10 @@ jobs: with: name: changed_images path: ${{ env.TEMP_PATH }} - - name: Clear repository - run: | - sudo rm -rf "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Docs Check run: | cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -167,11 +161,10 @@ jobs: - DocsCheck 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" From 2f1166811e24ad695ff4f21b7948ba0dc44a80b6 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 01:04:21 +0100 Subject: [PATCH 064/339] Migrate docs_release.yml to ClickHouse/checkout --- .github/workflows/docs_release.yml | 28 ++++++++++++---------------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/.github/workflows/docs_release.yml b/.github/workflows/docs_release.yml index afad5cccea0..7947483c9bb 100644 --- a/.github/workflows/docs_release.yml +++ b/.github/workflows/docs_release.yml @@ -23,11 +23,10 @@ jobs: DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -40,11 +39,10 @@ jobs: DockerHubPushAmd64: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -58,11 +56,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: @@ -97,11 +94,10 @@ jobs: ${{secrets.ROBOT_CLICKHOUSE_SSH_KEY}} RCSK EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed images uses: actions/download-artifact@v3 with: From 50563c253fe36d97c3c378a6b9ee3e79e105e28a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 01:06:11 +0100 Subject: [PATCH 065/339] Migrate jepsen.yml to ClickHouse/checkout --- .github/workflows/jepsen.yml | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 5afc066065e..f13d32a909e 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -19,12 +19,10 @@ jobs: TEMP_PATH=${{runner.temp}}/keeper_jepsen REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 - name: Jepsen Test run: | @@ -50,12 +48,10 @@ jobs: # TEMP_PATH=${{runner.temp}}/server_jepsen # REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse # EOF - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" # - name: Check out repository code - # uses: actions/checkout@v2 + # uses: ClickHouse/checkout@v0 # with: + # clear-repository: true # fetch-depth: 0 # - name: Jepsen Test # run: | From ac4f968a985332cf6c7524220cc2374453bce0f2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 01:07:58 +0100 Subject: [PATCH 066/339] Migrate woboq.yml to ClickHouse/checkout --- .github/workflows/woboq.yml | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/.github/workflows/woboq.yml b/.github/workflows/woboq.yml index b928a4a8d3d..d66d1cc6c8e 100644 --- a/.github/workflows/woboq.yml +++ b/.github/workflows/woboq.yml @@ -21,12 +21,10 @@ jobs: REPO_COPY=${{runner.temp}}/codebrowser/ClickHouse IMAGES_PATH=${{runner.temp}}/images_path EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: + clear-repository: true submodules: 'true' - name: Codebrowser run: | From 9bcd98f5efff63216333752f6cad79215a99d5e1 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 01:10:04 +0100 Subject: [PATCH 067/339] Migrate tags_stable.yml and release.yml to ClickHouse/checkout --- .github/workflows/release.yml | 8 +++----- .github/workflows/tags_stable.yml | 2 +- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 0b0f125d641..1b5cdb93a8f 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -20,7 +20,7 @@ jobs: REPO_COPY=${{runner.temp}}/release_packages/ClickHouse EOF - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: # Always use the most recent script version ref: master @@ -50,12 +50,10 @@ jobs: DockerServerImages: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # otherwise we will have no version info - name: Check docker clickhouse/clickhouse-server building run: | diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml index f8cfa1137cc..f577abb104c 100644 --- a/.github/workflows/tags_stable.yml +++ b/.github/workflows/tags_stable.yml @@ -34,7 +34,7 @@ jobs: run: | echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: ref: master fetch-depth: 0 From ca02ce176fb6ba81fb7c28937384e7bff5a48348 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 11:14:42 +0100 Subject: [PATCH 068/339] Migrate nightly.yml to ClickHouse/checkout --- .github/workflows/nightly.yml | 36 +++++++++++++++-------------------- 1 file changed, 15 insertions(+), 21 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 916249327c2..a4cc7e1046c 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -16,11 +16,10 @@ jobs: DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -33,11 +32,10 @@ jobs: DockerHubPushAmd64: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -51,11 +49,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: @@ -94,18 +91,13 @@ jobs: 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 - id: coverity-checkout - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: - fetch-depth: 0 # otherwise we will have no info about contributors + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -134,8 +126,10 @@ jobs: CC: clang-15 CXX: clang++-15 steps: - - uses: actions/checkout@v2 + - name: Check out repository code + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # Shallow clones should be disabled for a better relevancy of analysis submodules: true - name: Set up JDK 11 From 722ea8d42497badc11ec8cbf7711b74e2cc687b7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 11:24:27 +0100 Subject: [PATCH 069/339] Migrate release_branches.yml to ClickHouse/checkout --- .github/workflows/release_branches.yml | 392 +++++++++++-------------- 1 file changed, 164 insertions(+), 228 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 349ff1186ae..3f67cb2a0e8 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -15,11 +15,10 @@ jobs: DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -32,11 +31,10 @@ jobs: DockerHubPushAmd64: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -50,11 +48,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: @@ -85,11 +82,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download json reports uses: actions/download-artifact@v3 with: @@ -127,17 +123,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -173,13 +166,13 @@ jobs: name: changed_images path: ${{ runner.temp }}/images_path - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: - fetch-depth: 0 # otherwise we will have no info about contributors + clear-repository: true + submodules: true + 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -213,15 +206,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -256,15 +247,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -299,15 +288,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -342,15 +329,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -385,15 +370,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -428,17 +411,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -473,17 +453,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -509,12 +486,10 @@ jobs: - 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - name: Check docker clickhouse/clickhouse-server building run: | @@ -556,11 +531,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -595,11 +569,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -623,11 +596,10 @@ jobs: - 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Mark Commit Release Ready run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -652,11 +624,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -687,11 +658,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -724,11 +694,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -761,11 +730,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -798,11 +766,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -835,11 +802,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -872,11 +838,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -907,11 +872,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -944,11 +908,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -981,11 +944,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1018,11 +980,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1055,11 +1016,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1092,11 +1052,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1129,11 +1088,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1167,11 +1125,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1202,11 +1159,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1237,11 +1193,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1272,11 +1227,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1307,11 +1261,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1342,11 +1295,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1377,11 +1329,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1414,11 +1365,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -1452,11 +1402,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -1486,11 +1435,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -1520,11 +1468,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -1554,11 +1501,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -1593,11 +1539,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1629,11 +1574,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1665,11 +1609,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1701,11 +1644,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1737,11 +1679,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1773,11 +1714,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1809,11 +1749,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1845,11 +1784,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1881,11 +1819,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -1944,11 +1881,10 @@ jobs: - CompatibilityCheck 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" From 3f32477abda509fef096ea82c8902d9dbdb49f65 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 11:34:56 +0100 Subject: [PATCH 070/339] Migrate master.yml to ClickHouse/checkout --- .github/workflows/master.yml | 653 +++++++++++++++-------------------- 1 file changed, 270 insertions(+), 383 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index a88efa377c3..262a1873975 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -12,11 +12,10 @@ jobs: PythonUnitTests: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Python unit tests run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -24,11 +23,10 @@ jobs: DockerHubPushAarch64: runs-on: [self-hosted, style-checker-aarch64] steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -41,11 +39,10 @@ jobs: DockerHubPushAmd64: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Images check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -59,11 +56,10 @@ jobs: needs: [DockerHubPushAmd64, DockerHubPushAarch64, PythonUnitTests] 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download changed aarch64 images uses: actions/download-artifact@v3 with: @@ -100,11 +96,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Style Check run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -126,11 +121,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download json reports uses: actions/download-artifact@v3 with: @@ -158,11 +152,10 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Download json reports uses: actions/download-artifact@v3 with: @@ -200,17 +193,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -246,13 +236,13 @@ jobs: name: changed_images path: ${{ runner.temp }}/images_path - name: Check out repository code - uses: actions/checkout@v2 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -286,17 +276,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -331,15 +318,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -374,15 +359,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -417,15 +400,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -460,15 +441,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -503,15 +482,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -549,15 +526,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -592,15 +567,13 @@ jobs: 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true + submodules: true - name: Build run: | - git -C "$GITHUB_WORKSPACE" submodule sync - git -C "$GITHUB_WORKSPACE" submodule update --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -635,17 +608,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -680,17 +650,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -725,17 +692,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -770,17 +734,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -815,17 +776,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -860,17 +818,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -905,17 +860,14 @@ jobs: 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true + submodules: true 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 --single-branch --depth=1 --init --jobs=10 sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" @@ -941,12 +893,10 @@ jobs: - 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 + uses: ClickHouse/checkout@v0 with: + clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - name: Check docker clickhouse/clickhouse-server building run: | @@ -989,11 +939,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -1036,11 +985,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Report Builder run: | sudo rm -fr "$TEMP_PATH" @@ -1064,11 +1012,10 @@ jobs: - 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Mark Commit Release Ready run: | cd "$GITHUB_WORKSPACE/tests/ci" @@ -1093,11 +1040,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1128,11 +1074,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1165,11 +1110,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1202,11 +1146,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1237,11 +1180,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1272,11 +1214,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1309,11 +1250,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1346,11 +1286,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1383,11 +1322,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1420,11 +1358,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1457,11 +1394,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1492,11 +1428,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1529,11 +1464,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1566,11 +1500,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1603,11 +1536,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1640,11 +1572,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1677,11 +1608,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1714,11 +1644,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1752,11 +1681,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1787,11 +1715,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1822,11 +1749,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1857,11 +1783,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1892,11 +1817,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1927,11 +1851,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1962,11 +1885,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Functional test run: | sudo rm -fr "$TEMP_PATH" @@ -1999,11 +1921,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -2037,11 +1958,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -2071,11 +1991,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -2105,11 +2024,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -2139,11 +2057,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Stress test run: | sudo rm -fr "$TEMP_PATH" @@ -2178,11 +2095,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2214,11 +2130,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2250,11 +2165,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2286,11 +2200,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2322,11 +2235,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2358,11 +2270,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2394,11 +2305,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2430,11 +2340,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2466,11 +2375,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Integration test run: | sudo rm -fr "$TEMP_PATH" @@ -2503,11 +2411,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -2537,11 +2444,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -2571,11 +2477,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -2605,11 +2510,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -2639,11 +2543,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Fuzzer run: | sudo rm -fr "$TEMP_PATH" @@ -2676,11 +2579,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -2710,11 +2612,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -2778,11 +2679,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -2812,11 +2712,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -2846,11 +2745,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Unit test run: | sudo rm -fr "$TEMP_PATH" @@ -2885,11 +2783,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -2921,11 +2818,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -2957,11 +2853,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -2993,11 +2888,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -3029,11 +2923,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -3065,11 +2958,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -3101,11 +2993,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -3137,11 +3028,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Performance Comparison run: | sudo rm -fr "$TEMP_PATH" @@ -3174,11 +3064,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: SQLancer run: | sudo rm -fr "$TEMP_PATH" @@ -3208,11 +3097,10 @@ jobs: uses: actions/download-artifact@v3 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: SQLancer run: | sudo rm -fr "$TEMP_PATH" @@ -3291,11 +3179,10 @@ jobs: - SQLancerTestDebug 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 + uses: ClickHouse/checkout@v0 + with: + clear-repository: true - name: Finish label run: | cd "$GITHUB_WORKSPACE/tests/ci" From 06bfa8b8853c0cfcf0cf9bc7d35d79f78b4621f4 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 20 Dec 2022 11:35:17 +0100 Subject: [PATCH 071/339] Remove stale code --- .github/workflows/master.yml | 35 ----------------------------------- 1 file changed, 35 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 262a1873975..6e06d98da40 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -964,7 +964,6 @@ jobs: - BuilderBinDarwin - BuilderBinDarwinAarch64 - BuilderBinFreeBSD - # - BuilderBinGCC - BuilderBinPPC64 - BuilderBinAmd64SSE2 - BuilderBinAarch64V80Compat @@ -2629,40 +2628,6 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: 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) - # REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - # EOF - # - name: Download json reports - # uses: actions/download-artifact@v3 - # 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: | - # docker ps --quiet | xargs --no-run-if-empty docker kill ||: - # docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - # sudo rm -fr "$TEMP_PATH" UnitTestsTsan: needs: [BuilderDebTsan] runs-on: [self-hosted, fuzzer-unit-tester] From faf7ac2aefe2d574960c0424b303faadeb2a43de Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Dec 2022 15:19:41 +0300 Subject: [PATCH 072/339] Update 02509_h3_arguments.sql --- tests/queries/0_stateless/02509_h3_arguments.sql | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/queries/0_stateless/02509_h3_arguments.sql b/tests/queries/0_stateless/02509_h3_arguments.sql index 268012ecf4f..f3a2a5d94bf 100644 --- a/tests/queries/0_stateless/02509_h3_arguments.sql +++ b/tests/queries/0_stateless/02509_h3_arguments.sql @@ -6,11 +6,6 @@ DROP TABLE IF EXISTS data_table; CREATE TABLE data_table (id UInt64, longitude Float64, latitude Float64) ENGINE=MergeTree ORDER BY id; INSERT INTO data_table SELECT number, number, number FROM numbers(10); SELECT geoToH3(longitude, latitude, toUInt8(8)) AS h3Index FROM data_table ORDER BY 1; - -DROP TABLE data_table; - -CREATE TABLE data_table (id UInt64, longitude Float64, latitude Float64) ENGINE=MergeTree ORDER BY id; -INSERT INTO data_table SELECT number, number, number FROM numbers(10); SELECT geoToH3(longitude, latitude, toUInt8(longitude - longitude + 8)) AS h3Index FROM data_table ORDER BY 1; DROP TABLE data_table; From 10ea709931de1685ccc7727038de185e30bbaf97 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 20 Dec 2022 14:00:17 +0100 Subject: [PATCH 073/339] Fix broken test --- ...513_insert_without_materialized_columns.sh | 29 +++++++++++++++++++ ...13_insert_without_materialized_columns.sql | 20 ------------- 2 files changed, 29 insertions(+), 20 deletions(-) create mode 100755 tests/queries/0_stateless/02513_insert_without_materialized_columns.sh delete mode 100644 tests/queries/0_stateless/02513_insert_without_materialized_columns.sql diff --git a/tests/queries/0_stateless/02513_insert_without_materialized_columns.sh b/tests/queries/0_stateless/02513_insert_without_materialized_columns.sh new file mode 100755 index 00000000000..3faa404917d --- /dev/null +++ b/tests/queries/0_stateless/02513_insert_without_materialized_columns.sh @@ -0,0 +1,29 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +FILE_NAME="${CLICKHOUSE_DATABASE}_test.native.zstd" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS test" + +${CLICKHOUSE_CLIENT} --query "CREATE TABLE test (a Int64, b Int64 MATERIALIZED a) ENGINE = MergeTree() PRIMARY KEY tuple()" + +${CLICKHOUSE_CLIENT} --query "INSERT INTO test VALUES (1)" + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test" + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test INTO OUTFILE '${CLICKHOUSE_TMP}/${FILE_NAME}' FORMAT Native" + +${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE test" + +${CLICKHOUSE_CLIENT} --query "INSERT INTO test FROM INFILE '${CLICKHOUSE_TMP}/${FILE_NAME}'" + +${CLICKHOUSE_CLIENT} --query "SELECT * FROM test" + +${CLICKHOUSE_CLIENT} --query "DROP TABLE test" + +rm -f "${CLICKHOUSE_TMP}/${FILE_NAME}" diff --git a/tests/queries/0_stateless/02513_insert_without_materialized_columns.sql b/tests/queries/0_stateless/02513_insert_without_materialized_columns.sql deleted file mode 100644 index b16f9715b2f..00000000000 --- a/tests/queries/0_stateless/02513_insert_without_materialized_columns.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Tags: no-parallel - -DROP TABLE IF EXISTS test; -create table test ( - a Int64, - b Int64 materialized a -) -engine = MergeTree() -primary key tuple(); - -insert into test values (1); -SELECT * FROM test; - -select * from test into outfile 'test.native.zstd' format Native; -truncate table test; -insert into test from infile 'test.native.zstd'; - -SELECT * FROM test; - -DROP TABLE test; From fe28faa32d514f38adc57e148a012a9cd3673f94 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Dec 2022 14:49:39 +0100 Subject: [PATCH 074/339] Fix style --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 30fd3399c31..cbc87f921ef 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -69,7 +69,6 @@ namespace ErrorCodes extern const int DUPLICATE_COLUMN; extern const int THERE_IS_NO_COLUMN; extern const int UNKNOWN_EXCEPTION; - extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INCORRECT_DATA; } From 9c5a0aafcd02ea619f185886a9f9f64725780e06 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Dec 2022 14:50:06 +0100 Subject: [PATCH 075/339] Skip fasttest --- tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh b/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh index 780504b8807..455dccafbb9 100755 --- a/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh +++ b/tests/queries/0_stateless/02511_parquet_orc_missing_columns.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +#Tags: no-fasttest, no-parallel CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 00be9639f76ccb5365d30ff9637e14023f3febdc Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Dec 2022 15:21:41 +0000 Subject: [PATCH 076/339] Fix compiled functions --- src/AggregateFunctions/AggregateFunctionIf.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index 793ece53d23..09f3ca60c72 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -183,6 +183,11 @@ public: #if USE_EMBEDDED_COMPILER + bool isCompilable() const override + { + return canBeNativeType(*this->argument_types.back()) && this->nested_function->isCompilable(); + } + void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector & argument_values) const override { llvm::IRBuilder<> & b = static_cast &>(builder); @@ -360,6 +365,11 @@ public: #if USE_EMBEDDED_COMPILER + bool isCompilable() const override + { + return canBeNativeType(*this->argument_types.back()) && this->nested_function->isCompilable(); + } + void compileAdd(llvm::IRBuilderBase & builder, llvm::Value * aggregate_data_ptr, const DataTypes & arguments_types, const std::vector & argument_values) const override { /// TODO: Check From 5349a0b9663fe37f44301e85f59208024c264daa Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 20 Dec 2022 16:50:31 +0100 Subject: [PATCH 077/339] Update stateful test results, diff is caused by duplicate U+258B in the old version --- tests/queries/1_stateful/00047_bar.reference | 116 +++++++++--------- .../1_stateful/00062_loyalty.reference | 6 +- .../1_stateful/00063_loyalty_joins.reference | 6 +- 3 files changed, 64 insertions(+), 64 deletions(-) diff --git a/tests/queries/1_stateful/00047_bar.reference b/tests/queries/1_stateful/00047_bar.reference index c038f59946e..86f7ca3a9b6 100644 --- a/tests/queries/1_stateful/00047_bar.reference +++ b/tests/queries/1_stateful/00047_bar.reference @@ -2,21 +2,21 @@ 732797 475698 ████████████████████████████████████████████████████████████████████████▋ 598875 337212 ███████████████████████████████████████████████████▌ 792887 252197 ██████████████████████████████████████▌ -3807842 196036 █████████████████████████████▊ +3807842 196036 █████████████████████████████▉ 25703952 147211 ██████████████████████▌ -716829 90109 █████████████▋ +716829 90109 █████████████▊ 59183 85379 █████████████ -33010362 77807 ███████████▊ -800784 77492 ███████████▋ +33010362 77807 ███████████▉ +800784 77492 ███████████▊ 20810645 73213 ███████████▏ 25843850 68945 ██████████▌ 23447120 67570 ██████████▎ -14739804 64174 █████████▋ +14739804 64174 █████████▊ 32077710 60456 █████████▏ -22446879 58389 ████████▊ +22446879 58389 ████████▉ 170282 57017 ████████▋ 11482817 52345 ████████ -63469 52142 ███████▊ +63469 52142 ███████▉ 29103473 47758 ███████▎ 10136747 44080 ██████▋ 27528801 43395 ██████▋ @@ -27,12 +27,12 @@ 28600281 32776 █████ 32046685 28788 ████▍ 10130880 26603 ████ -8676831 25733 ███▊ -53230 25595 ███▊ -20271226 25585 ███▊ -17420663 25496 ███▊ -631207 25270 ███▋ -633130 24744 ███▋ +8676831 25733 ███▉ +53230 25595 ███▉ +20271226 25585 ███▉ +17420663 25496 ███▉ +631207 25270 ███▊ +633130 24744 ███▊ 14324015 23349 ███▌ 8537965 21270 ███▎ 11285298 20825 ███▏ @@ -41,9 +41,9 @@ 16368233 19897 ███ 81602 19724 ███ 62896 19717 ███ -12967664 19402 ██▊ -15996597 18557 ██▋ -4379238 18370 ██▋ +12967664 19402 ██▉ +15996597 18557 ██▊ +4379238 18370 ██▊ 90982 17443 ██▋ 18211045 17390 ██▋ 14625884 17302 ██▋ @@ -68,19 +68,19 @@ 125776 13308 ██ 11312316 13181 ██ 32667326 13181 ██ -28628973 12922 █▊ -122804 12520 █▊ -12322758 12352 █▊ -1301819 12283 █▊ -10769545 12183 █▋ -21566939 12170 █▋ -28905364 12158 █▋ -4250765 12049 █▋ -15009727 11818 █▋ -12761932 11733 █▋ -26995888 11658 █▋ -12759346 11514 █▋ -1507911 11452 █▋ +28628973 12922 █▉ +122804 12520 █▉ +12322758 12352 █▉ +1301819 12283 █▉ +10769545 12183 █▊ +21566939 12170 █▊ +28905364 12158 █▊ +4250765 12049 █▊ +15009727 11818 █▊ +12761932 11733 █▊ +26995888 11658 █▊ +12759346 11514 █▊ +1507911 11452 █▊ 968488 11444 █▋ 15736172 11358 █▋ 54310 11193 █▋ @@ -102,21 +102,21 @@ 732797 475698 ████████████████████████████████████████████████████████████████████████▋ 598875 337212 ███████████████████████████████████████████████████▌ 792887 252197 ██████████████████████████████████████▌ -3807842 196036 █████████████████████████████▊ +3807842 196036 █████████████████████████████▉ 25703952 147211 ██████████████████████▌ -716829 90109 █████████████▋ +716829 90109 █████████████▊ 59183 85379 █████████████ -33010362 77807 ███████████▊ -800784 77492 ███████████▋ +33010362 77807 ███████████▉ +800784 77492 ███████████▊ 20810645 73213 ███████████▏ 25843850 68945 ██████████▌ 23447120 67570 ██████████▎ -14739804 64174 █████████▋ +14739804 64174 █████████▊ 32077710 60456 █████████▏ -22446879 58389 ████████▊ +22446879 58389 ████████▉ 170282 57017 ████████▋ 11482817 52345 ████████ -63469 52142 ███████▊ +63469 52142 ███████▉ 29103473 47758 ███████▎ 10136747 44080 ██████▋ 27528801 43395 ██████▋ @@ -127,12 +127,12 @@ 28600281 32776 █████ 32046685 28788 ████▍ 10130880 26603 ████ -8676831 25733 ███▊ -53230 25595 ███▊ -20271226 25585 ███▊ -17420663 25496 ███▊ -631207 25270 ███▋ -633130 24744 ███▋ +8676831 25733 ███▉ +53230 25595 ███▉ +20271226 25585 ███▉ +17420663 25496 ███▉ +631207 25270 ███▊ +633130 24744 ███▊ 14324015 23349 ███▌ 8537965 21270 ███▎ 11285298 20825 ███▏ @@ -141,9 +141,9 @@ 16368233 19897 ███ 81602 19724 ███ 62896 19717 ███ -12967664 19402 ██▊ -15996597 18557 ██▋ -4379238 18370 ██▋ +12967664 19402 ██▉ +15996597 18557 ██▊ +4379238 18370 ██▊ 90982 17443 ██▋ 18211045 17390 ██▋ 14625884 17302 ██▋ @@ -168,19 +168,19 @@ 125776 13308 ██ 11312316 13181 ██ 32667326 13181 ██ -28628973 12922 █▊ -122804 12520 █▊ -12322758 12352 █▊ -1301819 12283 █▊ -10769545 12183 █▋ -21566939 12170 █▋ -28905364 12158 █▋ -4250765 12049 █▋ -15009727 11818 █▋ -12761932 11733 █▋ -26995888 11658 █▋ -12759346 11514 █▋ -1507911 11452 █▋ +28628973 12922 █▉ +122804 12520 █▉ +12322758 12352 █▉ +1301819 12283 █▉ +10769545 12183 █▊ +21566939 12170 █▊ +28905364 12158 █▊ +4250765 12049 █▊ +15009727 11818 █▊ +12761932 11733 █▊ +26995888 11658 █▊ +12759346 11514 █▊ +1507911 11452 █▊ 968488 11444 █▋ 15736172 11358 █▋ 54310 11193 █▋ diff --git a/tests/queries/1_stateful/00062_loyalty.reference b/tests/queries/1_stateful/00062_loyalty.reference index 605e4881dd4..f6451faa815 100644 --- a/tests/queries/1_stateful/00062_loyalty.reference +++ b/tests/queries/1_stateful/00062_loyalty.reference @@ -1,12 +1,12 @@ -10 5604 ███████████████████████████████████████████████████████████████████████████████▎ --9 603 ██████████████████████████████████████████████████████████▊ +-9 603 ██████████████████████████████████████████████████████████▉ -8 236 ██████████████████████████████████████████████████▎ -7 133 █████████████████████████████████████████████ -6 123 ████████████████████████████████████████████▎ --5 105 ██████████████████████████████████████████▊ +-5 105 ██████████████████████████████████████████▉ 5 82 ████████████████████████████████████████▋ 6 91 █████████████████████████████████████████▌ 7 102 ██████████████████████████████████████████▌ 8 156 ██████████████████████████████████████████████▍ 9 222 █████████████████████████████████████████████████▋ -10 4291 ████████████████████████████████████████████████████████████████████████████▊ +10 4291 ████████████████████████████████████████████████████████████████████████████▉ diff --git a/tests/queries/1_stateful/00063_loyalty_joins.reference b/tests/queries/1_stateful/00063_loyalty_joins.reference index e4c3619bf5a..f925b457c6a 100644 --- a/tests/queries/1_stateful/00063_loyalty_joins.reference +++ b/tests/queries/1_stateful/00063_loyalty_joins.reference @@ -37,15 +37,15 @@ 8 74083 9 145771 10 1244506 --10 2932018 ███████████████████████████████████████████████████████████████████████████████▊ +-10 2932018 ███████████████████████████████████████████████████████████████████████████████▉ -9 472052 ██████████████████████████████████████████████████████████████████████ -8 136048 ███████████████████████████████████████████████████████████████▍ -7 73688 ████████████████████████████████████████████████████████████ -6 56766 ██████████████████████████████████████████████████████████▋ -5 55691 ██████████████████████████████████████████████████████████▌ 5 47082 █████████████████████████████████████████████████████████▋ -6 32860 ███████████████████████████████████████████████████████▋ +6 32860 ███████████████████████████████████████████████████████▊ 7 52819 ██████████████████████████████████████████████████████████▎ 8 74083 ████████████████████████████████████████████████████████████▏ -9 145771 ███████████████████████████████████████████████████████████████▋ +9 145771 ███████████████████████████████████████████████████████████████▊ 10 1244506 ███████████████████████████████████████████████████████████████████████████▎ From 424ae4686e8e82fc2555e7bb360d6820c63a3a1c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Tue, 20 Dec 2022 17:02:30 +0100 Subject: [PATCH 078/339] Cleaups based on code review comments --- src/Common/UnicodeBar.cpp | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp index 259bdf227b2..37cc7592b6f 100644 --- a/src/Common/UnicodeBar.cpp +++ b/src/Common/UnicodeBar.cpp @@ -33,22 +33,25 @@ namespace UnicodeBar return (x - min) / (max - min) * max_width; } - /// We use the following Unicode characters to draw the bar: - /// U+2588 "█" Full block - /// U+2589 "▉" Left seven eighths block - /// U+258A "▊" Left three quarters block - /// U+258B "▋" Left five eighths block - /// U+258C "▌" Left half block - /// U+258D "▍" Left three eighths block - /// U+258E "▎" Left one quarter block - /// U+258F "▏" Left one eighth block - constexpr size_t GRADES_IN_FULL_BAR = 8; - constexpr char FULL_BAR[] = "█"; - constexpr char FRACTIONAL_BARS[] = "▏▎▍▌▋▊▉"; /// 7 elements: 1/8, 2/8, 3/8, 4/8, 5/8, 6/8, 7/8 + namespace + { + /// We use the following Unicode characters to draw the bar: + /// U+2588 "█" Full block + /// U+2589 "▉" Left seven eighths block + /// U+258A "▊" Left three quarters block + /// U+258B "▋" Left five eighths block + /// U+258C "▌" Left half block + /// U+258D "▍" Left three eighths block + /// U+258E "▎" Left one quarter block + /// U+258F "▏" Left one eighth block + constexpr size_t GRADES_IN_FULL_BAR = 8; + constexpr char FULL_BAR[] = "█"; + constexpr char FRACTIONAL_BARS[] = "▏▎▍▌▋▊▉"; /// 7 elements: 1/8, 2/8, 3/8, 4/8, 5/8, 6/8, 7/8 + } size_t getWidthInBytes(double width) { - Int64 int_width = Int64(width * GRADES_IN_FULL_BAR); + Int64 int_width = static_cast(width * GRADES_IN_FULL_BAR); return (int_width / GRADES_IN_FULL_BAR) * UNICODE_BAR_CHAR_SIZE + (int_width % GRADES_IN_FULL_BAR ? UNICODE_BAR_CHAR_SIZE : 0); } @@ -66,7 +69,7 @@ namespace UnicodeBar void render(double width, char * dst, const char * dst_end) { - Int64 int_width = Int64(width * GRADES_IN_FULL_BAR); + Int64 int_width = static_cast(width * GRADES_IN_FULL_BAR); size_t floor_width = (int_width / GRADES_IN_FULL_BAR); for (size_t i = 0; i < floor_width; ++i) From 20ee13613974ae307b2a7b9fe2ddd6fd31782b2c Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Dec 2022 16:12:24 +0000 Subject: [PATCH 079/339] Allow CRLF after bool value in CSV format --- src/DataTypes/Serializations/SerializationBool.cpp | 2 +- .../0_stateless/02513_csv_bool_allow_crlf.reference | 2 ++ tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh | 9 +++++++++ 3 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference create mode 100755 tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 1efacaaecc5..2a63b24c837 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -259,7 +259,7 @@ void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, if (istr.eof()) throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_BOOL); - deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n'; }); + deserializeImpl(column, istr, settings, [&](ReadBuffer & buf){ return buf.eof() || *buf.position() == settings.csv.delimiter || *buf.position() == '\n' || *buf.position() == '\r'; }); } void SerializationBool::serializeTextRaw(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const diff --git a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference new file mode 100644 index 00000000000..1cc8b5e10d3 --- /dev/null +++ b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference @@ -0,0 +1,2 @@ +True +False diff --git a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh new file mode 100755 index 00000000000..46cbf80b8e6 --- /dev/null +++ b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh @@ -0,0 +1,9 @@ +#!/usr/bin/env bash +# Tags: no-parallel, no-fasttest + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +echo -ne "True\r\nFalse\r\n" | $CLICKHOUSE_LOCAL --input-format=CSV -q "select * from table"; + From dfb2ae9113361f359b1798b45bf68c6c908ee109 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Dec 2022 16:13:39 +0000 Subject: [PATCH 080/339] Remove tags --- tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh index 46cbf80b8e6..2b75fe544e9 100755 --- a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh +++ b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh @@ -1,5 +1,4 @@ #!/usr/bin/env bash -# Tags: no-parallel, no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 03ed7c4097795a8348f0ed229c3455bde0464a28 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Dec 2022 19:09:48 +0100 Subject: [PATCH 081/339] Fix the test --- tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh index 2b75fe544e9..6ee4885c57f 100755 --- a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh +++ b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh @@ -4,5 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo -ne "True\r\nFalse\r\n" | $CLICKHOUSE_LOCAL --input-format=CSV -q "select * from table"; - +echo -ne "True\r\nFalse\r\n" | $CLICKHOUSE_LOCAL --stracture='x Bool' --input-format=CSV -q "select * from table"; From 2663fdbc4c7a811de57444efec2f0ca90c554fb6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Dec 2022 19:14:52 +0100 Subject: [PATCH 082/339] Update tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh Co-authored-by: Nick Sweeting --- tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh index 6ee4885c57f..ef75514cac6 100755 --- a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh +++ b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.sh @@ -4,4 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo -ne "True\r\nFalse\r\n" | $CLICKHOUSE_LOCAL --stracture='x Bool' --input-format=CSV -q "select * from table"; +echo -ne "True\r\nFalse\r\n" | $CLICKHOUSE_LOCAL --structure='x Bool' --input-format=CSV -q "select * from table"; From c79c80f841e0cf178dbb51782acd29ed3f1d398c Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 20 Dec 2022 19:56:29 +0100 Subject: [PATCH 083/339] better --- docker/test/fuzzer/run-fuzzer.sh | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 780bafe68eb..9a774a8a43a 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -91,16 +91,9 @@ function configure cp -av --dereference "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d cp -av --dereference "$script_dir"/allow-nullable-key.xml db/config.d - # the following memory limits configuration is copy-pasted from docker/test/stress/run.sh - local total_mem - total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB - total_mem=$(( total_mem*1024 )) # bytes - local max_server_mem - max_server_mem=$((total_mem*75/100)) # 75% - echo "Setting max_server_memory_usage=$max_server_mem" - cat > db/config.d/max_server_memory_usage.xml < db/config.d/max_server_memory_usage_to_ram_ratio.xml < - ${max_server_mem} + 0.75 EOL From bc0b35da1e3dea5f02e6c9f083d459e42d277bfb Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 20 Dec 2022 21:26:10 +0100 Subject: [PATCH 084/339] Update tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference Co-authored-by: Nick Sweeting --- tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference index 1cc8b5e10d3..da29283aaa4 100644 --- a/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference +++ b/tests/queries/0_stateless/02513_csv_bool_allow_crlf.reference @@ -1,2 +1,2 @@ -True -False +true +false From a7129332af1bab72e17fb4b2b73bb03e586021af Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Dec 2022 22:07:59 +0000 Subject: [PATCH 085/339] Don't execute and/or/if/multiIf on LowCardinality dictionary --- src/Functions/FunctionsLogical.h | 1 + src/Functions/if.cpp | 1 + src/Functions/multiIf.cpp | 1 + .../02514_if_with_lazy_low_cardinality.reference | 1 + .../0_stateless/02514_if_with_lazy_low_cardinality.sql | 5 +++++ .../02515_and_or_if_multiif_not_return_lc.reference | 4 ++++ .../0_stateless/02515_and_or_if_multiif_not_return_lc.sql | 6 ++++++ 7 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.reference create mode 100644 tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql create mode 100644 tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.reference create mode 100644 tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql diff --git a/src/Functions/FunctionsLogical.h b/src/Functions/FunctionsLogical.h index 30d8983b8cc..833191866e5 100644 --- a/src/Functions/FunctionsLogical.h +++ b/src/Functions/FunctionsLogical.h @@ -176,6 +176,7 @@ public: ColumnPtr executeShortCircuit(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const; bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } size_t getNumberOfArguments() const override { return 0; } + bool canBeExecutedOnLowCardinalityDictionary() const override { return false; } bool useDefaultImplementationForNulls() const override { return !Impl::specialImplementationForNulls(); } diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 0baf64c83d9..049e6d24920 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -1026,6 +1026,7 @@ public: } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t /*number_of_arguments*/) const override { return {0}; } + bool canBeExecutedOnLowCardinalityDictionary() const override { return false; } /// Get result types by argument types. If the function does not apply to these arguments, throw an exception. DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override diff --git a/src/Functions/multiIf.cpp b/src/Functions/multiIf.cpp index f658528a2a7..37301037c0e 100644 --- a/src/Functions/multiIf.cpp +++ b/src/Functions/multiIf.cpp @@ -51,6 +51,7 @@ public: size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForNothing() const override { return false; } + bool canBeExecutedOnLowCardinalityDictionary() const override { return false; } ColumnNumbers getArgumentsThatDontImplyNullableReturnType(size_t number_of_arguments) const override { diff --git a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.reference b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql new file mode 100644 index 00000000000..80e3c0a9ece --- /dev/null +++ b/tests/queries/0_stateless/02514_if_with_lazy_low_cardinality.sql @@ -0,0 +1,5 @@ +create table if not exists t (`arr.key` Array(LowCardinality(String)), `arr.value` Array(LowCardinality(String))) engine = Memory; +insert into t (`arr.key`, `arr.value`) values (['a'], ['b']); +select if(true, if(lowerUTF8(arr.key) = 'a', 1, 2), 3) as x from t left array join arr; +drop table t; + diff --git a/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.reference b/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.reference new file mode 100644 index 00000000000..805bbdf7a59 --- /dev/null +++ b/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.reference @@ -0,0 +1,4 @@ +UInt8 +UInt8 +UInt8 +UInt8 diff --git a/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql b/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql new file mode 100644 index 00000000000..f5ac870cdfe --- /dev/null +++ b/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql @@ -0,0 +1,6 @@ +select toTypeName(if(toLowCardinality(number % 2), 1, 2)) from numbers(1); +select toTypeName(multiIf(toLowCardinality(number % 2), 1, 1, 2, 3)) from numbers(1); +select toTypeName(toLowCardinality(number % 2) and 2) from numbers(1); +select toTypeName(toLowCardinality(number % 2) or 2) from numbers(1); + + From 4f376308b633a9534cc9e090d8d7f2203a11a739 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 20 Dec 2022 22:09:49 +0000 Subject: [PATCH 086/339] Remove extra line --- .../0_stateless/02515_and_or_if_multiif_not_return_lc.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql b/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql index f5ac870cdfe..0ccccd4d9a7 100644 --- a/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql +++ b/tests/queries/0_stateless/02515_and_or_if_multiif_not_return_lc.sql @@ -3,4 +3,3 @@ select toTypeName(multiIf(toLowCardinality(number % 2), 1, 1, 2, 3)) from number select toTypeName(toLowCardinality(number % 2) and 2) from numbers(1); select toTypeName(toLowCardinality(number % 2) or 2) from numbers(1); - From 2c66d21dc1428cf31b456e95dcf20939f5d64b44 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 21 Dec 2022 01:44:17 +0300 Subject: [PATCH 087/339] Update 02509_h3_arguments.sql --- tests/queries/0_stateless/02509_h3_arguments.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02509_h3_arguments.sql b/tests/queries/0_stateless/02509_h3_arguments.sql index f3a2a5d94bf..b5b8b9497f9 100644 --- a/tests/queries/0_stateless/02509_h3_arguments.sql +++ b/tests/queries/0_stateless/02509_h3_arguments.sql @@ -1,3 +1,5 @@ +-- Tags: no-fasttest + select h3ToParent(641573946153969375, 1); select h3ToParent(641573946153969375, arrayJoin([1,2])); From 823cb8a8f0ef1c87a9b8741d61c34df0ad19d499 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Fri, 16 Dec 2022 19:27:48 +0800 Subject: [PATCH 088/339] Call std::condition_variable::notify_one w/o lock in ThreadPool Calling notify_one function with lock held would let the notified thread immediately block again, and increase the lock contention. This commit moves the calls of notify_one function out of critical sections of ThreadPool, whose lock (ThreadPoolImpl::mutex) is most extensively contended in some workloads. --- src/Common/ThreadPool.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index b70b1fc5e60..e65b5511e05 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -156,9 +156,10 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, ssize_t priority, std:: propagate_opentelemetry_tracing_context ? DB::OpenTelemetry::CurrentContext() : DB::OpenTelemetry::TracingContextOnThread()); ++scheduled_jobs; - new_job_or_shutdown.notify_one(); } + new_job_or_shutdown.notify_one(); + return static_cast(true); } From 4b0cbdb4a4871a738c0a95b69d0140c56e9b4896 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 21 Dec 2022 10:55:20 +0100 Subject: [PATCH 089/339] Fixed unused value warning --- src/Common/UnicodeBar.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/UnicodeBar.cpp b/src/Common/UnicodeBar.cpp index 37cc7592b6f..bad39d8080c 100644 --- a/src/Common/UnicodeBar.cpp +++ b/src/Common/UnicodeBar.cpp @@ -84,7 +84,7 @@ namespace UnicodeBar dst = checkedCopy(&FRACTIONAL_BARS[(remainder - 1) * UNICODE_BAR_CHAR_SIZE], UNICODE_BAR_CHAR_SIZE, dst, dst_end); } - dst = checkedCopy("\0", 1, dst, dst_end); + checkedCopy("\0", 1, dst, dst_end); } std::string render(double width) From 6a9ec7efb1cf1e162fe1e5a599c2c7ff2d55bd88 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 21 Dec 2022 23:43:56 +0800 Subject: [PATCH 090/339] If user only need virtual columns, we don't need to initialize ReadBufferFromS3 --- src/Storages/StorageS3.cpp | 92 ++++++++++++++++++++++++-------------- src/Storages/StorageS3.h | 4 +- 2 files changed, 62 insertions(+), 34 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 9b50b41ddaf..9387e370b70 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -529,7 +529,8 @@ StorageS3Source::StorageS3Source( const String & bucket_, const String & version_id_, std::shared_ptr file_iterator_, - const size_t download_thread_num_) + const size_t download_thread_num_, + bool only_need_virtual_columns_) : ISource(getHeader(sample_block_, requested_virtual_columns_)) , WithContext(context_) , name(std::move(name_)) @@ -543,6 +544,7 @@ StorageS3Source::StorageS3Source( , client(client_) , sample_block(sample_block_) , format_settings(format_settings_) + , only_need_virtual_columns(only_need_virtual_columns_) , requested_virtual_columns(requested_virtual_columns_) , file_iterator(file_iterator_) , download_thread_num(download_thread_num_) @@ -569,6 +571,9 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() if (current_key.empty()) return {}; + if (only_need_virtual_columns) + return ReaderHolder{fs::path(bucket) / current_key, nullptr, nullptr, nullptr}; + size_t object_size = info ? info->size : S3::getObjectSize(*client, bucket, current_key, version_id, true, false); @@ -683,6 +688,33 @@ String StorageS3Source::getName() const Chunk StorageS3Source::generate() { + auto add_virtual_column = [&](Chunk & chunk, UInt64 num_rows) + { + const auto & file_path = reader.getPath(); + + for (const auto & virtual_column : requested_virtual_columns) + { + if (virtual_column.name == "_path") + { + chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); + } + else if (virtual_column.name == "_file") + { + size_t last_slash_pos = file_path.find_last_of('/'); + auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); + chunk.addColumn(column->convertToFullColumnIfConst()); + } + } + }; + + if (!reader.getPath().empty() && only_need_virtual_columns) + { + Chunk chunk; + add_virtual_column(chunk, 1); + only_need_virtual_columns = false; + return chunk; + } + while (true) { if (!reader || isCancelled()) @@ -693,7 +725,6 @@ Chunk StorageS3Source::generate() { UInt64 num_rows = chunk.getNumRows(); - const auto & file_path = reader.getPath(); size_t total_size = file_iterator->getTotalSize(); if (num_rows && total_size) { @@ -701,20 +732,7 @@ Chunk StorageS3Source::generate() *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); } - for (const auto & virtual_column : requested_virtual_columns) - { - if (virtual_column.name == "_path") - { - chunk.addColumn(virtual_column.type->createColumnConst(num_rows, file_path)->convertToFullColumnIfConst()); - } - else if (virtual_column.name == "_file") - { - size_t last_slash_pos = file_path.find_last_of('/'); - auto column = virtual_column.type->createColumnConst(num_rows, file_path.substr(last_slash_pos + 1)); - chunk.addColumn(column->convertToFullColumnIfConst()); - } - } - + add_virtual_column(chunk, num_rows); return chunk; } @@ -1035,6 +1053,10 @@ Pipe StorageS3::read( requested_virtual_columns.push_back(virtual_column); } + bool only_need_virtual_columns = true; + if (column_names_set.size() > requested_virtual_columns.size()) + only_need_virtual_columns = false; + std::shared_ptr iterator_wrapper = createFileIterator( s3_configuration, keys, @@ -1047,25 +1069,28 @@ Pipe StorageS3::read( ColumnsDescription columns_description; Block block_for_format; - if (supportsSubsetOfColumns()) + if (!only_need_virtual_columns) { - auto fetch_columns = column_names; - const auto & virtuals = getVirtuals(); - std::erase_if( - fetch_columns, - [&](const String & col) - { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); + if (supportsSubsetOfColumns()) + { + auto fetch_columns = column_names; + const auto & virtuals = getVirtuals(); + std::erase_if( + fetch_columns, + [&](const String & col) + { return std::any_of(virtuals.begin(), virtuals.end(), [&](const NameAndTypePair & virtual_col){ return col == virtual_col.name; }); }); - if (fetch_columns.empty()) - fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); + if (fetch_columns.empty()) + fetch_columns.push_back(ExpressionActions::getSmallestColumn(storage_snapshot->metadata->getColumns().getAllPhysical())); - columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); - block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); - } - else - { - columns_description = storage_snapshot->metadata->getColumns(); - block_for_format = storage_snapshot->metadata->getSampleBlock(); + columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); + block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + } + else + { + columns_description = storage_snapshot->metadata->getColumns(); + block_for_format = storage_snapshot->metadata->getSampleBlock(); + } } const size_t max_download_threads = local_context->getSettingsRef().max_download_threads; @@ -1086,7 +1111,8 @@ Pipe StorageS3::read( s3_configuration.uri.bucket, s3_configuration.uri.version_id, iterator_wrapper, - max_download_threads)); + max_download_threads, + only_need_virtual_columns)); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 671610173bd..acd5c264822 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -137,7 +137,8 @@ public: const String & bucket, const String & version_id, std::shared_ptr file_iterator_, - size_t download_thread_num); + size_t download_thread_num, + bool only_need_virtual_columns_ = false); ~StorageS3Source() override; @@ -159,6 +160,7 @@ private: std::shared_ptr client; Block sample_block; std::optional format_settings; + bool only_need_virtual_columns{false}; struct ReaderHolder { From 07f2f2fae5ac4efbe0f37a7bcc5984569fb1f43d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 21 Dec 2022 21:30:33 +0000 Subject: [PATCH 091/339] remove clickhouse_parsers dependency on DataTypes --- src/Parsers/ParserCreateQuery.h | 9 +++++++-- src/Parsers/examples/CMakeLists.txt | 4 ++-- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index 6ad7b9e6dd9..ef87988aab2 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -240,7 +240,12 @@ bool IParserColumnDeclaration::parseImpl(Pos & pos, ASTPtr & node, E if (!expr_parser.parse(pos, default_expression, expected) && type) { ephemeral_default = true; - default_expression = std::make_shared(DataTypeFactory::instance().get(type)->getDefault()); + + auto default_function = std::make_shared(); + default_function->name = "defaultValueOfTypeName"; + default_function->arguments = std::make_shared(); + default_function->arguments->children.emplace_back(std::make_shared(type->as()->formatWithSecretsHidden())); + default_expression = default_function; } if (!default_expression && !type) diff --git a/src/Parsers/examples/CMakeLists.txt b/src/Parsers/examples/CMakeLists.txt index 7a422e833d8..82ca7bc0688 100644 --- a/src/Parsers/examples/CMakeLists.txt +++ b/src/Parsers/examples/CMakeLists.txt @@ -4,7 +4,7 @@ clickhouse_add_executable(lexer lexer.cpp ${SRCS}) target_link_libraries(lexer PRIVATE clickhouse_parsers) clickhouse_add_executable(select_parser select_parser.cpp ${SRCS}) -target_link_libraries(select_parser PRIVATE clickhouse_parsers dbms) +target_link_libraries(select_parser PRIVATE clickhouse_parsers) clickhouse_add_executable(create_parser create_parser.cpp ${SRCS}) -target_link_libraries(create_parser PRIVATE clickhouse_parsers dbms) +target_link_libraries(create_parser PRIVATE clickhouse_parsers) From 3d04a33de5d471309146f210ce8de8ce13331db1 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 21 Dec 2022 23:15:45 +0000 Subject: [PATCH 092/339] update tests --- tests/queries/0_stateless/02205_ephemeral_1.reference | 2 +- ...ompatibility_ignore_auto_increment_in_create_table.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02205_ephemeral_1.reference b/tests/queries/0_stateless/02205_ephemeral_1.reference index ba39033668f..7c034ca72ea 100644 --- a/tests/queries/0_stateless/02205_ephemeral_1.reference +++ b/tests/queries/0_stateless/02205_ephemeral_1.reference @@ -7,7 +7,7 @@ z UInt32 DEFAULT 5 7 5 21 5 x UInt32 DEFAULT y -y UInt32 EPHEMERAL 0 +y UInt32 EPHEMERAL defaultValueOfTypeName(\'UInt32\') z UInt32 DEFAULT 5 1 2 0 2 diff --git a/tests/queries/0_stateless/02293_compatibility_ignore_auto_increment_in_create_table.reference b/tests/queries/0_stateless/02293_compatibility_ignore_auto_increment_in_create_table.reference index 2db591f7e5a..63d74e4ea1e 100644 --- a/tests/queries/0_stateless/02293_compatibility_ignore_auto_increment_in_create_table.reference +++ b/tests/queries/0_stateless/02293_compatibility_ignore_auto_increment_in_create_table.reference @@ -11,7 +11,7 @@ s String create table, several columns with different default specifiers di UInt8 DEFAULT 1 id Int32 -s String EPHEMERAL \'\' +s String EPHEMERAL defaultValueOfTypeName(\'String\') create table failed, column +type +DEFAULT +AUTO_INCREMENT create table failed, column -type +DEFAULT +AUTO_INCREMENT create table failed, column +type +AUTO_INCREMENT +DEFAULT From 9fc102085586433722a7aef72900e4442fcd8b5d Mon Sep 17 00:00:00 2001 From: chen Date: Thu, 22 Dec 2022 08:15:50 +0800 Subject: [PATCH 093/339] Update StorageS3.cpp --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 9387e370b70..04155d699bd 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1086,7 +1086,7 @@ Pipe StorageS3::read( columns_description = storage_snapshot->getDescriptionForColumns(fetch_columns); block_for_format = storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); } - else + else { columns_description = storage_snapshot->metadata->getColumns(); block_for_format = storage_snapshot->metadata->getSampleBlock(); From 5c8fb627b3657a7cd64cdd23fe0fe37724e77fc5 Mon Sep 17 00:00:00 2001 From: chen Date: Thu, 22 Dec 2022 08:28:09 +0800 Subject: [PATCH 094/339] Update StorageS3.cpp --- src/Storages/StorageS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 04155d699bd..d3c09d36872 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -691,7 +691,7 @@ Chunk StorageS3Source::generate() auto add_virtual_column = [&](Chunk & chunk, UInt64 num_rows) { const auto & file_path = reader.getPath(); - + for (const auto & virtual_column : requested_virtual_columns) { if (virtual_column.name == "_path") From fa467b3398c8fe12147351de5b16899370db7e28 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Dec 2022 14:31:42 +0100 Subject: [PATCH 095/339] Minor code polishing --- src/Common/ZooKeeper/TestKeeper.cpp | 1 + .../ZooKeeper}/ZooKeeperWithFaultInjection.h | 0 src/Disks/FakeDiskTransaction.h | 1 + src/Disks/IDiskTransaction.h | 2 + .../ObjectStorages/DiskObjectStorage.cpp | 6 + .../DiskObjectStorageMetadata.cpp | 20 +- .../DiskObjectStorageMetadata.h | 13 + .../DiskObjectStorageTransaction.cpp | 6 + .../DiskObjectStorageTransaction.h | 1 + .../FakeMetadataStorageFromDisk.cpp | 29 ++- .../FakeMetadataStorageFromDisk.h | 6 +- src/Disks/ObjectStorages/IMetadataStorage.h | 14 +- .../MetadataStorageFromDisk.cpp | 17 +- .../ObjectStorages/MetadataStorageFromDisk.h | 8 +- .../MetadataStorageFromPlainObjectStorage.cpp | 2 +- .../MetadataStorageFromPlainObjectStorage.h | 2 +- .../ObjectStorages/S3/registerDiskS3.cpp | 1 + ...etadataStorageFromStaticFilesWebServer.cpp | 2 +- .../MetadataStorageFromStaticFilesWebServer.h | 2 +- src/Interpreters/DDLWorker.cpp | 6 +- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- .../ServerAsynchronousMetrics.cpp | 2 +- src/Storages/AlterCommands.cpp | 10 +- src/Storages/AlterCommands.h | 2 +- src/Storages/MergeTree/ActiveDataPartSet.cpp | 25 +- src/Storages/MergeTree/ActiveDataPartSet.h | 3 + .../MergeTree/DataPartStorageOnDisk.cpp | 14 +- .../MergeTree/DataPartStorageOnDisk.h | 7 +- .../MergeTree/EphemeralLockInZooKeeper.cpp | 2 +- src/Storages/MergeTree/IDataPartStorage.h | 9 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 6 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/MergeTask.cpp | 11 +- src/Storages/MergeTree/MergeTask.h | 4 + src/Storages/MergeTree/MergeTreeData.cpp | 245 ++++++++++++------ src/Storages/MergeTree/MergeTreeData.h | 29 ++- .../MergeTree/MergeTreeDataFormatVersion.h | 1 + .../MergeTree/MergeTreeDataMergerMutator.cpp | 10 +- .../MergeTree/MergeTreeDataMergerMutator.h | 4 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- .../MergeTree/MergeTreeDataPartInMemory.h | 2 +- .../MergeTree/MergeTreeDataWriter.cpp | 45 +++- src/Storages/MergeTree/MergeTreeDataWriter.h | 10 + .../MergeTree/MergeTreeMutationEntry.cpp | 4 +- .../MergeTree/MergeTreeMutationStatus.h | 3 + src/Storages/MergeTree/MergeTreePartInfo.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 20 +- .../MergeTree/MutateFromLogEntryTask.cpp | 1 - src/Storages/MergeTree/MutateTask.cpp | 28 +- src/Storages/MergeTree/MutateTask.h | 5 +- .../ReplicatedMergeTreeMutationEntry.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeSink.h | 2 +- .../MergeTree/ReplicatedTableStatus.h | 35 +++ .../MergeTree/registerStorageMergeTree.cpp | 140 +++++----- src/Storages/MutationCommands.cpp | 39 ++- src/Storages/MutationCommands.h | 7 +- src/Storages/RenamingRestrictions.h | 13 + src/Storages/StorageMergeTree.cpp | 7 +- src/Storages/StorageReplicatedMergeTree.cpp | 11 +- src/Storages/StorageReplicatedMergeTree.h | 36 +-- src/Storages/System/StorageSystemReplicas.cpp | 2 +- 62 files changed, 662 insertions(+), 289 deletions(-) rename src/{Storages/MergeTree => Common/ZooKeeper}/ZooKeeperWithFaultInjection.h (100%) create mode 100644 src/Storages/MergeTree/ReplicatedTableStatus.h create mode 100644 src/Storages/RenamingRestrictions.h diff --git a/src/Common/ZooKeeper/TestKeeper.cpp b/src/Common/ZooKeeper/TestKeeper.cpp index 134374f98d0..4f53a8ac307 100644 --- a/src/Common/ZooKeeper/TestKeeper.cpp +++ b/src/Common/ZooKeeper/TestKeeper.cpp @@ -219,6 +219,7 @@ std::pair TestKeeperCreateRequest::process(TestKeeper::Contai created_node.stat.mtime = created_node.stat.ctime; created_node.stat.numChildren = 0; created_node.stat.dataLength = static_cast(data.length()); + created_node.stat.ephemeralOwner = is_ephemeral ? 1 : 0; created_node.data = data; created_node.is_ephemeral = is_ephemeral; created_node.is_sequental = is_sequential; diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h similarity index 100% rename from src/Storages/MergeTree/ZooKeeperWithFaultInjection.h rename to src/Common/ZooKeeper/ZooKeeperWithFaultInjection.h diff --git a/src/Disks/FakeDiskTransaction.h b/src/Disks/FakeDiskTransaction.h index e80b45a94ec..46be885739e 100644 --- a/src/Disks/FakeDiskTransaction.h +++ b/src/Disks/FakeDiskTransaction.h @@ -16,6 +16,7 @@ public: {} void commit() override {} + void undo() override {} void createDirectory(const std::string & path) override { diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index 572d86dcfdb..02c8731428d 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -30,6 +30,8 @@ public: /// If something fails rollback and throw exception. virtual void commit() = 0; + virtual void undo() = 0; + virtual ~IDiskTransaction() = default; /// Create directory. diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 263a9a9d0e1..81076396e17 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes extern const int FILE_DOESNT_EXIST; extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int CANNOT_READ_ALL_DATA; + extern const int DIRECTORY_DOESNT_EXIST; } namespace @@ -126,6 +127,9 @@ StoredObjects DiskObjectStorage::getStorageObjects(const String & local_path) co void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std::vector & paths_map) { + if (!metadata_storage->exists(local_path)) + return; + /// Protect against concurrent delition of files (for example because of a merge). if (metadata_storage->isFile(local_path)) { @@ -138,6 +142,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: /// Unfortunately in rare cases it can happen when files disappear /// or can be empty in case of operation interruption (like cancelled metadata fetch) if (e.code() == ErrorCodes::FILE_DOESNT_EXIST || + e.code() == ErrorCodes::DIRECTORY_DOESNT_EXIST || e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF || e.code() == ErrorCodes::CANNOT_READ_ALL_DATA) return; @@ -157,6 +162,7 @@ void DiskObjectStorage::getRemotePathsRecursive(const String & local_path, std:: /// Unfortunately in rare cases it can happen when files disappear /// or can be empty in case of operation interruption (like cancelled metadata fetch) if (e.code() == ErrorCodes::FILE_DOESNT_EXIST || + e.code() == ErrorCodes::DIRECTORY_DOESNT_EXIST || e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF || e.code() == ErrorCodes::CANNOT_READ_ALL_DATA) return; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp index dc4898559c0..c3284b635da 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.cpp @@ -20,7 +20,7 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) UInt32 version; readIntText(version, buf); - if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_READ_ONLY_FLAG) + if (version < VERSION_ABSOLUTE_PATHS || version > VERSION_INLINE_DATA) throw Exception( ErrorCodes::UNKNOWN_FORMAT, "Unknown metadata file version. Path: {}. Version: {}. Maximum expected version: {}", @@ -65,6 +65,12 @@ void DiskObjectStorageMetadata::deserialize(ReadBuffer & buf) readBoolText(read_only, buf); assertChar('\n', buf); } + + if (version >= VERSION_INLINE_DATA) + { + readEscapedString(inline_data, buf); + assertChar('\n', buf); + } } void DiskObjectStorageMetadata::deserializeFromString(const std::string & data) @@ -75,7 +81,11 @@ void DiskObjectStorageMetadata::deserializeFromString(const std::string & data) void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const { - writeIntText(VERSION_READ_ONLY_FLAG, buf); + if (inline_data.empty()) + writeIntText(VERSION_READ_ONLY_FLAG, buf); + else + writeIntText(VERSION_INLINE_DATA, buf); + writeChar('\n', buf); writeIntText(storage_objects.size(), buf); @@ -97,6 +107,12 @@ void DiskObjectStorageMetadata::serialize(WriteBuffer & buf, bool sync) const writeBoolText(read_only, buf); writeChar('\n', buf); + if (!inline_data.empty()) + { + writeEscapedString(inline_data, buf); + writeChar('\n', buf); + } + buf.finalize(); if (sync) buf.sync(); diff --git a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h index d3ea5795dd3..a2d0653e4aa 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageMetadata.h @@ -16,6 +16,7 @@ private: static constexpr uint32_t VERSION_ABSOLUTE_PATHS = 1; static constexpr uint32_t VERSION_RELATIVE_PATHS = 2; static constexpr uint32_t VERSION_READ_ONLY_FLAG = 3; + static constexpr uint32_t VERSION_INLINE_DATA = 4; const std::string & common_metadata_path; @@ -39,6 +40,9 @@ private: /// Flag indicates that file is read only. bool read_only = false; + /// This data will be stored inline + std::string inline_data; + public: DiskObjectStorageMetadata( @@ -99,6 +103,15 @@ public: read_only = true; } + void setInlineData(const std::string & data) + { + inline_data = data; + } + + const std::string & getInlineData() const + { + return inline_data; + } }; using DiskObjectStorageMetadataPtr = std::unique_ptr; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index b55fb2c4fa5..69dba263dda 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -749,4 +749,10 @@ void DiskObjectStorageTransaction::commit() operation->finalize(); } +void DiskObjectStorageTransaction::undo() +{ + for (const auto & operation : operations_to_execute | std::views::reverse) + operation->undo(); +} + } diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index 9c42203b613..9e6bd5b6307 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -70,6 +70,7 @@ public: DiskObjectStorageRemoteMetadataRestoreHelper * metadata_helper_); void commit() override; + void undo() override; void createDirectory(const std::string & path) override; diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp index 383bbebd880..dbfdb2f7b1a 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.cpp @@ -23,7 +23,7 @@ FakeMetadataStorageFromDisk::FakeMetadataStorageFromDisk( { } -MetadataTransactionPtr FakeMetadataStorageFromDisk::createTransaction() const +MetadataTransactionPtr FakeMetadataStorageFromDisk::createTransaction() { return std::make_shared(*this, disk); } @@ -66,12 +66,7 @@ uint64_t FakeMetadataStorageFromDisk::getFileSize(const String & path) const std::vector FakeMetadataStorageFromDisk::listDirectory(const std::string & path) const { std::vector result; - auto it = disk->iterateDirectory(path); - while (it->isValid()) - { - result.push_back(it->path()); - it->next(); - } + disk->listFiles(path, result); return result; } @@ -85,6 +80,19 @@ std::string FakeMetadataStorageFromDisk::readFileToString(const std::string &) c throw Exception(ErrorCodes::NOT_IMPLEMENTED, "readFileToString is not implemented for FakeMetadataStorageFromDisk"); } +std::string FakeMetadataStorageFromDisk::readInlineDataToString(const std::string & path) const +{ + auto rb = disk->readFile(path); + std::string result; + std::array buf; + while (!rb->eof()) + { + auto sz = rb->read(buf.data(), buf.size()); + result.append(buf.data(), buf.data() + sz); + } + return result; +} + std::unordered_map FakeMetadataStorageFromDisk::getSerializedMetadata(const std::vector &) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getSerializedMetadata is not implemented for FakeMetadataStorageFromDisk"); @@ -118,6 +126,13 @@ void FakeMetadataStorageFromDiskTransaction::writeStringToFile(const std::string wb->finalize(); } +void FakeMetadataStorageFromDiskTransaction::writeInlineDataToFile(const std::string & path, const std::string & data) +{ + auto wb = disk->writeFile(path); + wb->write(data.data(), data.size()); + wb->finalize(); +} + void FakeMetadataStorageFromDiskTransaction::setLastModified(const std::string & path, const Poco::Timestamp & timestamp) { disk->setLastModified(path, timestamp); diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h index 65cf012ddab..849e7235c0a 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h @@ -27,7 +27,7 @@ public: ObjectStoragePtr object_storage_, const std::string & object_storage_root_path_); - MetadataTransactionPtr createTransaction() const override; + MetadataTransactionPtr createTransaction() override; const std::string & getPath() const override; @@ -55,6 +55,8 @@ public: std::string readFileToString(const std::string & path) const override; + std::string readInlineDataToString(const std::string & path) const override; + std::unordered_map getSerializedMetadata(const std::vector & file_paths) const override; uint32_t getHardlinkCount(const std::string & path) const override; @@ -88,6 +90,8 @@ public: void writeStringToFile(const std::string & path, const std::string & data) override; + void writeInlineDataToFile(const std::string & path, const std::string & data) override; + void createEmptyMetadataFile(const std::string & path) override; void createMetadataFile(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 597d7744c78..00150df9fa3 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -44,6 +44,12 @@ public: throwNotImplemented(); } + /// Writes the data inline with the metadata + virtual void writeInlineDataToFile(const std::string & /* path */, const std::string & /* data */) + { + throwNotImplemented(); + } + virtual void setLastModified(const std::string & /* path */, const Poco::Timestamp & /* timestamp */) { throwNotImplemented(); @@ -143,7 +149,7 @@ using MetadataTransactionPtr = std::shared_ptr; class IMetadataStorage : private boost::noncopyable { public: - virtual MetadataTransactionPtr createTransaction() const = 0; + virtual MetadataTransactionPtr createTransaction() = 0; /// Get metadata root path. virtual const std::string & getPath() const = 0; @@ -185,6 +191,12 @@ public: throwNotImplemented(); } + /// Read inline data for file to string from path + virtual std::string readInlineDataToString(const std::string & /* path */) const + { + throwNotImplemented(); + } + virtual ~IMetadataStorage() = default; /// ==== More specific methods. Previous were almost general purpose. ==== diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 0ad46a1327d..625350eeeff 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -78,6 +78,11 @@ std::string MetadataStorageFromDisk::readFileToString(const std::string & path) return result; } +std::string MetadataStorageFromDisk::readInlineDataToString(const std::string & path) const +{ + return readMetadata(path)->getInlineData(); +} + DiskObjectStorageMetadataPtr MetadataStorageFromDisk::readMetadataUnlocked(const std::string & path, std::shared_lock &) const { auto metadata = std::make_unique(disk->getPath(), object_storage_root_path, path); @@ -122,7 +127,7 @@ void MetadataStorageFromDiskTransaction::createHardLink(const std::string & path addOperation(std::make_unique(path_from, path_to, *metadata_storage.disk, metadata_storage)); } -MetadataTransactionPtr MetadataStorageFromDisk::createTransaction() const +MetadataTransactionPtr MetadataStorageFromDisk::createTransaction() { return std::make_shared(*this); } @@ -244,6 +249,16 @@ void MetadataStorageFromDiskTransaction::writeStringToFile( addOperation(std::make_unique(path, *metadata_storage.getDisk(), data)); } +void MetadataStorageFromDiskTransaction::writeInlineDataToFile( + const std::string & path, + const std::string & data) +{ + auto metadata = std::make_unique( + metadata_storage.getDisk()->getPath(), metadata_storage.getObjectStorageRootPath(), path); + metadata->setInlineData(data); + writeStringToFile(path, metadata->serializeToString()); +} + void MetadataStorageFromDiskTransaction::setLastModified(const std::string & path, const Poco::Timestamp & timestamp) { addOperation(std::make_unique(path, timestamp, *metadata_storage.getDisk())); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index b06a2a41f2b..2c80572e7b4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -25,7 +25,7 @@ private: public: MetadataStorageFromDisk(DiskPtr disk_, const std::string & object_storage_root_path_); - MetadataTransactionPtr createTransaction() const override; + MetadataTransactionPtr createTransaction() override; const std::string & getPath() const override; @@ -53,6 +53,8 @@ public: std::string readFileToString(const std::string & path) const override; + std::string readInlineDataToString(const std::string & path) const override; + std::unordered_map getSerializedMetadata(const std::vector & file_paths) const override; uint32_t getHardlinkCount(const std::string & path) const override; @@ -94,6 +96,8 @@ public: void writeStringToFile(const std::string & path, const std::string & data) override; + void writeInlineDataToFile(const std::string & path, const std::string & data) override; + void createEmptyMetadataFile(const std::string & path) override; void createMetadataFile(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; @@ -127,6 +131,8 @@ public: void replaceFile(const std::string & path_from, const std::string & path_to) override; void unlinkMetadata(const std::string & path) override; + + }; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 34a9ae021b7..62c6d57b16f 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -23,7 +23,7 @@ MetadataStorageFromPlainObjectStorage::MetadataStorageFromPlainObjectStorage( { } -MetadataTransactionPtr MetadataStorageFromPlainObjectStorage::createTransaction() const +MetadataTransactionPtr MetadataStorageFromPlainObjectStorage::createTransaction() { return std::make_shared(*this); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 99cc960b9e4..0beed65879b 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -33,7 +33,7 @@ public: ObjectStoragePtr object_storage_, const std::string & object_storage_root_path_); - MetadataTransactionPtr createTransaction() const override; + MetadataTransactionPtr createTransaction() override; const std::string & getPath() const override; diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index 533a925aa1b..860a729b12c 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -13,6 +13,7 @@ #include #include +#include #include #include #include diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp index ab5d86fd836..d39582a089e 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp @@ -21,7 +21,7 @@ MetadataStorageFromStaticFilesWebServer::MetadataStorageFromStaticFilesWebServer { } -MetadataTransactionPtr MetadataStorageFromStaticFilesWebServer::createTransaction() const +MetadataTransactionPtr MetadataStorageFromStaticFilesWebServer::createTransaction() { return std::make_shared(*this); } diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h index 6a7c8128b4a..a04a1359d34 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h @@ -24,7 +24,7 @@ private: public: explicit MetadataStorageFromStaticFilesWebServer(const WebObjectStorage & object_storage_); - MetadataTransactionPtr createTransaction() const override; + MetadataTransactionPtr createTransaction() override; const std::string & getPath() const override; diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index 2e1918e1a37..4888428979f 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -748,13 +748,13 @@ bool DDLWorker::tryExecuteQueryOnLeaderReplica( /// but DDL worker can continue processing other queries. while (stopwatch.elapsedSeconds() <= MAX_EXECUTION_TIMEOUT_SEC) { - StorageReplicatedMergeTree::Status status; + ReplicatedTableStatus status; // Has to get with zk fields to get active replicas field replicated_storage->getStatus(status, true); // Should return as soon as possible if the table is dropped. - bool replica_dropped = replicated_storage->is_dropped; - bool all_replicas_likely_detached = status.active_replicas == 0 && !DatabaseCatalog::instance().isTableExist(replicated_storage->getStorageID(), context); + bool replica_dropped = storage->is_dropped; + bool all_replicas_likely_detached = status.active_replicas == 0 && !DatabaseCatalog::instance().isTableExist(storage->getStorageID(), context); if (replica_dropped || all_replicas_likely_detached) { LOG_WARNING(log, ", task {} will not be executed.", task.entry_name); diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 22edac051a5..14628f34111 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -143,7 +143,7 @@ BlockIO InterpreterAlterQuery::executeToTable(const ASTAlterQuery & alter) "to execute ALTERs of different types in single query"); } - if (!mutation_commands.empty()) + if (mutation_commands.hasNonEmptyMutationCommands()) { table->checkMutationIsPossible(mutation_commands, getContext()->getSettingsRef()); MutationsInterpreter(table, metadata_snapshot, mutation_commands, getContext(), false).validate(); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index d05fd70e074..f41686e8a2a 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -723,7 +723,7 @@ void InterpreterSystemQuery::dropReplica(ASTSystemQuery & query) { if (auto * storage_replicated = dynamic_cast(iterator->table().get())) { - StorageReplicatedMergeTree::Status status; + ReplicatedTableStatus status; storage_replicated->getStatus(status); if (status.zookeeper_path == query.replica_zk_path) throw Exception("There is a local table " + storage_replicated->getStorageID().getNameForLogs() + @@ -759,7 +759,7 @@ bool InterpreterSystemQuery::dropReplicaImpl(ASTSystemQuery & query, const Stora if (!storage_replicated) return false; - StorageReplicatedMergeTree::Status status; + ReplicatedTableStatus status; auto zookeeper = getContext()->getZooKeeper(); storage_replicated->getStatus(status); diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index dc4a2a8e435..0777ffd6c44 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -250,7 +250,7 @@ void ServerAsynchronousMetrics::updateImpl(AsynchronousMetricValues & new_values if (StorageReplicatedMergeTree * table_replicated_merge_tree = typeid_cast(table.get())) { - StorageReplicatedMergeTree::Status status; + ReplicatedTableStatus status; table_replicated_merge_tree->getStatus(status, false); calculateMaxAndSum(max_queue_size, sum_queue_size, status.queue.queue_size); diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index c2c12da6077..16cb45ae6f3 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1355,12 +1355,20 @@ static MutationCommand createMaterializeTTLCommand() return command; } -MutationCommands AlterCommands::getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context) const +MutationCommands AlterCommands::getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters) const { MutationCommands result; for (const auto & alter_cmd : *this) + { if (auto mutation_cmd = alter_cmd.tryConvertToMutationCommand(metadata, context); mutation_cmd) + { result.push_back(*mutation_cmd); + } + else if (with_alters) + { + result.push_back(MutationCommand{.ast = alter_cmd.ast->clone(), .type = MutationCommand::Type::ALTER_WITHOUT_MUTATION}); + } + } if (materialize_ttl) { diff --git a/src/Storages/AlterCommands.h b/src/Storages/AlterCommands.h index 71c622cb9be..c91c82e9c7a 100644 --- a/src/Storages/AlterCommands.h +++ b/src/Storages/AlterCommands.h @@ -209,7 +209,7 @@ public: /// alter. If alter can be performed as pure metadata update, than result is /// empty. If some TTL changes happened than, depending on materialize_ttl /// additional mutation command (MATERIALIZE_TTL) will be returned. - MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context) const; + MutationCommands getMutationCommands(StorageInMemoryMetadata metadata, bool materialize_ttl, ContextPtr context, bool with_alters=false) const; }; } diff --git a/src/Storages/MergeTree/ActiveDataPartSet.cpp b/src/Storages/MergeTree/ActiveDataPartSet.cpp index a482dd21099..67199ca02ac 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.cpp +++ b/src/Storages/MergeTree/ActiveDataPartSet.cpp @@ -23,8 +23,13 @@ ActiveDataPartSet::ActiveDataPartSet(MergeTreeDataFormatVersion format_version_, bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) { - /// TODO make it exception safe (out_replaced_parts->push_back(...) may throw) auto part_info = MergeTreePartInfo::fromPartName(name, format_version); + return add(part_info, name, out_replaced_parts); +} + +bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts) +{ + /// TODO make it exception safe (out_replaced_parts->push_back(...) may throw) if (getContainingPartImpl(part_info) != part_info_to_name.end()) return false; @@ -42,7 +47,7 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) if (!part_info.contains(it->first)) { if (!part_info.isDisjoint(it->first)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects previous part {}. It is a bug or a result of manual intervention in the ZooKeeper data.", name, it->first.getPartName()); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} intersects previous part {}. It is a bug or a result of manual intervention in the ZooKeeper data.", part_info.getPartName(), it->first.getPartName()); ++it; break; } @@ -69,6 +74,12 @@ bool ActiveDataPartSet::add(const String & name, Strings * out_replaced_parts) part_info_to_name.emplace(part_info, name); return true; + +} + +bool ActiveDataPartSet::add(const MergeTreePartInfo & part_info, Strings * out_replaced_parts) +{ + return add(part_info, part_info.getPartName(), out_replaced_parts); } @@ -160,6 +171,16 @@ Strings ActiveDataPartSet::getParts() const return res; } +std::vector ActiveDataPartSet::getPartInfos() const +{ + std::vector res; + res.reserve(part_info_to_name.size()); + for (const auto & kv : part_info_to_name) + res.push_back(kv.first); + + return res; +} + size_t ActiveDataPartSet::size() const { return part_info_to_name.size(); diff --git a/src/Storages/MergeTree/ActiveDataPartSet.h b/src/Storages/MergeTree/ActiveDataPartSet.h index 8ab03625d5c..f3cd6b0019d 100644 --- a/src/Storages/MergeTree/ActiveDataPartSet.h +++ b/src/Storages/MergeTree/ActiveDataPartSet.h @@ -40,6 +40,8 @@ public: /// Returns true if the part was actually added. If out_replaced_parts != nullptr, it will contain /// parts that were replaced from the set by the newly added part. bool add(const String & name, Strings * out_replaced_parts = nullptr); + bool add(const MergeTreePartInfo & part_info, const String & name, Strings * out_replaced_parts = nullptr); + bool add(const MergeTreePartInfo & part_info, Strings * out_replaced_parts = nullptr); bool remove(const MergeTreePartInfo & part_info) { @@ -83,6 +85,7 @@ public: /// Returns parts in ascending order of the partition_id and block number. Strings getParts() const; + std::vector getPartInfos() const; size_t size() const; diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 347ea16950e..741c7c6378a 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -59,9 +59,9 @@ std::string DataPartStorageOnDisk::getFullRootPath() const return fs::path(volume->getDisk()->getPath()) / root_path / ""; } -MutableDataPartStoragePtr DataPartStorageOnDisk::getProjection(const std::string & name) +MutableDataPartStoragePtr DataPartStorageOnDisk::getProjection(const std::string & name, bool use_parent_transaction) { - return std::shared_ptr(new DataPartStorageOnDisk(volume, std::string(fs::path(root_path) / part_dir), name, transaction)); + return std::shared_ptr(new DataPartStorageOnDisk(volume, std::string(fs::path(root_path) / part_dir), name, use_parent_transaction ? transaction : nullptr)); } DataPartStoragePtr DataPartStorageOnDisk::getProjection(const std::string & name) const @@ -638,12 +638,17 @@ MutableDataPartStoragePtr DataPartStorageOnDisk::clonePart( } void DataPartStorageOnDisk::rename( - const std::string & new_root_path, - const std::string & new_part_dir, + std::string new_root_path, + std::string new_part_dir, Poco::Logger * log, bool remove_new_dir_if_exists, bool fsync_part_dir) { + if (new_root_path.ends_with('/')) + new_root_path.pop_back(); + if (new_part_dir.ends_with('/')) + new_part_dir.pop_back(); + String to = fs::path(new_root_path) / new_part_dir / ""; if (volume->getDisk()->exists(to)) @@ -668,7 +673,6 @@ void DataPartStorageOnDisk::rename( fullPath(volume->getDisk(), to)); } } - String from = getRelativePath(); /// Why? diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index bea1596e1f7..fd408af9cf1 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -21,7 +21,7 @@ public: std::string getPartDirectory() const override { return part_dir; } std::string getFullRootPath() const override; - MutableDataPartStoragePtr getProjection(const std::string & name) override; + MutableDataPartStoragePtr getProjection(const std::string & name, bool use_parent_transaction = true) override; // NOLINT DataPartStoragePtr getProjection(const std::string & name) const override; bool exists() const override; @@ -123,8 +123,8 @@ public: void createHardLinkFrom(const IDataPartStorage & source, const std::string & from, const std::string & to) override; void rename( - const std::string & new_root_path, - const std::string & new_part_dir, + std::string new_root_path, + std::string new_part_dir, Poco::Logger * log, bool remove_new_dir_if_exists, bool fsync_part_dir) override; @@ -156,5 +156,4 @@ private: Poco::Logger * log, bool is_projection) const; }; - } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index bc28a555f77..ca81578c5c6 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index 53ee2738fc6..d7c0c9c76e3 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -8,6 +8,8 @@ #include #include #include +#include +#include namespace DB { @@ -85,7 +87,7 @@ public: /// virtual std::string getRelativeRootPath() const = 0; /// Get a storage for projection. - virtual std::shared_ptr getProjection(const std::string & name) = 0; + virtual std::shared_ptr getProjection(const std::string & name, bool use_parent_transaction = true) = 0; // NOLINT virtual std::shared_ptr getProjection(const std::string & name) const = 0; /// Part directory exists. @@ -237,12 +239,13 @@ public: /// Examples are: 'all_1_2_1' -> 'detached/all_1_2_1' /// 'moving/tmp_all_1_2_1' -> 'all_1_2_1' virtual void rename( - const std::string & new_root_path, - const std::string & new_part_dir, + std::string new_root_path, + std::string new_part_dir, Poco::Logger * log, bool remove_new_dir_if_exists, bool fsync_part_dir) = 0; + /// Starts a transaction of mutable operations. virtual void beginTransaction() = 0; /// Commits a transaction of mutable operations. diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 02a7a2ae641..b61cde8dbbc 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1719,7 +1719,7 @@ void IMergeTreeDataPart::renameToDetached(const String & prefix) part_is_probably_removed_from_disk = true; } -void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const +DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & /*metadata_snapshot*/) const { auto storage_settings = storage.getSettings(); @@ -1732,9 +1732,9 @@ void IMergeTreeDataPart::makeCloneInDetached(const String & prefix, const Storag bool broken = !prefix.empty(); auto maybe_path_in_detached = getRelativePathForDetachedPart(prefix, broken); if (!maybe_path_in_detached) - return; + return nullptr; - getDataPartStorage().freeze( + return getDataPartStorage().freeze( storage.relative_data_path, *maybe_path_in_detached, /*make_source_readonly*/ true, diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 963e874b2a3..2c5169a1729 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -334,7 +334,7 @@ public: virtual void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists); /// Makes clone of a part in detached/ directory via hard links - virtual void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const; + virtual DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const; /// Makes full clone of part in specified subdirectory (relative to storage data directory, e.g. "detached") on another disk MutableDataPartStoragePtr makeCloneOnDisk(const DiskPtr & disk, const String & directory_name) const; diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 0b6fe23e961..ea6ed4b403a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -96,9 +96,13 @@ static void extractMergingAndGatheringColumns( bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() { - // projection parts have different prefix and suffix compared to normal parts. - // E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge. - const String local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_"; + String local_tmp_prefix; + if (global_ctx->need_prefix) + { + // projection parts have different prefix and suffix compared to normal parts. + // E.g. `proj_a.proj` for a normal projection merge and `proj_a.tmp_proj` for a projection materialization merge. + local_tmp_prefix = global_ctx->parent_part ? "" : "tmp_merge_"; + } const String local_tmp_suffix = global_ctx->parent_part ? ctx->suffix : ""; if (global_ctx->merges_blocker->isCancelled() || global_ctx->merge_list_element_ptr->is_cancelled.load(std::memory_order_relaxed)) @@ -653,6 +657,7 @@ bool MergeTask::MergeProjectionsStage::mergeMinMaxIndexAndPrepareProjections() c global_ctx->deduplicate, global_ctx->deduplicate_by_columns, projection_merging_params, + global_ctx->need_prefix, global_ctx->new_data_part.get(), ".proj", NO_TRANSACTION_PTR, diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 6a29cdbb5ca..b8f833e1144 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -59,6 +59,7 @@ public: bool deduplicate_, Names deduplicate_by_columns_, MergeTreeData::MergingParams merging_params_, + bool need_prefix, IMergeTreeDataPart * parent_part_, String suffix_, MergeTreeTransactionPtr txn, @@ -86,6 +87,7 @@ public: global_ctx->merges_blocker = std::move(merges_blocker_); global_ctx->ttl_merges_blocker = std::move(ttl_merges_blocker_); global_ctx->txn = std::move(txn); + global_ctx->need_prefix = need_prefix; auto prepare_stage_ctx = std::make_shared(); @@ -171,6 +173,7 @@ private: IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{}; MergeTreeTransactionPtr txn; + bool need_prefix; scope_guard temporary_directory_lock; }; @@ -184,6 +187,7 @@ private: { /// Dependencies String suffix; + bool need_prefix; MergeTreeData::MergingParams merging_params{}; DiskPtr tmp_disk{nullptr}; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index e6b792e8be1..9757171b37b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -209,9 +209,86 @@ static void checkSampleExpression(const StorageInMemoryMetadata & metadata, bool ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); } + +void MergeTreeData::initializeDirectoriesAndFormatVersion(const std::string & relative_data_path_, bool attach, const std::string & date_column_name, bool need_create_directories) +{ + relative_data_path = relative_data_path_; + + MergeTreeDataFormatVersion min_format_version(0); + if (date_column_name.empty()) + min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; + + if (relative_data_path.empty()) + throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); + + const auto format_version_path = fs::path(relative_data_path) / MergeTreeData::FORMAT_VERSION_FILE_NAME; + std::optional read_format_version; + + for (const auto & disk : getDisks()) + { + if (disk->isBroken()) + continue; + + if (need_create_directories) + { + disk->createDirectories(relative_data_path); + disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); + } + + if (disk->exists(format_version_path)) + { + auto buf = disk->readFile(format_version_path); + UInt32 current_format_version{0}; + readIntText(current_format_version, *buf); + if (!buf->eof()) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Bad version file: {}", fullPath(disk, format_version_path)); + + if (!read_format_version.has_value()) + read_format_version = current_format_version; + else if (*read_format_version != current_format_version) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Version file on {} contains version {} expected version is {}.", fullPath(disk, format_version_path), current_format_version, *read_format_version); + } + } + + + // When data path or file not exists, ignore the format_version check + if (!attach || !read_format_version) + { + format_version = min_format_version; + + // try to write to first non-readonly disk + for (const auto & disk : getStoragePolicy()->getDisks()) + { + if (disk->isBroken()) + continue; + + if (!disk->isReadOnly()) + { + auto buf = disk->writeFile(format_version_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, getContext()->getWriteSettings()); + writeIntText(format_version.toUnderType(), *buf); + if (getContext()->getSettingsRef().fsync_metadata) + buf->sync(); + } + + break; + } + } + else + { + format_version = *read_format_version; + } + + if (format_version < min_format_version) + { + if (min_format_version == MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING.toUnderType()) + throw Exception( + "MergeTree data format version on disk doesn't support custom partitioning", + ErrorCodes::METADATA_MISMATCH); + } +} + MergeTreeData::MergeTreeData( const StorageID & table_id_, - const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, ContextMutablePtr context_, const String & date_column_name, @@ -222,9 +299,9 @@ MergeTreeData::MergeTreeData( BrokenPartCallback broken_part_callback_) : IStorage(table_id_) , WithMutableContext(context_->getGlobalContext()) + , format_version(date_column_name.empty() ? MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING : MERGE_TREE_DATA_OLD_FORMAT_VERSION) , merging_params(merging_params_) , require_part_metadata(require_part_metadata_) - , relative_data_path(relative_data_path_) , broken_part_callback(broken_part_callback_) , log_name(std::make_shared(table_id_.getNameForLogs())) , log(&Poco::Logger::get(*log_name)) @@ -242,14 +319,10 @@ MergeTreeData::MergeTreeData( const auto settings = getSettings(); allow_nullable_key = attach || settings->allow_nullable_key; - if (relative_data_path.empty()) - throw Exception("MergeTree storages require data path", ErrorCodes::INCORRECT_FILE_NAME); - /// Check sanity of MergeTreeSettings. Only when table is created. if (!attach) settings->sanityCheck(getContext()->getMergeMutateExecutor()->getMaxTasksCount()); - MergeTreeDataFormatVersion min_format_version(0); if (!date_column_name.empty()) { try @@ -270,7 +343,6 @@ MergeTreeData::MergeTreeData( { is_custom_partitioned = true; checkPartitionKeyAndInitMinMax(metadata_.partition_key); - min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; } setProperties(metadata_, metadata_, attach); @@ -286,64 +358,6 @@ MergeTreeData::MergeTreeData( checkTTLExpressions(metadata_, metadata_); - const auto format_version_path = fs::path(relative_data_path) / MergeTreeData::FORMAT_VERSION_FILE_NAME; - std::optional read_format_version; - /// Creating directories, if not exist. - for (const auto & disk : getDisks()) - { - if (disk->isBroken()) - continue; - - disk->createDirectories(relative_data_path); - disk->createDirectories(fs::path(relative_data_path) / MergeTreeData::DETACHED_DIR_NAME); - - if (disk->exists(format_version_path)) - { - auto buf = disk->readFile(format_version_path); - UInt32 current_format_version{0}; - readIntText(current_format_version, *buf); - if (!buf->eof()) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Bad version file: {}", fullPath(disk, format_version_path)); - - if (!read_format_version.has_value()) - read_format_version = current_format_version; - else if (*read_format_version != current_format_version) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Version file on {} contains version {} expected version is {}.", fullPath(disk, format_version_path), current_format_version, *read_format_version); - } - } - - // When data path or file not exists, ignore the format_version check - if (!attach || !read_format_version) - { - format_version = min_format_version; - - // try to write to first non-readonly disk - for (const auto & disk : getStoragePolicy()->getDisks()) - { - if (!disk->isReadOnly()) - { - auto buf = disk->writeFile(format_version_path, DBMS_DEFAULT_BUFFER_SIZE, WriteMode::Rewrite, context_->getWriteSettings()); - writeIntText(format_version.toUnderType(), *buf); - if (getContext()->getSettingsRef().fsync_metadata) - buf->sync(); - - break; - } - } - } - else - { - format_version = *read_format_version; - } - - if (format_version < min_format_version) - { - if (min_format_version == MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING.toUnderType()) - throw Exception( - "MergeTree data format version on disk doesn't support custom partitioning", - ErrorCodes::METADATA_MISMATCH); - } - String reason; if (!canUsePolymorphicParts(*settings, &reason) && !reason.empty()) LOG_WARNING(log, "{} Settings 'min_rows_for_wide_part', 'min_bytes_for_wide_part', " @@ -1819,7 +1833,7 @@ MergeTreeData::DataPartsVector MergeTreeData::grabOldParts(bool force) } if (!res.empty()) - LOG_TRACE(log, "Found {} old parts to remove. Parts {}", + LOG_TRACE(log, "Found {} old parts to remove. Parts: [{}]", res.size(), fmt::join(getPartsNames(res), ", ")); return res; @@ -1964,7 +1978,7 @@ void MergeTreeData::clearPartsFromFilesystemImpl(const DataPartsVector & parts_t { const auto settings = getSettings(); bool has_zero_copy_parts = false; - if (supportsReplication() && settings->allow_remote_fs_zero_copy_replication) + if (settings->allow_remote_fs_zero_copy_replication && dynamic_cast(this) != nullptr) { has_zero_copy_parts = std::any_of( parts_to_remove.begin(), parts_to_remove.end(), @@ -2298,7 +2312,9 @@ void MergeTreeData::dropAllData() try { - if (!disk->isDirectoryEmpty(relative_data_path) && supportsReplication() && disk->supportZeroCopyReplication() && settings_ptr->allow_remote_fs_zero_copy_replication) + if (!disk->isDirectoryEmpty(relative_data_path) && + supportsReplication() && disk->supportZeroCopyReplication() + && settings_ptr->allow_remote_fs_zero_copy_replication) { std::vector files_left; disk->listFiles(relative_data_path, files_left); @@ -3091,7 +3107,7 @@ void MergeTreeData::checkPartDuplicate(MutableDataPartPtr & part, Transaction & } } -void MergeTreeData::preparePartForCommit(MutableDataPartPtr & part, Transaction & out_transaction) +void MergeTreeData::preparePartForCommit(MutableDataPartPtr & part, Transaction & out_transaction, bool need_rename) { part->is_temp = false; part->setState(DataPartState::PreActive); @@ -3103,12 +3119,55 @@ void MergeTreeData::preparePartForCommit(MutableDataPartPtr & part, Transaction return !may_be_cleaned_up || temporary_parts.contains(dir_name); }()); - part->renameTo(part->name, true); + if (need_rename) + part->renameTo(part->name, true); data_parts_indexes.insert(part); out_transaction.addPart(part); } +bool MergeTreeData::addTempPart( + MutableDataPartPtr & part, + Transaction & out_transaction, + DataPartsLock & lock, + DataPartsVector * out_covered_parts) +{ + LOG_TRACE(log, "Adding temporary part from directory {} with name {}.", part->getDataPartStorage().getPartDirectory(), part->name); + if (&out_transaction.data != this) + throw Exception("MergeTreeData::Transaction for one table cannot be used with another. It is a bug.", + ErrorCodes::LOGICAL_ERROR); + + if (part->hasLightweightDelete()) + has_lightweight_delete_parts.store(true); + + checkPartPartition(part, lock); + checkPartDuplicate(part, out_transaction, lock); + + DataPartPtr covering_part; + DataPartsVector covered_parts = getActivePartsToReplace(part->info, part->name, covering_part, lock); + + if (covering_part) + { + LOG_WARNING(log, "Tried to add obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); + return false; + } + + /// All checks are passed. Now we can rename the part on disk. + /// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts + preparePartForCommit(part, out_transaction, /* need_rename = */false); + + if (out_covered_parts) + { + out_covered_parts->reserve(covered_parts.size()); + + for (DataPartPtr & covered_part : covered_parts) + out_covered_parts->emplace_back(std::move(covered_part)); + } + + return true; +} + + bool MergeTreeData::renameTempPartAndReplaceImpl( MutableDataPartPtr & part, Transaction & out_transaction, @@ -3150,7 +3209,7 @@ bool MergeTreeData::renameTempPartAndReplaceImpl( /// All checks are passed. Now we can rename the part on disk. /// So, we maintain invariant: if a non-temporary part in filesystem then it is in data_parts - preparePartForCommit(part, out_transaction); + preparePartForCommit(part, out_transaction, /* need_rename */ true); if (out_covered_parts) { @@ -3271,8 +3330,8 @@ void MergeTreeData::removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(txn, drop_range, lock); } -MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper( - MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock) +DataPartsVector MergeTreeData::grabActivePartsToRemoveForDropRange( + MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock) { DataPartsVector parts_to_remove; @@ -3339,6 +3398,14 @@ MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromW parts_to_remove.emplace_back(part); } + return parts_to_remove; +} + +MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper( + MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock) +{ + + auto parts_to_remove = grabActivePartsToRemoveForDropRange(txn, drop_range, lock); bool clear_without_timeout = true; /// We a going to remove active parts covered by drop_range without timeout. @@ -3915,10 +3982,25 @@ MergeTreeData::DataPartsVector MergeTreeData::getVisibleDataPartsVectorInPartiti return res; } -MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInfo & part_info, const MergeTreeData::DataPartStates & valid_states, DataPartsLock * acquired_lock) +MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInfo & part_info, const MergeTreeData::DataPartStates & valid_states) { - auto lock = (acquired_lock) ? DataPartsLock() : lockParts(); + auto lock = lockParts(); + return getPartIfExistsUnlocked(part_info, valid_states, lock); +} +MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_name, const MergeTreeData::DataPartStates & valid_states) +{ + auto lock = lockParts(); + return getPartIfExistsUnlocked(part_name, valid_states, lock); +} + +MergeTreeData::DataPartPtr MergeTreeData::getPartIfExistsUnlocked(const String & part_name, const DataPartStates & valid_states, DataPartsLock & acquired_lock) +{ + return getPartIfExistsUnlocked(MergeTreePartInfo::fromPartName(part_name, format_version), valid_states, acquired_lock); +} + +MergeTreeData::DataPartPtr MergeTreeData::getPartIfExistsUnlocked(const MergeTreePartInfo & part_info, const DataPartStates & valid_states, DataPartsLock & /* acquired_lock */) +{ auto it = data_parts_by_info.find(part_info); if (it == data_parts_by_info.end()) return nullptr; @@ -3930,12 +4012,6 @@ MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const MergeTreePartInf return nullptr; } -MergeTreeData::DataPartPtr MergeTreeData::getPartIfExists(const String & part_name, const MergeTreeData::DataPartStates & valid_states, DataPartsLock * acquired_lock) -{ - return getPartIfExists(MergeTreePartInfo::fromPartName(part_name, format_version), valid_states, acquired_lock); -} - - static void loadPartAndFixMetadataImpl(MergeTreeData::MutableDataPartPtr part) { part->loadColumnsChecksumsIndexes(false, true); @@ -4987,6 +5063,8 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const { const String containing_part = active_parts.getContainingPart(part_info.dir_name); + LOG_DEBUG(log, "Found containing part {} for part {}", containing_part, part_info.dir_name); + if (!containing_part.empty() && containing_part != part_info.dir_name) part_info.disk->moveDirectory(fs::path(relative_data_path) / source_dir / part_info.dir_name, fs::path(relative_data_path) / source_dir / ("inactive_" + part_info.dir_name)); @@ -5000,7 +5078,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const renamed_parts.tryRenameAll(); /// Synchronously check that added parts exist and are not broken. We will write checksums.txt if it does not exist. - LOG_DEBUG(log, "Checking parts"); + LOG_DEBUG(log, "Checking {} parts", renamed_parts.old_and_new_names.size()); MutableDataPartsVector loaded_parts; loaded_parts.reserve(renamed_parts.old_and_new_names.size()); @@ -6419,7 +6497,6 @@ std::pair MergeTreeData::cloneAn quoteString(src_part->getDataPartStorage().getFullPath())); String dst_part_name = src_part->getNewName(dst_part_info); - assert(!tmp_part_prefix.empty()); String tmp_dst_part_name = tmp_part_prefix + dst_part_name; auto temporary_directory_lock = getTemporaryPartDirectoryHolder(tmp_dst_part_name); @@ -6517,7 +6594,7 @@ DiskPtr MergeTreeData::tryGetDiskForDetachedPart(const String & part_name) const const auto disks = getStoragePolicy()->getDisks(); for (const DiskPtr & disk : disks) - if (disk->exists(relative_data_path + additional_path + part_name)) + if (disk->exists(fs::path(relative_data_path) / additional_path / part_name)) return disk; return nullptr; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 16e6e7aa809..670c755cf72 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -273,6 +273,7 @@ public: tryLogCurrentException("~MergeTreeData::Transaction"); } } + void clear(); TransactionID getTID() const; @@ -284,7 +285,6 @@ public: MutableDataParts precommitted_parts; MutableDataParts locked_parts; - void clear(); }; using TransactionUniquePtr = std::unique_ptr; @@ -376,7 +376,6 @@ public: /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. /// attach - whether the existing table is attached or the new table is created. MergeTreeData(const StorageID & table_id_, - const String & relative_data_path_, const StorageInMemoryMetadata & metadata_, ContextMutablePtr context_, const String & date_column_name, @@ -517,8 +516,10 @@ public: DataPartsVector getDataPartsVectorInPartitionForInternalUsage(const DataPartStates & affordable_states, const String & partition_id, DataPartsLock * acquired_lock = nullptr) const; /// Returns the part with the given name and state or nullptr if no such part. - DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states, DataPartsLock * acquired_lock = nullptr); - DataPartPtr getPartIfExists(const MergeTreePartInfo & part_info, const DataPartStates & valid_states, DataPartsLock * acquired_lock = nullptr); + DataPartPtr getPartIfExistsUnlocked(const String & part_name, const DataPartStates & valid_states, DataPartsLock & acquired_lock); + DataPartPtr getPartIfExistsUnlocked(const MergeTreePartInfo & part_info, const DataPartStates & valid_states, DataPartsLock & acquired_lock); + DataPartPtr getPartIfExists(const String & part_name, const DataPartStates & valid_states); + DataPartPtr getPartIfExists(const MergeTreePartInfo & part_info, const DataPartStates & valid_states); /// Total size of active parts in bytes. size_t getTotalActiveSizeInBytes() const; @@ -592,6 +593,8 @@ public: /// Used in REPLACE PARTITION command. void removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock); + DataPartsVector grabActivePartsToRemoveForDropRange( + MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock); /// This wrapper is required to restrict access to parts in Deleting state class PartToRemoveFromZooKeeper { @@ -975,6 +978,14 @@ public: /// If one_part is true, fill in at most one part. Block getBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, bool one_part, bool ignore_empty = false) const; + /// In merge tree we do inserts with several steps. One of them: + /// X. write part to temporary directory with some temp name + /// Y. rename temporary directory to final name with correct block number value + /// As temp name MergeTree use just ordinary in memory counter, but in some cases + /// it can be useful to add additional part in temp name to avoid collisions on FS. + /// FIXME: Currently unused. + virtual std::string getPostfixForTempInsertName() const { return ""; } + /// For generating names of temporary parts during insertion. SimpleIncrement insert_increment; @@ -1088,6 +1099,8 @@ protected: struct TagByInfo{}; struct TagByStateAndInfo{}; + void initializeDirectoriesAndFormatVersion(const std::string & relative_data_path_, bool attach, const std::string & date_column_name, bool need_create_directories=true); + static const MergeTreePartInfo & dataPartPtrToInfo(const DataPartPtr & part) { return part->info; @@ -1318,6 +1331,12 @@ protected: static void incrementInsertedPartsProfileEvent(MergeTreeDataPartType type); static void incrementMergedPartsProfileEvent(MergeTreeDataPartType type); + bool addTempPart( + MutableDataPartPtr & part, + Transaction & out_transaction, + DataPartsLock & lock, + DataPartsVector * out_covered_parts); + private: /// Checking that candidate part doesn't break invariants: correct partition void checkPartPartition(MutableDataPartPtr & part, DataPartsLock & lock) const; @@ -1325,7 +1344,7 @@ private: /// Preparing itself to be committed in memory: fill some fields inside part, add it to data_parts_indexes /// in precommitted state and to transaction - void preparePartForCommit(MutableDataPartPtr & part, Transaction & out_transaction); + void preparePartForCommit(MutableDataPartPtr & part, Transaction & out_transaction, bool need_rename); /// Low-level method for preparing parts for commit (in-memory). /// FIXME Merge MergeTreeTransaction and Transaction diff --git a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h index 1e21070dd6b..0a84f08ea71 100644 --- a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h +++ b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h @@ -8,6 +8,7 @@ namespace DB STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion) +const MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 79670c0ab27..297515e3ef5 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -523,6 +523,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( const Names & deduplicate_by_columns, const MergeTreeData::MergingParams & merging_params, const MergeTreeTransactionPtr & txn, + bool need_prefix, IMergeTreeDataPart * parent_part, const String & suffix) { @@ -537,6 +538,7 @@ MergeTaskPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( deduplicate, deduplicate_by_columns, merging_params, + need_prefix, parent_part, suffix, txn, @@ -556,7 +558,8 @@ MutateTaskPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart( ContextPtr context, const MergeTreeTransactionPtr & txn, ReservationSharedPtr space_reservation, - TableLockHolder & holder) + TableLockHolder & holder, + bool need_prefix) { return std::make_shared( future_part, @@ -570,7 +573,8 @@ MutateTaskPtr MergeTreeDataMergerMutator::mutatePartToTemporaryPart( txn, data, *this, - merges_blocker + merges_blocker, + need_prefix ); } @@ -627,7 +631,7 @@ MergeTreeData::DataPartPtr MergeTreeDataMergerMutator::renameMergedTemporaryPart + " instead of " + parts[i]->name, ErrorCodes::LOGICAL_ERROR); } - LOG_TRACE(log, "Merged {} parts: from {} to {}", parts.size(), parts.front()->name, parts.back()->name); + LOG_TRACE(log, "Merged {} parts: [{}, {}] -> []", parts.size(), parts.front()->name, parts.back()->name, new_data_part->name); return new_data_part; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 5d98f526325..013cbb8f212 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -113,6 +113,7 @@ public: const Names & deduplicate_by_columns, const MergeTreeData::MergingParams & merging_params, const MergeTreeTransactionPtr & txn, + bool need_prefix = true, IMergeTreeDataPart * parent_part = nullptr, const String & suffix = ""); @@ -126,7 +127,8 @@ public: ContextPtr context, const MergeTreeTransactionPtr & txn, ReservationSharedPtr space_reservation, - TableLockHolder & table_lock_holder); + TableLockHolder & table_lock_holder, + bool need_prefix = true); MergeTreeData::DataPartPtr renameMergedTemporaryPart( MergeTreeData::MutableDataPartPtr & new_data_part, diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index 48b1b6bab60..ac56868894f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -149,10 +149,10 @@ MutableDataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & return new_data_part_storage; } -void MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const +DataPartStoragePtr MergeTreeDataPartInMemory::makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const { String detached_path = *getRelativePathForDetachedPart(prefix, /* broken */ false); - flushToDisk(detached_path, metadata_snapshot); + return flushToDisk(detached_path, metadata_snapshot); } void MergeTreeDataPartInMemory::renameTo(const String & new_relative_path, bool /* remove_new_dir_if_exists */) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h index e58701b04a1..acb1cd8c844 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.h +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.h @@ -47,7 +47,7 @@ public: bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column.getNameInStorage()); } String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override; - void makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const override; + DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot) const override; MutableDataPartStoragePtr flushToDisk(const String & new_relative_path, const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index c50c01ea356..e314c3f2e58 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -322,8 +322,19 @@ Block MergeTreeDataWriter::mergeBlock( return block.cloneWithColumns(status.chunk.getColumns()); } -MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( - BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) + +MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) +{ + return writeTempPartImpl(block, metadata_snapshot, context, data.insert_increment.get(), /*need_tmp_prefix = */true); +} + +MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartWithoutPrefix(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, int64_t block_number, ContextPtr context) +{ + return writeTempPartImpl(block, metadata_snapshot, context, block_number, /*need_tmp_prefix = */false); +} + +MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( + BlockWithPartition & block_with_partition, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, int64_t block_number, bool need_tmp_prefix) { TemporaryPart temp_part; Block & block = block_with_partition.block; @@ -334,17 +345,12 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( if (column.type->hasDynamicSubcolumns()) column.type = block.getByName(column.name).type; - static const String TMP_PREFIX = "tmp_insert_"; - - /// This will generate unique name in scope of current server process. - Int64 temp_index = data.insert_increment.get(); - auto minmax_idx = std::make_shared(); minmax_idx->update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); - MergeTreePartition partition(std::move(block_with_partition.partition)); + MergeTreePartition partition(block_with_partition.partition); - MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), temp_index, temp_index, 0); + MergeTreePartInfo new_part_info(partition.getID(metadata_snapshot->getPartitionKey().sample_block), block_number, block_number, 0); String part_name; if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) { @@ -364,7 +370,19 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( else part_name = new_part_info.getPartName(); - String part_dir = TMP_PREFIX + part_name; + std::string part_dir; + if (need_tmp_prefix) + { + std::string temp_prefix = "tmp_insert_"; + const auto & temp_postfix = data.getPostfixForTempInsertName(); + if (!temp_postfix.empty()) + temp_prefix += temp_postfix + "_"; + part_dir = temp_prefix + part_name; + } + else + { + part_dir = part_name; + } temp_part.temporary_directory_lock = data.getTemporaryPartDirectoryHolder(part_dir); /// If we need to calculate some columns to sort. @@ -419,7 +437,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( auto data_part_storage = std::make_shared( data_part_volume, data.relative_data_path, - TMP_PREFIX + part_name); + part_dir); data_part_storage->beginTransaction(); @@ -549,7 +567,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( } auto relative_path = part_name + (is_temp ? ".tmp_proj" : ".proj"); - auto projection_part_storage = parent_part->getDataPartStorage().getProjection(relative_path); + auto projection_part_storage = parent_part->getDataPartStorage().getProjection(relative_path, !is_temp); + if (is_temp) + projection_part_storage->beginTransaction(); + auto new_data_part = data.createPart( part_name, part_type, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 2d7e19cf9d5..cbf8094f7fd 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -79,6 +79,8 @@ public: */ TemporaryPart writeTempPart(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, ContextPtr context); + TemporaryPart writeTempPartWithoutPrefix(BlockWithPartition & block, const StorageMetadataPtr & metadata_snapshot, int64_t block_number, ContextPtr context); + /// For insertion. static TemporaryPart writeProjectionPart( const MergeTreeData & data, @@ -104,6 +106,14 @@ public: const MergeTreeData::MergingParams & merging_params); private: + + TemporaryPart writeTempPartImpl( + BlockWithPartition & block, + const StorageMetadataPtr & metadata_snapshot, + ContextPtr context, + int64_t block_number, + bool need_tmp_prefix); + static TemporaryPart writeProjectionPartImpl( const String & part_name, bool is_temp, diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index a222f2a8ad8..9906ea3d02a 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -61,7 +61,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP *out << "format version: 1\n" << "create time: " << LocalDateTime(create_time) << "\n"; *out << "commands: "; - commands.writeText(*out); + commands.writeText(*out, /* with_pure_metadata_commands = */ false); *out << "\n"; if (tid.isPrehistoric()) { @@ -174,7 +174,7 @@ std::shared_ptr MergeTreeMutationEntry::backup() const out << "block number: " << block_number << "\n"; out << "commands: "; - commands.writeText(out); + commands.writeText(out, /* with_pure_metadata_commands = */ false); out << "\n"; return std::make_shared(out.str()); diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.h b/src/Storages/MergeTree/MergeTreeMutationStatus.h index acda43b9254..5f29b777293 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.h +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.h @@ -27,6 +27,9 @@ struct MergeTreeMutationStatus String latest_failed_part; time_t latest_fail_time = 0; String latest_fail_reason; + + /// FIXME: currently unused, but would be much better to report killed mutations with this flag. + bool is_killed = false; }; /// Check mutation status and throw exception in case of error during mutation diff --git a/src/Storages/MergeTree/MergeTreePartInfo.cpp b/src/Storages/MergeTree/MergeTreePartInfo.cpp index 8c518e4d17f..f537e7cb285 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.cpp +++ b/src/Storages/MergeTree/MergeTreePartInfo.cpp @@ -18,7 +18,7 @@ MergeTreePartInfo MergeTreePartInfo::fromPartName(const String & part_name, Merg if (auto part_opt = tryParsePartName(part_name, format_version)) return *part_opt; else - throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {}", part_name); + throw Exception(ErrorCodes::BAD_DATA_PART_NAME, "Unexpected part name: {} for format version: {}", part_name, format_version); } void MergeTreePartInfo::validatePartitionID(const String & partition_id, MergeTreeDataFormatVersion format_version) diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 991a8d359a8..a4cfe77e0e7 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -93,15 +93,29 @@ void MergedBlockOutputStream::Finalizer::Impl::finish() { writer.finish(sync); - for (const auto & file_name : files_to_remove_after_finish) - part->getDataPartStorage().removeFile(file_name); - for (auto & file : written_files) { file->finalize(); if (sync) file->sync(); } + + /// TODO: this code looks really stupid. It's because DiskTransaction is + /// unable to see own write operations. When we merge part with column TTL + /// and column completely outdated we first write empty column and after + /// remove it. In case of single DiskTransaction it's impossible because + /// remove operation will not see just written files. That is why we finish + /// one transaction and start new... + /// + /// FIXME: DiskTransaction should see own writes. Column TTL implementation shouldn't be so stupid... + if (!files_to_remove_after_finish.empty()) + { + part->getDataPartStorage().commitTransaction(); + part->getDataPartStorage().beginTransaction(); + } + + for (const auto & file_name : files_to_remove_after_finish) + part->getDataPartStorage().removeFile(file_name); } MergedBlockOutputStream::Finalizer::~Finalizer() diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 9e3cbb0640b..b432841d5b0 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -160,7 +160,6 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() } } - const Settings & settings = storage.getContext()->getSettingsRef(); merge_mutate_entry = storage.getContext()->getMergeList().insert( storage.getStorageID(), diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 2b186795723..ed9794945ba 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -678,7 +678,6 @@ void finalizeMutatedPart( new_data_part->minmax_idx = source_part->minmax_idx; new_data_part->modification_time = time(nullptr); new_data_part->loadProjections(false, false); - /// All information about sizes is stored in checksums. /// It doesn't make sense to touch filesystem for sizes. new_data_part->setBytesOnDisk(new_data_part->checksums.getTotalSizeOnDisk()); @@ -756,6 +755,8 @@ struct MutationContext MergeTreeData::HardlinkedFiles hardlinked_files; + bool need_prefix = true; + scope_guard temporary_directory_lock; }; @@ -862,6 +863,7 @@ public: {}, projection_merging_params, NO_TRANSACTION_PTR, + /* need_prefix */ true, ctx->new_data_part.get(), ".tmp_proj"); @@ -1024,6 +1026,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() auto tmp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num); tmp_part.finalize(); + tmp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(tmp_part.part)); } } @@ -1046,6 +1049,7 @@ bool PartMergerWriter::mutateOriginalPartAndPrepareProjections() auto temp_part = MergeTreeDataWriter::writeTempProjectionPart( *ctx->data, ctx->log, projection_block, projection, ctx->new_data_part.get(), ++block_num); temp_part.finalize(); + temp_part.part->getDataPartStorage().commitTransaction(); projection_parts[projection.name].emplace_back(std::move(temp_part.part)); } } @@ -1442,7 +1446,8 @@ MutateTask::MutateTask( const MergeTreeTransactionPtr & txn, MergeTreeData & data_, MergeTreeDataMergerMutator & mutator_, - ActionBlocker & merges_blocker_) + ActionBlocker & merges_blocker_, + bool need_prefix_) : ctx(std::make_shared()) { ctx->data = &data_; @@ -1460,6 +1465,7 @@ MutateTask::MutateTask( ctx->txn = txn; ctx->source_part = ctx->future_part->parts[0]; ctx->storage_from_source_part = std::make_shared(ctx->source_part); + ctx->need_prefix = need_prefix_; auto storage_snapshot = ctx->storage_from_source_part->getStorageSnapshot(ctx->metadata_snapshot, context_); extendObjectColumns(ctx->storage_columns, storage_snapshot->object_columns, /*with_subcolumns=*/ false); @@ -1547,7 +1553,14 @@ bool MutateTask::prepare() files_to_copy_instead_of_hardlinks.insert(IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK); LOG_TRACE(ctx->log, "Part {} doesn't change up to mutation version {}", ctx->source_part->name, ctx->future_part->part_info.mutation); - auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_clone_", ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks); + std::string prefix; + if (ctx->need_prefix) + prefix = "tmp_clone_"; + + auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks); + + part->getDataPartStorage().beginTransaction(); + ctx->temporary_directory_lock = std::move(lock); promise.set_value(std::move(part)); return false; @@ -1580,7 +1593,10 @@ bool MutateTask::prepare() /// FIXME new_data_part is not used in the case when we clone part with cloneAndLoadDataPartOnSameDisk and return false /// Is it possible to handle this case earlier? - String tmp_part_dir_name = "tmp_mut_" + ctx->future_part->name; + std::string prefix; + if (ctx->need_prefix) + prefix = "tmp_mut_"; + String tmp_part_dir_name = prefix + ctx->future_part->name; ctx->temporary_directory_lock = ctx->data->getTemporaryPartDirectoryHolder(tmp_part_dir_name); auto data_part_storage = std::make_shared( @@ -1674,7 +1690,9 @@ bool MutateTask::prepare() if (copy_checksumns) files_to_copy_instead_of_hardlinks.insert(IMergeTreeDataPart::FILE_FOR_REFERENCES_CHECK); - auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, "tmp_mut_", ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks); + auto [part, lock] = ctx->data->cloneAndLoadDataPartOnSameDisk(ctx->source_part, prefix, ctx->future_part->part_info, ctx->metadata_snapshot, ctx->txn, &ctx->hardlinked_files, false, files_to_copy_instead_of_hardlinks); + + part->getDataPartStorage().beginTransaction(); ctx->temporary_directory_lock = std::move(lock); promise.set_value(std::move(part)); return false; diff --git a/src/Storages/MergeTree/MutateTask.h b/src/Storages/MergeTree/MutateTask.h index 3df30670d7f..54ad996ad4c 100644 --- a/src/Storages/MergeTree/MutateTask.h +++ b/src/Storages/MergeTree/MutateTask.h @@ -35,7 +35,8 @@ public: const MergeTreeTransactionPtr & txn, MergeTreeData & data_, MergeTreeDataMergerMutator & mutator_, - ActionBlocker & merges_blocker_); + ActionBlocker & merges_blocker_, + bool need_prefix_); bool execute(); @@ -46,8 +47,6 @@ public: const MergeTreeData::HardlinkedFiles & getHardlinkedFiles() const; - MutableDataPartStoragePtr getBuilder() const; - private: bool prepare(); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp index 080066c1dff..1efb3f6826b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMutationEntry.cpp @@ -24,7 +24,7 @@ void ReplicatedMergeTreeMutationEntry::writeText(WriteBuffer & out) const } out << "commands: "; - commands.writeText(out); + commands.writeText(out, /* with_pure_metadata_commands = */ false); out << "\n"; out << "alter version: "; @@ -93,7 +93,7 @@ std::shared_ptr ReplicatedMergeTreeMutationEntry::backup() c } out << "commands: "; - commands.writeText(out); + commands.writeText(out, /* with_pure_metadata_commands = */ false); out << "\n"; return std::make_shared(out.str()); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 57fd6035471..1199df95b67 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include namespace Poco { class Logger; } diff --git a/src/Storages/MergeTree/ReplicatedTableStatus.h b/src/Storages/MergeTree/ReplicatedTableStatus.h new file mode 100644 index 00000000000..b9f84091e9b --- /dev/null +++ b/src/Storages/MergeTree/ReplicatedTableStatus.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** For the system table replicas. */ +struct ReplicatedTableStatus +{ + bool is_leader; + bool can_become_leader; + bool is_readonly; + bool is_session_expired; + + ReplicatedMergeTreeQueue::Status queue; + UInt32 parts_to_check; + String zookeeper_path; + String replica_name; + String replica_path; + Int32 columns_version; + UInt64 log_max_index; + UInt64 log_pointer; + UInt64 absolute_delay; + UInt8 total_replicas; + UInt8 active_replicas; + String last_queue_update_exception; + /// If the error has happened fetching the info from ZooKeeper, this field will be set. + String zookeeper_exception; + + std::unordered_map replica_is_active; +}; + +} diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index ae2abaf8ea5..46bf7debe62 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -314,76 +314,17 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// For Replicated. String zookeeper_path; String replica_name; - StorageReplicatedMergeTree::RenamingRestrictions renaming_restrictions = StorageReplicatedMergeTree::RenamingRestrictions::ALLOW_ANY; + RenamingRestrictions renaming_restrictions = RenamingRestrictions::ALLOW_ANY; bool is_on_cluster = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY; bool is_replicated_database = args.getLocalContext()->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY && DatabaseCatalog::instance().getDatabase(args.table_id.database_name)->getEngineName() == "Replicated"; - if (replicated) + /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries + bool allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; + + auto expand_macro = [&] (ASTLiteral * ast_zk_path, ASTLiteral * ast_replica_name) { - bool has_arguments = arg_num + 2 <= arg_cnt; - bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); - - ASTLiteral * ast_zk_path; - ASTLiteral * ast_replica_name; - - if (has_valid_arguments) - { - /// Get path and name from engine arguments - ast_zk_path = engine_args[arg_num]->as(); - if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) - zookeeper_path = ast_zk_path->value.safeGet(); - else - throw Exception( - "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), - ErrorCodes::BAD_ARGUMENTS); - ++arg_num; - - ast_replica_name = engine_args[arg_num]->as(); - if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) - replica_name = ast_replica_name->value.safeGet(); - else - throw Exception( - "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); - - if (replica_name.empty()) - throw Exception( - "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); - ++arg_num; - } - else if (is_extended_storage_def - && (arg_cnt == 0 - || !engine_args[arg_num]->as() - || (arg_cnt == 1 && merging_params.mode == MergeTreeData::MergingParams::Graphite))) - { - /// Try use default values if arguments are not specified. - /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. - const auto & config = args.getContext()->getConfigRef(); - zookeeper_path = StorageReplicatedMergeTree::getDefaultZooKeeperPath(config); - /// TODO maybe use hostname if {replica} is not defined? - replica_name = StorageReplicatedMergeTree::getDefaultReplicaName(config); - - /// Modify query, so default values will be written to metadata - assert(arg_num == 0); - ASTs old_args; - std::swap(engine_args, old_args); - auto path_arg = std::make_shared(zookeeper_path); - auto name_arg = std::make_shared(replica_name); - ast_zk_path = path_arg.get(); - ast_replica_name = name_arg.get(); - engine_args.emplace_back(std::move(path_arg)); - engine_args.emplace_back(std::move(name_arg)); - std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args)); - arg_num = 2; - arg_cnt += 2; - } - else - throw Exception("Expected two string literal arguments: zookeeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); - - /// Allow implicit {uuid} macros only for zookeeper_path in ON CLUSTER queries - bool allow_uuid_macro = is_on_cluster || is_replicated_database || args.query.attach; - /// Unfold {database} and {table} macro on table creation, so table can be renamed. if (!args.attach) { @@ -427,9 +368,76 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// or if one of these macros is recursively expanded from some other macro. /// Also do not allow to move table from Atomic to Ordinary database if there's {uuid} macro if (info.expanded_database || info.expanded_table) - renaming_restrictions = StorageReplicatedMergeTree::RenamingRestrictions::DO_NOT_ALLOW; + renaming_restrictions = RenamingRestrictions::DO_NOT_ALLOW; else if (info.expanded_uuid) - renaming_restrictions = StorageReplicatedMergeTree::RenamingRestrictions::ALLOW_PRESERVING_UUID; + renaming_restrictions = RenamingRestrictions::ALLOW_PRESERVING_UUID; + }; + + if (replicated) + { + bool has_arguments = arg_num + 2 <= arg_cnt; + bool has_valid_arguments = has_arguments && engine_args[arg_num]->as() && engine_args[arg_num + 1]->as(); + + ASTLiteral * ast_zk_path; + ASTLiteral * ast_replica_name; + + if (has_valid_arguments) + { + /// Get path and name from engine arguments + ast_zk_path = engine_args[arg_num]->as(); + if (ast_zk_path && ast_zk_path->value.getType() == Field::Types::String) + zookeeper_path = ast_zk_path->value.safeGet(); + else + throw Exception( + "Path in ZooKeeper must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), + ErrorCodes::BAD_ARGUMENTS); + ++arg_num; + + ast_replica_name = engine_args[arg_num]->as(); + if (ast_replica_name && ast_replica_name->value.getType() == Field::Types::String) + replica_name = ast_replica_name->value.safeGet(); + else + throw Exception( + "Replica name must be a string literal" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::BAD_ARGUMENTS); + + if (replica_name.empty()) + throw Exception( + "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); + ++arg_num; + + expand_macro(ast_zk_path, ast_replica_name); + } + else if (is_extended_storage_def + && (arg_cnt == 0 + || !engine_args[arg_num]->as() + || (arg_cnt == 1 && merging_params.mode == MergeTreeData::MergingParams::Graphite))) + { + /// Try use default values if arguments are not specified. + /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. + const auto & config = args.getContext()->getConfigRef(); + zookeeper_path = StorageReplicatedMergeTree::getDefaultZooKeeperPath(config); + /// TODO maybe use hostname if {replica} is not defined? + replica_name = StorageReplicatedMergeTree::getDefaultReplicaName(config); + + /// Modify query, so default values will be written to metadata + assert(arg_num == 0); + ASTs old_args; + std::swap(engine_args, old_args); + auto path_arg = std::make_shared(zookeeper_path); + auto name_arg = std::make_shared(replica_name); + ast_zk_path = path_arg.get(); + ast_replica_name = name_arg.get(); + + expand_macro(ast_zk_path, ast_replica_name); + + engine_args.emplace_back(std::move(path_arg)); + engine_args.emplace_back(std::move(name_arg)); + std::move(std::begin(old_args), std::end(old_args), std::back_inserter(engine_args)); + arg_num = 2; + arg_cnt += 2; + } + else + throw Exception("Expected two string literal arguments: zookeeper_path and replica_name", ErrorCodes::BAD_ARGUMENTS); } /// This merging param maybe used as part of sorting key diff --git a/src/Storages/MutationCommands.cpp b/src/Storages/MutationCommands.cpp index 28dfe488869..ffc2cfc3086 100644 --- a/src/Storages/MutationCommands.cpp +++ b/src/Storages/MutationCommands.cpp @@ -144,23 +144,32 @@ std::optional MutationCommand::parse(ASTAlterCommand * command, res.partition = command->partition; return res; } - return {}; + else + { + MutationCommand res; + res.ast = command->ptr(); + res.type = ALTER_WITHOUT_MUTATION; + return res; + } } -std::shared_ptr MutationCommands::ast() const +std::shared_ptr MutationCommands::ast(bool with_pure_metadata_commands) const { auto res = std::make_shared(); for (const MutationCommand & command : *this) - res->children.push_back(command.ast->clone()); + { + if (command.type != MutationCommand::ALTER_WITHOUT_MUTATION || with_pure_metadata_commands) + res->children.push_back(command.ast->clone()); + } return res; } -void MutationCommands::writeText(WriteBuffer & out) const +void MutationCommands::writeText(WriteBuffer & out, bool with_pure_metadata_commands) const { WriteBufferFromOwnString commands_buf; - formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true); + formatAST(*ast(with_pure_metadata_commands), commands_buf, /* hilite = */ false, /* one_line = */ true); writeEscapedString(commands_buf.str(), out); } @@ -169,9 +178,11 @@ void MutationCommands::readText(ReadBuffer & in) String commands_str; readEscapedString(commands_str, in); + ParserAlterCommandList p_alter_commands; auto commands_ast = parseQuery( p_alter_commands, commands_str.data(), commands_str.data() + commands_str.length(), "mutation commands list", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + for (const auto & child : commands_ast->children) { auto * command_ast = child->as(); @@ -182,4 +193,22 @@ void MutationCommands::readText(ReadBuffer & in) } } +std::string MutationCommands::toString() const +{ + WriteBufferFromOwnString commands_buf; + formatAST(*ast(), commands_buf, /* hilite = */ false, /* one_line = */ true); + return commands_buf.str(); +} + + +bool MutationCommands::hasNonEmptyMutationCommands() const +{ + for (const auto & command : *this) + { + if (command.type != MutationCommand::Type::EMPTY && command.type != MutationCommand::Type::ALTER_WITHOUT_MUTATION) + return true; + } + return false; +} + } diff --git a/src/Storages/MutationCommands.h b/src/Storages/MutationCommands.h index 3f8af2b4de5..aca91c16e85 100644 --- a/src/Storages/MutationCommands.h +++ b/src/Storages/MutationCommands.h @@ -37,6 +37,7 @@ struct MutationCommand MATERIALIZE_TTL, RENAME_COLUMN, MATERIALIZE_COLUMN, + ALTER_WITHOUT_MUTATION, /// pure metadata command, currently unusned }; Type type = EMPTY; @@ -72,10 +73,12 @@ struct MutationCommand class MutationCommands : public std::vector { public: - std::shared_ptr ast() const; + std::shared_ptr ast(bool with_pure_metadata_commands = false) const; - void writeText(WriteBuffer & out) const; + void writeText(WriteBuffer & out, bool with_pure_metadata_commands) const; void readText(ReadBuffer & in); + std::string toString() const; + bool hasNonEmptyMutationCommands() const; }; using MutationCommandsConstPtr = std::shared_ptr; diff --git a/src/Storages/RenamingRestrictions.h b/src/Storages/RenamingRestrictions.h new file mode 100644 index 00000000000..1b53ed0358a --- /dev/null +++ b/src/Storages/RenamingRestrictions.h @@ -0,0 +1,13 @@ +#pragma once + +namespace DB +{ + +enum RenamingRestrictions +{ + ALLOW_ANY, + ALLOW_PRESERVING_UUID, + DO_NOT_ALLOW, +}; + +} diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index f935fe2854e..2d1578d4bd0 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -91,7 +91,6 @@ StorageMergeTree::StorageMergeTree( bool has_force_restore_data_flag) : MergeTreeData( table_id_, - relative_data_path_, metadata_, context_, date_column_name, @@ -103,6 +102,8 @@ StorageMergeTree::StorageMergeTree( , writer(*this) , merger_mutator(*this, getContext()->getMergeMutateExecutor()->getMaxTasksCount()) { + initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); + loadDataParts(has_force_restore_data_flag); if (!attach && !getDataPartsForInternalUsage().empty()) @@ -1371,7 +1372,7 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(MergeTreeTransaction * txn, c /// Forcefully stop merges and make part outdated auto merge_blocker = stopMergesAndWait(); auto parts_lock = lockParts(); - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}, &parts_lock); + auto part = getPartIfExistsUnlocked(part_name, {MergeTreeDataPartState::Active}, parts_lock); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "Part {} not found, won't try to drop it.", part_name); @@ -1384,7 +1385,7 @@ MergeTreeDataPartPtr StorageMergeTree::outdatePart(MergeTreeTransaction * txn, c std::unique_lock lock(currently_processing_in_background_mutex); auto parts_lock = lockParts(); - auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}, &parts_lock); + auto part = getPartIfExistsUnlocked(part_name, {MergeTreeDataPartState::Active}, parts_lock); /// It's okay, part was already removed if (!part) return nullptr; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 99ceb1d90ae..b20bdd20c7f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -258,7 +258,6 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( bool has_force_restore_data_flag, RenamingRestrictions renaming_restrictions_) : MergeTreeData(table_id_, - relative_data_path_, metadata_, context_, date_column_name, @@ -286,6 +285,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( , replicated_fetches_throttler(std::make_shared(getSettings()->max_replicated_fetches_network_bandwidth, getContext()->getReplicatedFetchesThrottler())) , replicated_sends_throttler(std::make_shared(getSettings()->max_replicated_sends_network_bandwidth, getContext()->getReplicatedSendsThrottler())) { + initializeDirectoriesAndFormatVersion(relative_data_path_, attach, date_column_name); /// We create and deactivate all tasks for consistency. /// They all will be scheduled and activated by the restarting thread. queue_updating_task = getContext()->getSchedulePool().createTask( @@ -5051,8 +5051,9 @@ String getPartNamePossiblyFake(MergeTreeDataFormatVersion format_version, const return part_info.getPartName(); } -bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const String & partition_id, MergeTreePartInfo & part_info, - std::optional & delimiting_block_lock, bool for_replace_range) +bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition( + const String & partition_id, MergeTreePartInfo & part_info, + std::optional & delimiting_block_lock, bool for_replace_range) { /// Even if there is no data in the partition, you still need to mark the range for deletion. /// - Because before executing DETACH, tasks for downloading parts to this partition can be executed. @@ -5659,7 +5660,7 @@ bool StorageReplicatedMergeTree::tryWaitForReplicaToProcessLogEntry( } -void StorageReplicatedMergeTree::getStatus(Status & res, bool with_zk_fields) +void StorageReplicatedMergeTree::getStatus(ReplicatedTableStatus & res, bool with_zk_fields) { auto zookeeper = tryGetZooKeeper(); const auto storage_settings_ptr = getSettings(); @@ -8745,7 +8746,7 @@ void StorageReplicatedMergeTree::restoreDataFromBackup(RestorerFromBackup & rest { /// New parts could be in the replication queue but not fetched yet. /// In that case we consider the table as not empty. - StorageReplicatedMergeTree::Status status; + ReplicatedTableStatus status; getStatus(status, /* with_zk_fields = */ false); if (status.queue.inserts_in_queue) empty = false; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 67e79378b93..75f641e5a6e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -17,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -89,13 +91,6 @@ using ZooKeeperWithFaultInjectionPtr = std::shared_ptr replica_is_active; - }; - /// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK. - void getStatus(Status & res, bool with_zk_fields = true); + void getStatus(ReplicatedTableStatus & res, bool with_zk_fields = true); using LogEntriesData = std::vector; void getQueue(LogEntriesData & res, String & replica_name); @@ -879,7 +850,6 @@ private: // Create table id if needed void createTableSharedID() const; - bool checkZeroCopyLockExists(const String & part_name, const DiskPtr & disk); std::optional getZeroCopyPartPath(const String & part_name, const DiskPtr & disk); diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 0f7877a6e41..363b47d96cb 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -153,7 +153,7 @@ Pipe StorageSystemReplicas::read( for (size_t i = 0, size = col_database->size(); i < size; ++i) { - StorageReplicatedMergeTree::Status status; + ReplicatedTableStatus status; dynamic_cast( *replicated_tables [(*col_database)[i].safeGet()] From 6858f3fc01b2e45fe2213bfd528d6bdeeb3630e3 Mon Sep 17 00:00:00 2001 From: save-my-heart Date: Thu, 22 Dec 2022 23:00:54 +0800 Subject: [PATCH 096/339] add test --- tests/queries/0_stateless/02504_explain_ast_insert.reference | 4 ++++ tests/queries/0_stateless/02504_explain_ast_insert.sql | 2 ++ 2 files changed, 6 insertions(+) create mode 100644 tests/queries/0_stateless/02504_explain_ast_insert.reference create mode 100644 tests/queries/0_stateless/02504_explain_ast_insert.sql diff --git a/tests/queries/0_stateless/02504_explain_ast_insert.reference b/tests/queries/0_stateless/02504_explain_ast_insert.reference new file mode 100644 index 00000000000..1c149a0f2f4 --- /dev/null +++ b/tests/queries/0_stateless/02504_explain_ast_insert.reference @@ -0,0 +1,4 @@ +InsertQuery (children 1) + Identifier test +InsertQuery (children 1) + Identifier test diff --git a/tests/queries/0_stateless/02504_explain_ast_insert.sql b/tests/queries/0_stateless/02504_explain_ast_insert.sql new file mode 100644 index 00000000000..fc50feebaa4 --- /dev/null +++ b/tests/queries/0_stateless/02504_explain_ast_insert.sql @@ -0,0 +1,2 @@ +explain ast insert into test values balabala; +explain ast insert into test format TabSeparated balabala; \ No newline at end of file From 8d0ad4bbcdad4f7dd17a7645821bcbb080fd1ce1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 22 Dec 2022 16:14:19 +0100 Subject: [PATCH 097/339] Strange why I have to supress in .cpp --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 741c7c6378a..215d6034a53 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -59,7 +59,7 @@ std::string DataPartStorageOnDisk::getFullRootPath() const return fs::path(volume->getDisk()->getPath()) / root_path / ""; } -MutableDataPartStoragePtr DataPartStorageOnDisk::getProjection(const std::string & name, bool use_parent_transaction) +MutableDataPartStoragePtr DataPartStorageOnDisk::getProjection(const std::string & name, bool use_parent_transaction) // NOLINT { return std::shared_ptr(new DataPartStorageOnDisk(volume, std::string(fs::path(root_path) / part_dir), name, use_parent_transaction ? transaction : nullptr)); } From 501cc390f6a5e2fda37fcd64cef078882c3b1c92 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Thu, 22 Dec 2022 16:56:14 +0000 Subject: [PATCH 098/339] Prevent duplicates in column name hints. Improve formatting. --- src/IO/WriteHelpers.h | 22 ++++++++++++++++++++ src/IO/tests/gtest_WriteHelpers.cpp | 32 +++++++++++++++++++++++++++++ src/Interpreters/TreeRewriter.cpp | 11 ++++++++-- 3 files changed, 63 insertions(+), 2 deletions(-) create mode 100644 src/IO/tests/gtest_WriteHelpers.cpp diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 003e5a56958..e99182353e6 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1098,6 +1098,28 @@ inline String toString(const T & x) return buf.str(); } +template +inline String toStringWithFinalSeparator(const std::vector & x, const String & finalSep) +{ + WriteBufferFromOwnString buf; + auto size = std::ssize(x); + for (ssize_t i = 0; i < size - 2; ++i) + { + writeQuoted(x[i], buf); + writeChar(',', buf); + writeChar(' ', buf); + } + if (size >= 2) + { + writeQuoted(x[size - 2], buf); + writeText(finalSep, buf); + } + if (size >= 1) + writeQuoted(x[size - 1], buf); + + return buf.str(); +} + inline void writeNullTerminatedString(const String & s, WriteBuffer & buffer) { /// c_str is guaranteed to return zero-terminated string diff --git a/src/IO/tests/gtest_WriteHelpers.cpp b/src/IO/tests/gtest_WriteHelpers.cpp new file mode 100644 index 00000000000..b3c7062be58 --- /dev/null +++ b/src/IO/tests/gtest_WriteHelpers.cpp @@ -0,0 +1,32 @@ +#include + +#include +#include +#include + +using namespace DB; + + +TEST(WriteHelpersTest, ToStringWithFinalSeparatorTest) +{ + { + std::vector v; + EXPECT_EQ(toStringWithFinalSeparator(v, " or "), ""); + } + { + std::vector v = {"AAA"}; + EXPECT_EQ(toStringWithFinalSeparator(v, " or "), "'AAA'"); + } + { + std::vector v = {"AAA", "BBB"}; + EXPECT_EQ(toStringWithFinalSeparator(v, " or "), "'AAA' or 'BBB'"); + } + { + std::vector v = {"AAA", "BBB", "CCC"}; + EXPECT_EQ(toStringWithFinalSeparator(v, " or "), "'AAA', 'BBB' or 'CCC'"); + } + { + std::vector v = {"AAA", "BBB", "CCC", "DDD"}; + EXPECT_EQ(toStringWithFinalSeparator(v, " or "), "'AAA', 'BBB', 'CCC' or 'DDD'"); + } +} diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index e0da9e77b81..c009e8e849e 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -1243,16 +1244,22 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (storage) { std::vector hint_name{}; + std::set helper_hint_name{}; for (const auto & name : columns_context.requiredColumns()) { auto hints = storage->getHints(name); - hint_name.insert(hint_name.end(), hints.begin(), hints.end()); + for (auto& hint : hints) + { + auto res = helper_hint_name.insert(hint); + if (res.second) + hint_name.push_back(hint); + } } if (!hint_name.empty()) { ss << ", maybe you meant: "; - ss << toString(hint_name); + ss << toStringWithFinalSeparator(hint_name, " or "); } } else From 5d7b287e40a1ceef0b60fb1e14debcfbc6382481 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Thu, 22 Dec 2022 20:08:14 +0000 Subject: [PATCH 099/339] FIXUP: Simplify as per PR comment --- src/IO/WriteHelpers.h | 23 ++++++++++------------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index e99182353e6..8dbfe63be7e 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -1099,23 +1099,20 @@ inline String toString(const T & x) } template -inline String toStringWithFinalSeparator(const std::vector & x, const String & finalSep) +inline String toStringWithFinalSeparator(const std::vector & x, const String & final_sep) { WriteBufferFromOwnString buf; - auto size = std::ssize(x); - for (ssize_t i = 0; i < size - 2; ++i) + for (auto it = x.begin(); it != x.end(); ++it) { - writeQuoted(x[i], buf); - writeChar(',', buf); - writeChar(' ', buf); + if (it != x.begin()) + { + if (std::next(it) == x.end()) + writeString(final_sep, buf); + else + writeString(", ", buf); + } + writeQuoted(*it, buf); } - if (size >= 2) - { - writeQuoted(x[size - 2], buf); - writeText(finalSep, buf); - } - if (size >= 1) - writeQuoted(x[size - 1], buf); return buf.str(); } From f555048ae54ae27156cff21c561a474eef03795b Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 22 Dec 2022 21:55:39 +0000 Subject: [PATCH 100/339] Infer numbers starting from zero as strings in TSV --- src/Formats/EscapingRuleUtils.cpp | 6 ++++++ .../0_stateless/02514_tsv_zero_started_number.reference | 1 + tests/queries/0_stateless/02514_tsv_zero_started_number.sql | 2 ++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02514_tsv_zero_started_number.reference create mode 100644 tests/queries/0_stateless/02514_tsv_zero_started_number.sql diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index df9c711b432..c9736ca8cc5 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -315,6 +315,12 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation) return DataTypeFactory::instance().get("Bool"); + /// Special case when we have number that starts with 0. In TSV we don't parse such numbers, + /// see readIntTextUnsafe in ReadHelpers.h. If we see data started with 0, we can determine it + /// as a String, so parsing won't fail. + if (field[0] == '0') + return std::make_shared(); + if (auto date_type = tryInferDateOrDateTimeFromString(field, format_settings)) return date_type; diff --git a/tests/queries/0_stateless/02514_tsv_zero_started_number.reference b/tests/queries/0_stateless/02514_tsv_zero_started_number.reference new file mode 100644 index 00000000000..829ab6bc4d0 --- /dev/null +++ b/tests/queries/0_stateless/02514_tsv_zero_started_number.reference @@ -0,0 +1 @@ +Nullable(String) 0123 diff --git a/tests/queries/0_stateless/02514_tsv_zero_started_number.sql b/tests/queries/0_stateless/02514_tsv_zero_started_number.sql new file mode 100644 index 00000000000..d2058ea8f94 --- /dev/null +++ b/tests/queries/0_stateless/02514_tsv_zero_started_number.sql @@ -0,0 +1,2 @@ +select toTypeName(*), * from format(TSV, '0123'); + From d628deb2fcb9c7c3547d2be5e81bd4ab2e78f7b9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 23 Dec 2022 03:29:18 +0300 Subject: [PATCH 101/339] Update 02503_mysql_compat_utc_timestamp.sql --- tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql index fa327a38f21..d6716f272c6 100644 --- a/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql +++ b/tests/queries/0_stateless/02503_mysql_compat_utc_timestamp.sql @@ -1,2 +1,2 @@ -- PowerBI is doing this query. It should work at least somehow, not necessarily in the same way as in MySQL. -SELECT TIMEDIFF(NOW(), UTC_TIMESTAMP()); +SELECT TIMEDIFF(NOW(), UTC_TIMESTAMP()) DIV 600; From f364c28f4a583aff3b5ea6ba64d074a371539396 Mon Sep 17 00:00:00 2001 From: Denys Golotiuk <85636145+mrcrypster@users.noreply.github.com> Date: Fri, 23 Dec 2022 11:59:43 +0200 Subject: [PATCH 102/339] Added docs for randUniform Implemented in [22.10](https://clickhouse.com/docs/en/whats-new/changelog/#new-feature-2) and delivered in https://github.com/ClickHouse/ClickHouse/pull/42411 --- .../functions/random-functions.md | 43 +++++++++++++++++++ 1 file changed, 43 insertions(+) diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 2c8166116e2..f0c0d3e4802 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -76,6 +76,49 @@ These functions are available starting from 22.10. +## randUniform + +Return random number based on [continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution) in a specified range from `min` to `max`. + +**Syntax** + +``` sql +randUniform(min, max) +``` + +**Arguments** + +- `min` - `Float64` - min value of the range, +- `max` - `Float64` - max value of the range. + +**Returned value** + +- Pseudo-random number. + +Type: [Float64](/docs/en/sql-reference/data-types/float.md). + +**Example** + +Query: + +``` sql +SELECT randUniform(5.5, 10) FROM numbers(5) +``` + +Result: + +``` text +┌─randUniform(5.5, 10)─┐ +│ 8.094978491443102 │ +│ 7.3181248914450885 │ +│ 7.177741903868262 │ +│ 6.483347380953762 │ +│ 6.122286382885112 │ +└──────────────────────┘ +``` + + + ## randNormal Return random number based on [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). From b5fd23358f8df8c6addd54e2e98ddb948e056852 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 23 Dec 2022 19:09:58 +0800 Subject: [PATCH 103/339] fixed --- src/Storages/StorageS3.cpp | 59 ++++++++++++++++++++++++-------------- src/Storages/StorageS3.h | 16 ++++++++++- 2 files changed, 53 insertions(+), 22 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index d3c09d36872..a61a3d46ebc 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -552,7 +552,9 @@ StorageS3Source::StorageS3Source( , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) { reader = createReader(); - if (reader) + if (reader || + (reader.getReadMode() == StorageS3Source::ReaderHolder::ONLY_VIRTUAL_COLUMNS && + !reader.isFinishedForOnlyVirtualColumns())) reader_future = createReaderAsync(); } @@ -572,7 +574,12 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() return {}; if (only_need_virtual_columns) - return ReaderHolder{fs::path(bucket) / current_key, nullptr, nullptr, nullptr}; + return ReaderHolder{ + fs::path(bucket) / current_key, + nullptr, + nullptr, + nullptr, + StorageS3Source::ReaderHolder::ONLY_VIRTUAL_COLUMNS}; size_t object_size = info ? info->size @@ -707,33 +714,40 @@ Chunk StorageS3Source::generate() } }; - if (!reader.getPath().empty() && only_need_virtual_columns) - { - Chunk chunk; - add_virtual_column(chunk, 1); - only_need_virtual_columns = false; - return chunk; - } - while (true) { - if (!reader || isCancelled()) + if (isCancelled()) break; Chunk chunk; - if (reader->pull(chunk)) + if (reader.getReadMode() == ReaderHolder::ONLY_VIRTUAL_COLUMNS) { - UInt64 num_rows = chunk.getNumRows(); - - size_t total_size = file_iterator->getTotalSize(); - if (num_rows && total_size) + if (!reader.isFinishedForOnlyVirtualColumns()) { - updateRowsProgressApprox( - *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + add_virtual_column(chunk, 1); + reader.setFinishedForOnlyVirtualColumns(); + return chunk; } + } + else + { + if (!reader) + break; - add_virtual_column(chunk, num_rows); - return chunk; + if (reader->pull(chunk)) + { + UInt64 num_rows = chunk.getNumRows(); + + size_t total_size = file_iterator->getTotalSize(); + if (num_rows && total_size) + { + updateRowsProgressApprox( + *this, chunk, total_size, total_rows_approx_accumulated, total_rows_count_times, total_rows_approx_max); + } + + add_virtual_column(chunk, num_rows); + return chunk; + } } { @@ -742,7 +756,10 @@ Chunk StorageS3Source::generate() assert(reader_future.valid()); reader = reader_future.get(); - if (!reader) + if ((!reader && + reader.getReadMode() == ReaderHolder::ALL) || + (reader.getReadMode() == ReaderHolder::ONLY_VIRTUAL_COLUMNS && + reader.isFinishedForOnlyVirtualColumns())) break; /// Even if task is finished the thread may be not freed in pool. diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index acd5c264822..9dffd07d86a 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -165,15 +165,24 @@ private: struct ReaderHolder { public: + enum ReadMode + { + ONLY_VIRTUAL_COLUMNS, + ALL + }; + ReaderHolder( String path_, std::unique_ptr read_buf_, std::unique_ptr pipeline_, - std::unique_ptr reader_) + std::unique_ptr reader_, + ReadMode read_mode_ = ALL) : path(std::move(path_)) , read_buf(std::move(read_buf_)) , pipeline(std::move(pipeline_)) , reader(std::move(reader_)) + , is_finished_for_only_virtual_columns(path.empty() ? true : false) + , read_mode(read_mode_) { } @@ -183,12 +192,17 @@ private: PullingPipelineExecutor * operator->() { return reader.get(); } const PullingPipelineExecutor * operator->() const { return reader.get(); } const String & getPath() const { return path; } + ReadMode getReadMode() const { return read_mode; } + bool isFinishedForOnlyVirtualColumns() const { return is_finished_for_only_virtual_columns; } + void setFinishedForOnlyVirtualColumns() { is_finished_for_only_virtual_columns = true; } private: String path; std::unique_ptr read_buf; std::unique_ptr pipeline; std::unique_ptr reader; + bool is_finished_for_only_virtual_columns{false}; + ReadMode read_mode = ALL; }; ReaderHolder reader; From d0eb22a1cddada6b0a722d1168012e8a100f9dec Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 23 Dec 2022 19:25:14 +0800 Subject: [PATCH 104/339] fix --- src/Storages/StorageS3.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index a61a3d46ebc..dfb1aa9ddf0 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -553,7 +553,7 @@ StorageS3Source::StorageS3Source( { reader = createReader(); if (reader || - (reader.getReadMode() == StorageS3Source::ReaderHolder::ONLY_VIRTUAL_COLUMNS && + (reader.getReadMode() == StorageS3Source::ReaderHolder::ONLY_VIRTUAL_COLUMNS && !reader.isFinishedForOnlyVirtualColumns())) reader_future = createReaderAsync(); } @@ -758,7 +758,7 @@ Chunk StorageS3Source::generate() if ((!reader && reader.getReadMode() == ReaderHolder::ALL) || - (reader.getReadMode() == ReaderHolder::ONLY_VIRTUAL_COLUMNS && + (reader.getReadMode() == ReaderHolder::ONLY_VIRTUAL_COLUMNS && reader.isFinishedForOnlyVirtualColumns())) break; From 68aeb39892942d4206f7e8b51b753165f18ae6cd Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 23 Dec 2022 19:33:08 +0800 Subject: [PATCH 105/339] fix --- src/Storages/StorageS3.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index dfb1aa9ddf0..e477e522d65 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -552,7 +552,7 @@ StorageS3Source::StorageS3Source( , create_reader_scheduler(threadPoolCallbackRunner(create_reader_pool, "CreateS3Reader")) { reader = createReader(); - if (reader || + if (reader || (reader.getReadMode() == StorageS3Source::ReaderHolder::ONLY_VIRTUAL_COLUMNS && !reader.isFinishedForOnlyVirtualColumns())) reader_future = createReaderAsync(); @@ -756,7 +756,7 @@ Chunk StorageS3Source::generate() assert(reader_future.valid()); reader = reader_future.get(); - if ((!reader && + if ((!reader && reader.getReadMode() == ReaderHolder::ALL) || (reader.getReadMode() == ReaderHolder::ONLY_VIRTUAL_COLUMNS && reader.isFinishedForOnlyVirtualColumns())) From 2dd809e403a32142240b67becb267d7687d376da Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 23 Dec 2022 19:45:26 +0800 Subject: [PATCH 106/339] fix --- src/Storages/StorageS3.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e477e522d65..edd455a2cac 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -575,10 +575,10 @@ StorageS3Source::ReaderHolder StorageS3Source::createReader() if (only_need_virtual_columns) return ReaderHolder{ - fs::path(bucket) / current_key, - nullptr, - nullptr, - nullptr, + fs::path(bucket) / current_key, + nullptr, + nullptr, + nullptr, StorageS3Source::ReaderHolder::ONLY_VIRTUAL_COLUMNS}; size_t object_size = info From 6d0cd35cd012d4e3c580d8abb57cb1e442e26b21 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Dec 2022 13:08:59 +0100 Subject: [PATCH 107/339] Ignore exit code 1 for tar in integration tests --- tests/integration/ci-runner.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 551466cf583..487cf9b9869 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -358,10 +358,15 @@ class ClickhouseIntegrationTestsRunner: subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL "sync", shell=True ) - subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + retcode = subprocess.call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL "tar czf {} -C {} {}".format(result_path, dir, " ".join(relpaths)), shell=True, ) + # tar return 1 when the files are changed on compressing, we ignore it + if retcode in (0, 1): + return + # but even on the fatal errors it's better to retry + logging.error("Fatal error on compressing %s: %s", result_path, retcode) def _get_runner_opts(self): result = [] From 4f84becfa03959d51bdb351a3f6bc110fbefc6d5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 23 Dec 2022 13:00:23 +0000 Subject: [PATCH 108/339] Fix wrong condition for enabling asyn creading from MergeTree. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 556997b6e7a..0d8fe84f9d3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -66,7 +66,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( .read_in_order = query_info.input_order_info != nullptr, .apply_deleted_mask = context->applyDeletedMask(), .use_asynchronous_read_from_pool = settings.allow_asynchronous_read_from_io_pool_for_merge_tree - && (settings.max_streams_to_max_threads_ratio > 1 || settings.allow_asynchronous_read_from_io_pool_for_merge_tree), + && (settings.max_streams_to_max_threads_ratio > 1 || settings.max_streams_for_merge_tree_reading > 1), }; } From 123392c9961b68f1480f2585fb4510530bd90271 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Dec 2022 14:42:38 +0000 Subject: [PATCH 109/339] Fix tests --- src/Formats/EscapingRuleUtils.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index c9736ca8cc5..5a7a48f6e4c 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -315,15 +315,15 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation) return DataTypeFactory::instance().get("Bool"); + if (auto date_type = tryInferDateOrDateTimeFromString(field, format_settings)) + return date_type; + /// Special case when we have number that starts with 0. In TSV we don't parse such numbers, /// see readIntTextUnsafe in ReadHelpers.h. If we see data started with 0, we can determine it /// as a String, so parsing won't fail. - if (field[0] == '0') + if (field[0] == '0' && field.size() != 1) return std::make_shared(); - if (auto date_type = tryInferDateOrDateTimeFromString(field, format_settings)) - return date_type; - auto type = tryInferDataTypeForSingleField(field, format_settings); if (!type) return std::make_shared(); From 69fadd21933b02e09ec73e689273d6250eb73c31 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 23 Dec 2022 15:50:00 +0100 Subject: [PATCH 110/339] tests: capture dmesg in integration tests Signed-off-by: Azat Khuzhin --- tests/integration/runner | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/runner b/tests/integration/runner index f4f853e00ad..c327898b675 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -413,5 +413,11 @@ if __name__ == "__main__": subprocess.check_call(f"docker kill {' '.join(containers)}", shell=True) print(f"Containers {containers} killed") + # Avoid overlaps with previous runs + subprocess.check_call("dmesg --clear", shell=True) + print(("Running pytest container as: '" + cmd + "'.")) subprocess.check_call(cmd, shell=True) + + # Dump dmesg (to capture possible OOMs) + subprocess.check_call("dmesg -T", shell=True) From a86894cce56ccd1eb3ede31382644346e806d5fc Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Fri, 23 Dec 2022 15:38:14 +0000 Subject: [PATCH 111/339] FIXUP: PR comments --- src/Interpreters/TreeRewriter.cpp | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index c009e8e849e..7a8258b0fd9 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -1248,10 +1248,12 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select for (const auto & name : columns_context.requiredColumns()) { auto hints = storage->getHints(name); - for (auto& hint : hints) + for (const auto & hint : hints) { - auto res = helper_hint_name.insert(hint); - if (res.second) + // We want to preserve the ordering of the hints + // (as they are ordered by Levenshtein distance) + auto [_, inserted] = helper_hint_name.insert(hint); + if (inserted) hint_name.push_back(hint); } } From 168ba57ce462862110ae48c9d07e3e9e148a6e5b Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 23 Dec 2022 10:42:14 -0500 Subject: [PATCH 112/339] Update 02287_ephemeral_format_crash.sql --- tests/queries/0_stateless/02287_ephemeral_format_crash.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02287_ephemeral_format_crash.sql b/tests/queries/0_stateless/02287_ephemeral_format_crash.sql index efde057a070..466532970ab 100644 --- a/tests/queries/0_stateless/02287_ephemeral_format_crash.sql +++ b/tests/queries/0_stateless/02287_ephemeral_format_crash.sql @@ -10,4 +10,4 @@ CREATE TABLE test(a UInt8, b String EPHEMERAL test) Engine=Memory(); -- { server CREATE TABLE test(a UInt8, b String EPHEMERAL 1+2) Engine=Memory(); SHOW CREATE TABLE test; -DROP TABLE test; \ No newline at end of file +DROP TABLE test; From 7fdcf91b54aed58846fab558ab8811221c7ec986 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 23 Dec 2022 17:54:19 +0100 Subject: [PATCH 113/339] Update the ClickHouse/checkout to v1 --- .github/workflows/backport_branches.yml | 40 ++-- .github/workflows/cherry_pick.yml | 2 +- .github/workflows/docs_check.yml | 14 +- .github/workflows/docs_release.yml | 8 +- .github/workflows/jepsen.yml | 4 +- .github/workflows/master.yml | 176 ++++++++--------- .github/workflows/nightly.yml | 10 +- .github/workflows/pull_request.yml | 246 ++++++++++++------------ .github/workflows/release.yml | 4 +- .github/workflows/release_branches.yml | 106 +++++----- .github/workflows/tags_stable.yml | 2 +- .github/workflows/woboq.yml | 2 +- 12 files changed, 307 insertions(+), 307 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 4157cf446a0..cbd3bd7bec4 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -13,7 +13,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Python unit tests @@ -24,7 +24,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -40,7 +40,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -57,7 +57,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -91,7 +91,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download json reports @@ -132,7 +132,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -174,7 +174,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -216,7 +216,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -257,7 +257,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -298,7 +298,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -339,7 +339,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -381,7 +381,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -414,7 +414,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself @@ -456,7 +456,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -494,7 +494,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -533,7 +533,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -570,7 +570,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -610,7 +610,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -646,7 +646,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -676,7 +676,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Finish label diff --git a/.github/workflows/cherry_pick.yml b/.github/workflows/cherry_pick.yml index c0bae7f1ca3..065e584182b 100644 --- a/.github/workflows/cherry_pick.yml +++ b/.github/workflows/cherry_pick.yml @@ -28,7 +28,7 @@ jobs: REPO_TEAM=core EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true token: ${{secrets.ROBOT_CLICKHOUSE_COMMIT_TOKEN}} diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index ef93b4796c2..a513eb9216d 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -22,7 +22,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Labels check @@ -34,7 +34,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -51,7 +51,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -68,7 +68,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -111,7 +111,7 @@ jobs: name: changed_images path: ${{ env.TEMP_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Style Check @@ -140,7 +140,7 @@ jobs: name: changed_images path: ${{ env.TEMP_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Docs Check @@ -162,7 +162,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Finish label diff --git a/.github/workflows/docs_release.yml b/.github/workflows/docs_release.yml index 7947483c9bb..fc4b9d88c3e 100644 --- a/.github/workflows/docs_release.yml +++ b/.github/workflows/docs_release.yml @@ -24,7 +24,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -40,7 +40,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -57,7 +57,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -95,7 +95,7 @@ jobs: RCSK EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed images diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index f13d32a909e..e67df15c4d3 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -20,7 +20,7 @@ jobs: REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 @@ -49,7 +49,7 @@ jobs: # REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse # EOF # - name: Check out repository code - # uses: ClickHouse/checkout@v0 + # uses: ClickHouse/checkout@v1 # with: # clear-repository: true # fetch-depth: 0 diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 6e06d98da40..0efdb3caaad 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -13,7 +13,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Python unit tests @@ -24,7 +24,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -40,7 +40,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -57,7 +57,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -97,7 +97,7 @@ jobs: name: changed_images path: ${{ env.TEMP_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Style Check @@ -122,7 +122,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download json reports @@ -153,7 +153,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download json reports @@ -194,7 +194,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -236,7 +236,7 @@ jobs: name: changed_images path: ${{ runner.temp }}/images_path - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -277,7 +277,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -319,7 +319,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -360,7 +360,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -401,7 +401,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -442,7 +442,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -483,7 +483,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -527,7 +527,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -568,7 +568,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -609,7 +609,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -651,7 +651,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -693,7 +693,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -735,7 +735,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -777,7 +777,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -819,7 +819,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -861,7 +861,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -894,7 +894,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself @@ -940,7 +940,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -985,7 +985,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -1012,7 +1012,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Mark Commit Release Ready @@ -1040,7 +1040,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1074,7 +1074,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1110,7 +1110,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1146,7 +1146,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1180,7 +1180,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1214,7 +1214,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1250,7 +1250,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1286,7 +1286,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1322,7 +1322,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1358,7 +1358,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1394,7 +1394,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1428,7 +1428,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1464,7 +1464,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1500,7 +1500,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1536,7 +1536,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1572,7 +1572,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1608,7 +1608,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1644,7 +1644,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1681,7 +1681,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1715,7 +1715,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1749,7 +1749,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1783,7 +1783,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1817,7 +1817,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1851,7 +1851,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1885,7 +1885,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1921,7 +1921,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1958,7 +1958,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1991,7 +1991,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -2024,7 +2024,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -2057,7 +2057,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -2095,7 +2095,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2130,7 +2130,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2165,7 +2165,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2200,7 +2200,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2235,7 +2235,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2270,7 +2270,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2305,7 +2305,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2340,7 +2340,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2375,7 +2375,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -2411,7 +2411,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -2444,7 +2444,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -2477,7 +2477,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -2510,7 +2510,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -2543,7 +2543,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -2579,7 +2579,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -2612,7 +2612,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -2645,7 +2645,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -2678,7 +2678,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -2711,7 +2711,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -2749,7 +2749,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2784,7 +2784,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2819,7 +2819,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2854,7 +2854,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2889,7 +2889,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2924,7 +2924,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2959,7 +2959,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -2994,7 +2994,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -3030,7 +3030,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: SQLancer @@ -3063,7 +3063,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: SQLancer @@ -3145,7 +3145,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Finish label diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index a4cc7e1046c..415d1b8fdc4 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -17,7 +17,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -33,7 +33,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -50,7 +50,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -92,7 +92,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -127,7 +127,7 @@ jobs: CXX: clang++-15 steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # Shallow clones should be disabled for a better relevancy of analysis diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 3a9cc5aaf38..548327d1ebb 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -27,7 +27,7 @@ jobs: if: ${{ always() }} steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Labels check @@ -39,7 +39,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Python unit tests @@ -51,7 +51,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -68,7 +68,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -85,7 +85,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -129,7 +129,7 @@ jobs: name: changed_images path: ${{ env.TEMP_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Style Check @@ -154,7 +154,7 @@ jobs: CACHES_PATH=${{runner.temp}}/../ccaches EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed images @@ -184,7 +184,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download json reports @@ -215,7 +215,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download json reports @@ -256,7 +256,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # for performance artifact @@ -298,7 +298,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -339,7 +339,7 @@ jobs: name: changed_images path: ${{ runner.temp }}/images_path - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -381,7 +381,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -422,7 +422,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -463,7 +463,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -504,7 +504,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -545,7 +545,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -589,7 +589,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -630,7 +630,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -671,7 +671,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -712,7 +712,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -753,7 +753,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -794,7 +794,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -835,7 +835,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -876,7 +876,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -917,7 +917,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -949,7 +949,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself @@ -994,7 +994,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -1040,7 +1040,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -1079,7 +1079,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1115,7 +1115,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1151,7 +1151,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1187,7 +1187,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1223,7 +1223,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1257,7 +1257,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1293,7 +1293,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1329,7 +1329,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1365,7 +1365,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1401,7 +1401,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1437,7 +1437,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1473,7 +1473,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1509,7 +1509,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1545,7 +1545,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1581,7 +1581,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1617,7 +1617,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1653,7 +1653,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1689,7 +1689,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1725,7 +1725,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1759,7 +1759,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1795,7 +1795,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1831,7 +1831,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1867,7 +1867,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1903,7 +1903,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1939,7 +1939,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1975,7 +1975,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2011,7 +2011,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2047,7 +2047,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2083,7 +2083,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2119,7 +2119,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2155,7 +2155,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2191,7 +2191,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2227,7 +2227,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2263,7 +2263,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2299,7 +2299,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2335,7 +2335,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2371,7 +2371,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2407,7 +2407,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2443,7 +2443,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2479,7 +2479,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2515,7 +2515,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2551,7 +2551,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2585,7 +2585,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2619,7 +2619,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Bugfix test @@ -2667,7 +2667,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2701,7 +2701,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2735,7 +2735,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2769,7 +2769,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2803,7 +2803,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2837,7 +2837,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2871,7 +2871,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -2907,7 +2907,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -2944,7 +2944,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -2977,7 +2977,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -3010,7 +3010,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -3043,7 +3043,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -3079,7 +3079,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -3112,7 +3112,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -3145,7 +3145,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -3178,7 +3178,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -3211,7 +3211,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Fuzzer @@ -3249,7 +3249,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3284,7 +3284,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3319,7 +3319,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3354,7 +3354,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3389,7 +3389,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3424,7 +3424,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3459,7 +3459,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3494,7 +3494,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3529,7 +3529,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3564,7 +3564,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3599,7 +3599,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3634,7 +3634,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3669,7 +3669,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3704,7 +3704,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3739,7 +3739,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3774,7 +3774,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3807,7 +3807,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -3843,7 +3843,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -3876,7 +3876,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -3909,7 +3909,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -3942,7 +3942,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -3975,7 +3975,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Unit test @@ -4013,7 +4013,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4048,7 +4048,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4083,7 +4083,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4118,7 +4118,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4153,7 +4153,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4188,7 +4188,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4223,7 +4223,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4258,7 +4258,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Performance Comparison @@ -4294,7 +4294,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: SQLancer @@ -4327,7 +4327,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: SQLancer @@ -4456,7 +4456,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Finish label diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 1b5cdb93a8f..9200e5e87b8 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -20,7 +20,7 @@ jobs: REPO_COPY=${{runner.temp}}/release_packages/ClickHouse EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: # Always use the most recent script version ref: master @@ -51,7 +51,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # otherwise we will have no version info diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 3f67cb2a0e8..251087f33a5 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -16,7 +16,7 @@ jobs: runs-on: [self-hosted, style-checker-aarch64] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -32,7 +32,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Images check @@ -49,7 +49,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download changed aarch64 images @@ -83,7 +83,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Download json reports @@ -124,7 +124,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -166,7 +166,7 @@ jobs: name: changed_images path: ${{ runner.temp }}/images_path - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -207,7 +207,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -248,7 +248,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -289,7 +289,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -330,7 +330,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -371,7 +371,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -412,7 +412,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -454,7 +454,7 @@ jobs: name: changed_images path: ${{ env.IMAGES_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: true @@ -487,7 +487,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself @@ -532,7 +532,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -570,7 +570,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Report Builder @@ -597,7 +597,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Mark Commit Release Ready @@ -625,7 +625,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -659,7 +659,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -695,7 +695,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -731,7 +731,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -767,7 +767,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -803,7 +803,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -839,7 +839,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -873,7 +873,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -909,7 +909,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -945,7 +945,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -981,7 +981,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1017,7 +1017,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1053,7 +1053,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1089,7 +1089,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1126,7 +1126,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1160,7 +1160,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1194,7 +1194,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1228,7 +1228,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1262,7 +1262,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1296,7 +1296,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1330,7 +1330,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Functional test @@ -1366,7 +1366,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1403,7 +1403,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1436,7 +1436,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1469,7 +1469,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1502,7 +1502,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Stress test @@ -1540,7 +1540,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1575,7 +1575,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1610,7 +1610,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1645,7 +1645,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1680,7 +1680,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1715,7 +1715,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1750,7 +1750,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1785,7 +1785,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1820,7 +1820,7 @@ jobs: with: path: ${{ env.REPORTS_PATH }} - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Integration test @@ -1882,7 +1882,7 @@ jobs: runs-on: [self-hosted, style-checker] steps: - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true - name: Finish label diff --git a/.github/workflows/tags_stable.yml b/.github/workflows/tags_stable.yml index f577abb104c..f5b42e9c882 100644 --- a/.github/workflows/tags_stable.yml +++ b/.github/workflows/tags_stable.yml @@ -34,7 +34,7 @@ jobs: run: | echo "GITHUB_TAG=${GITHUB_REF#refs/tags/}" >> "$GITHUB_ENV" - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: ref: master fetch-depth: 0 diff --git a/.github/workflows/woboq.yml b/.github/workflows/woboq.yml index d66d1cc6c8e..363652c9f33 100644 --- a/.github/workflows/woboq.yml +++ b/.github/workflows/woboq.yml @@ -22,7 +22,7 @@ jobs: IMAGES_PATH=${{runner.temp}}/images_path EOF - name: Check out repository code - uses: ClickHouse/checkout@v0 + uses: ClickHouse/checkout@v1 with: clear-repository: true submodules: 'true' From 8dfe90a6c1a0fb0a3e16e5c484b2bea1bf973fdc Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Dec 2022 19:44:13 +0000 Subject: [PATCH 114/339] Improve json object as string inference --- src/DataTypes/transformTypesRecursively.cpp | 83 ++++++++++--------- src/Formats/SchemaInferenceUtils.cpp | 2 +- ...499_read_json_objects_as_strings.reference | 3 + .../02499_read_json_objects_as_strings.sql | 2 + 4 files changed, 52 insertions(+), 38 deletions(-) diff --git a/src/DataTypes/transformTypesRecursively.cpp b/src/DataTypes/transformTypesRecursively.cpp index da3af0beee7..9302e656010 100644 --- a/src/DataTypes/transformTypesRecursively.cpp +++ b/src/DataTypes/transformTypesRecursively.cpp @@ -8,12 +8,57 @@ namespace DB { -void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types) +TypeIndexesSet getTypesIndexes(const DataTypes & types) { TypeIndexesSet type_indexes; for (const auto & type : types) type_indexes.insert(type->getTypeId()); + return type_indexes; +} +void transformTypesRecursively(DataTypes & types, std::function transform_simple_types, std::function transform_complex_types) +{ + TypeIndexesSet type_indexes = getTypesIndexes(types); + + /// Nullable + if (type_indexes.contains(TypeIndex::Nullable)) + { + std::vector is_nullable; + is_nullable.reserve(types.size()); + DataTypes nested_types; + nested_types.reserve(types.size()); + for (const auto & type : types) + { + if (const DataTypeNullable * type_nullable = typeid_cast(type.get())) + { + is_nullable.push_back(1); + nested_types.push_back(type_nullable->getNestedType()); + } + else + { + is_nullable.push_back(0); + nested_types.push_back(type); + } + } + + transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types); + for (size_t i = 0; i != types.size(); ++i) + { + if (is_nullable[i]) + types[i] = makeNullable(nested_types[i]); + else + types[i] = nested_types[i]; + } + + if (transform_complex_types) + { + /// Some types could be changed. + type_indexes = getTypesIndexes(types); + transform_complex_types(types, type_indexes); + } + + return; + } /// Arrays if (type_indexes.contains(TypeIndex::Array)) @@ -114,42 +159,6 @@ void transformTypesRecursively(DataTypes & types, std::function is_nullable; - is_nullable.reserve(types.size()); - DataTypes nested_types; - nested_types.reserve(types.size()); - for (const auto & type : types) - { - if (const DataTypeNullable * type_nullable = typeid_cast(type.get())) - { - is_nullable.push_back(1); - nested_types.push_back(type_nullable->getNestedType()); - } - else - { - is_nullable.push_back(0); - nested_types.push_back(type); - } - } - - transformTypesRecursively(nested_types, transform_simple_types, transform_complex_types); - for (size_t i = 0; i != types.size(); ++i) - { - if (is_nullable[i]) - types[i] = makeNullable(nested_types[i]); - else - types[i] = nested_types[i]; - } - - if (transform_complex_types) - transform_complex_types(types, type_indexes); - - return; - } - transform_simple_types(types, type_indexes); } diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 29b530966a0..3f71f847a77 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -735,7 +735,7 @@ namespace if (settings.json.try_infer_objects) return std::make_shared("json", true); if (settings.json.read_objects_as_strings) - return makeNullable(std::make_shared()); + return std::make_shared(); return nullptr; } diff --git a/tests/queries/0_stateless/02499_read_json_objects_as_strings.reference b/tests/queries/0_stateless/02499_read_json_objects_as_strings.reference index 4042c1f4389..a78bd4be3e8 100644 --- a/tests/queries/0_stateless/02499_read_json_objects_as_strings.reference +++ b/tests/queries/0_stateless/02499_read_json_objects_as_strings.reference @@ -1,3 +1,6 @@ x Nullable(String) abc {"a" : 10, "b" : "abc"} +x Nullable(String) +{"a" : "b"} +{"a" : 1, "b" : [1,2,3]} diff --git a/tests/queries/0_stateless/02499_read_json_objects_as_strings.sql b/tests/queries/0_stateless/02499_read_json_objects_as_strings.sql index bb4c9e9da0f..12d709bdde1 100644 --- a/tests/queries/0_stateless/02499_read_json_objects_as_strings.sql +++ b/tests/queries/0_stateless/02499_read_json_objects_as_strings.sql @@ -2,3 +2,5 @@ set input_format_json_read_objects_as_strings=1; desc format(JSONEachRow, '{"x" : "abc"}, {"x" : {"a" : 10, "b" : "abc"}}'); select * from format(JSONEachRow, '{"x" : "abc"}, {"x" : {"a" : 10, "b" : "abc"}}'); +desc format(JSONEachRow, '{"x" : {"a" : "b"}}, {"x" : {"a" : 1, "b" : [1,2,3]}}'); +select * from format(JSONEachRow, '{"x" : {"a" : "b"}}, {"x" : {"a" : 1, "b" : [1,2,3]}}'); From f15bf1839ad7203677b88f8ae1134651ab97eb09 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Dec 2022 19:52:54 +0000 Subject: [PATCH 115/339] Add missed settings into additional cache info --- src/Formats/EscapingRuleUtils.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index df9c711b432..3aeb94ad35a 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -424,6 +424,8 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, try_infer_objects={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, + settings.json.read_objects_as_strings, + settings.json.read_numbers_as_strings, settings.json.try_infer_objects); break; default: From 331f4bfee16b11f13c37a5a4068e5390b778bb96 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Dec 2022 19:58:50 +0000 Subject: [PATCH 116/339] Fix --- src/Formats/EscapingRuleUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 3aeb94ad35a..42555968b44 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -421,7 +421,7 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo break; case FormatSettings::EscapingRule::JSON: result += fmt::format( - ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, try_infer_objects={}", + ", try_infer_numbers_from_strings={}, read_bools_as_numbers={}, read_objects_as_strings={}, read_numbers_as_strings={}, try_infer_objects={}", settings.json.try_infer_numbers_from_strings, settings.json.read_bools_as_numbers, settings.json.read_objects_as_strings, From 835412af8d3d5b795b56b6b697e8566ead40858c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 23 Dec 2022 21:37:09 +0000 Subject: [PATCH 117/339] Fix --- src/DataTypes/transformTypesRecursively.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/DataTypes/transformTypesRecursively.cpp b/src/DataTypes/transformTypesRecursively.cpp index 9302e656010..fd97254c7ef 100644 --- a/src/DataTypes/transformTypesRecursively.cpp +++ b/src/DataTypes/transformTypesRecursively.cpp @@ -44,7 +44,8 @@ void transformTypesRecursively(DataTypes & types, std::functioncanBeInsideNullable()) types[i] = makeNullable(nested_types[i]); else types[i] = nested_types[i]; From 798c3111ed0e0b961d1618e00e2ac9c0aea56100 Mon Sep 17 00:00:00 2001 From: avogar Date: Sat, 24 Dec 2022 00:21:27 +0000 Subject: [PATCH 118/339] Improve inferring arrays with nulls in JSON formats --- src/Formats/SchemaInferenceUtils.cpp | 45 +++++++++++++++---- ..._tuple_to_array_schema_inference.reference | 3 ++ ...6_json_tuple_to_array_schema_inference.sql | 4 ++ 3 files changed, 44 insertions(+), 8 deletions(-) create mode 100644 tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference create mode 100644 tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.sql diff --git a/src/Formats/SchemaInferenceUtils.cpp b/src/Formats/SchemaInferenceUtils.cpp index 29b530966a0..9d40ac98964 100644 --- a/src/Formats/SchemaInferenceUtils.cpp +++ b/src/Formats/SchemaInferenceUtils.cpp @@ -250,7 +250,7 @@ namespace { if (isArray(type)) nested_types.push_back(assert_cast(*type).getNestedType()); - else + else if (isTuple(type)) { const auto & elements = assert_cast(*type).getElements(); for (const auto & element : elements) @@ -262,7 +262,10 @@ namespace if (checkIfTypesAreEqual(nested_types)) { for (auto & type : data_types) - type = std::make_shared(nested_types.back()); + { + if (isArray(type) || isTuple(type)) + type = std::make_shared(nested_types.back()); + } } } @@ -826,14 +829,40 @@ void transformInferredJSONTypesIfNeeded( void transformJSONTupleToArrayIfPossible(DataTypePtr & data_type, const FormatSettings & settings, JSONInferenceInfo * json_info) { - if (!data_type || !isTuple(data_type)) + if (!data_type) return; - const auto * tuple_type = assert_cast(data_type.get()); - auto nested_types = tuple_type->getElements(); - transformInferredTypesIfNeededImpl(nested_types, settings, json_info); - if (checkIfTypesAreEqual(nested_types)) - data_type = std::make_shared(nested_types.back()); + if (const auto * array_type = typeid_cast(data_type.get())) + { + auto nested_type = array_type->getNestedType(); + transformJSONTupleToArrayIfPossible(nested_type, settings, json_info); + data_type = std::make_shared(nested_type); + return; + } + + if (const auto * map_type = typeid_cast(data_type.get())) + { + auto value_type = map_type->getValueType(); + transformJSONTupleToArrayIfPossible(value_type, settings, json_info); + data_type = std::make_shared(map_type->getKeyType(), value_type); + return; + } + + if (const auto * tuple_type = typeid_cast(data_type.get())) + { + auto nested_types = tuple_type->getElements(); + for (auto & nested_type : nested_types) + transformJSONTupleToArrayIfPossible(nested_type, settings, json_info); + + auto nested_types_copy = nested_types; + transformInferredTypesIfNeededImpl(nested_types_copy, settings, json_info); + if (checkIfTypesAreEqual(nested_types_copy)) + data_type = std::make_shared(nested_types_copy.back()); + else + data_type = std::make_shared(nested_types); + + return; + } } DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSettings & settings) diff --git a/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference new file mode 100644 index 00000000000..f44e051e6bf --- /dev/null +++ b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.reference @@ -0,0 +1,3 @@ +x Array(Array(Nullable(Int64))) +x Tuple(Array(Array(Nullable(Int64))), Nullable(Int64)) +x Map(String, Array(Nullable(Int64))) diff --git a/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.sql b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.sql new file mode 100644 index 00000000000..ae3142f0b21 --- /dev/null +++ b/tests/queries/0_stateless/02416_json_tuple_to_array_schema_inference.sql @@ -0,0 +1,4 @@ +desc format(JSONEachRow, '{"x" : [[42, null], [24, null]]}'); +desc format(JSONEachRow, '{"x" : [[[42, null], []], 24]}'); +desc format(JSONEachRow, '{"x" : {"key" : [42, null]}}'); + From 54e66d875c1d31933990f95ca7c204244996cf79 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 23 Dec 2022 19:33:10 -0500 Subject: [PATCH 119/339] move title to frontatter --- docs/en/interfaces/cli.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 4f07f99fb26..e3b40d83efe 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -2,11 +2,10 @@ slug: /en/interfaces/cli sidebar_position: 17 sidebar_label: Command-Line Client +title: Command-Line Client --- import ConnectionDetails from '@site/docs/en/_snippets/_gather_your_details_native.md'; -# Command-line Client - ## clickhouse-client ClickHouse provides a native command-line client: `clickhouse-client`. The client supports command-line options and configuration files. For more information, see [Configuring](#interfaces_cli_configuration). From f66a0c01ad058b60e1f181f6e096afad40eb5957 Mon Sep 17 00:00:00 2001 From: mayamika Date: Thu, 22 Dec 2022 00:36:41 +0300 Subject: [PATCH 120/339] Add null dictionary source --- src/Dictionaries/NullDictionarySource.cpp | 48 +++++++++++++++++ src/Dictionaries/NullDictionarySource.h | 53 +++++++++++++++++++ src/Dictionaries/registerDictionaries.cpp | 2 + .../02514_null_dictionary_source.reference | 4 ++ .../02514_null_dictionary_source.sql | 48 +++++++++++++++++ 5 files changed, 155 insertions(+) create mode 100644 src/Dictionaries/NullDictionarySource.cpp create mode 100644 src/Dictionaries/NullDictionarySource.h create mode 100644 tests/queries/0_stateless/02514_null_dictionary_source.reference create mode 100644 tests/queries/0_stateless/02514_null_dictionary_source.sql diff --git a/src/Dictionaries/NullDictionarySource.cpp b/src/Dictionaries/NullDictionarySource.cpp new file mode 100644 index 00000000000..45dcc77f93d --- /dev/null +++ b/src/Dictionaries/NullDictionarySource.cpp @@ -0,0 +1,48 @@ +#include "NullDictionarySource.h" +#include +#include +#include +#include "DictionarySourceFactory.h" +#include "DictionarySourceHelpers.h" +#include "DictionaryStructure.h" +#include "registerDictionaries.h" + + +namespace DB +{ +NullDictionarySource::NullDictionarySource(Block & sample_block_) : sample_block(sample_block_) +{ +} + +NullDictionarySource::NullDictionarySource(const NullDictionarySource & other) : sample_block(other.sample_block) +{ +} + +QueryPipeline NullDictionarySource::loadAll() +{ + LOG_TRACE(&Poco::Logger::get("NullDictionarySource"), "loadAll {}", toString()); + return QueryPipeline(std::make_shared(sample_block)); +} + + +std::string NullDictionarySource::toString() const +{ + return "Null"; +} + + +void registerDictionarySourceNull(DictionarySourceFactory & factory) +{ + auto create_table_source + = [=](const DictionaryStructure & /* dict_struct */, + const Poco::Util::AbstractConfiguration & /* config */, + const std::string & /* config_prefix */, + Block & sample_block, + ContextPtr /* global_context */, + const std::string & /* default_database */, + bool /* created_from_ddl*/) -> DictionarySourcePtr { return std::make_unique(sample_block); }; + + factory.registerSource("null", create_table_source); +} + +} diff --git a/src/Dictionaries/NullDictionarySource.h b/src/Dictionaries/NullDictionarySource.h new file mode 100644 index 00000000000..7eb02055e3a --- /dev/null +++ b/src/Dictionaries/NullDictionarySource.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include "IDictionarySource.h" + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +/// Allows creating empty dictionary +class NullDictionarySource final : public IDictionarySource +{ +public: + NullDictionarySource(Block & sample_block_); + + NullDictionarySource(const NullDictionarySource & other); + + QueryPipeline loadAll() override; + + QueryPipeline loadUpdatedAll() override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for NullDictionarySource"); + } + + QueryPipeline loadIds(const std::vector & /*ids*/) override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadIds is unsupported for NullDictionarySource"); + } + + QueryPipeline loadKeys(const Columns & /*key_columns*/, const std::vector & /*requested_rows*/) override + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadKeys is unsupported for NullDictionarySource"); + } + + bool isModified() const override { return false; } + + bool supportsSelectiveLoad() const override { return false; } + + ///Not supported for NullDictionarySource + bool hasUpdateField() const override { return false; } + + DictionarySourcePtr clone() const override { return std::make_shared(*this); } + + std::string toString() const override; + +private: + Block sample_block; +}; + +} diff --git a/src/Dictionaries/registerDictionaries.cpp b/src/Dictionaries/registerDictionaries.cpp index 4ade5d88bd4..f0526f4ce37 100644 --- a/src/Dictionaries/registerDictionaries.cpp +++ b/src/Dictionaries/registerDictionaries.cpp @@ -6,6 +6,7 @@ namespace DB class DictionarySourceFactory; +void registerDictionarySourceNull(DictionarySourceFactory & factory); void registerDictionarySourceFile(DictionarySourceFactory & source_factory); void registerDictionarySourceMysql(DictionarySourceFactory & source_factory); void registerDictionarySourceClickHouse(DictionarySourceFactory & source_factory); @@ -36,6 +37,7 @@ void registerDictionaries() { { auto & source_factory = DictionarySourceFactory::instance(); + registerDictionarySourceNull(source_factory); registerDictionarySourceFile(source_factory); registerDictionarySourceMysql(source_factory); registerDictionarySourceClickHouse(source_factory); diff --git a/tests/queries/0_stateless/02514_null_dictionary_source.reference b/tests/queries/0_stateless/02514_null_dictionary_source.reference new file mode 100644 index 00000000000..bb08ece2bcf --- /dev/null +++ b/tests/queries/0_stateless/02514_null_dictionary_source.reference @@ -0,0 +1,4 @@ +0 \N 111 0 111 +123 \N 111 123 111 +\N \N 111 +77 diff --git a/tests/queries/0_stateless/02514_null_dictionary_source.sql b/tests/queries/0_stateless/02514_null_dictionary_source.sql new file mode 100644 index 00000000000..74fb57707ff --- /dev/null +++ b/tests/queries/0_stateless/02514_null_dictionary_source.sql @@ -0,0 +1,48 @@ +-- Tags: no-parallel + +DROP DICTIONARY IF EXISTS null_dict; +CREATE DICTIONARY null_dict ( + id UInt64, + val UInt8, + default_val UInt8 DEFAULT 123, + nullable_val Nullable(UInt8) +) +PRIMARY KEY id +SOURCE(NULL()) +LAYOUT(FLAT()) +LIFETIME(0); + +SELECT + dictGet('null_dict', 'val', 1337), + dictGetOrNull('null_dict', 'val', 1337), + dictGetOrDefault('null_dict', 'val', 1337, 111), + dictGetUInt8('null_dict', 'val', 1337), + dictGetUInt8OrDefault('null_dict', 'val', 1337, 111); + +SELECT + dictGet('null_dict', 'default_val', 1337), + dictGetOrNull('null_dict', 'default_val', 1337), + dictGetOrDefault('null_dict', 'default_val', 1337, 111), + dictGetUInt8('null_dict', 'default_val', 1337), + dictGetUInt8OrDefault('null_dict', 'default_val', 1337, 111); + +SELECT + dictGet('null_dict', 'nullable_val', 1337), + dictGetOrNull('null_dict', 'nullable_val', 1337), + dictGetOrDefault('null_dict', 'nullable_val', 1337, 111); + +SELECT val, nullable_val FROM null_dict; + +DROP DICTIONARY IF EXISTS null_ip_dict; +CREATE DICTIONARY null_ip_dict ( + network String, + val UInt8 DEFAULT 77 +) +PRIMARY KEY network +SOURCE(NULL()) +LAYOUT(IP_TRIE()) +LIFETIME(0); + +SELECT dictGet('null_ip_dict', 'val', toIPv4('127.0.0.1')); + +SELECT network, val FROM null_ip_dict; From bd19c6b699029e257c45dec921ae4b36e1913980 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 24 Dec 2022 17:46:22 +0100 Subject: [PATCH 121/339] Make BC check optional (if env var set) --- docker/test/stress/run.sh | 412 +++++++++++++++++++------------------- 1 file changed, 207 insertions(+), 205 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 01e0f5b4897..d948a246275 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -334,219 +334,221 @@ zgrep -Fa "########################################" /test_output/* > /dev/null zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \ && echo -e 'Found signal in gdb.log\tFAIL' >> /test_output/test_results.tsv -echo -e "Backward compatibility check\n" +if [ -z "$DISABLE_BC_CHECK" ]; then + echo -e "Backward compatibility check\n" -echo "Get previous release tag" -previous_release_tag=$(clickhouse-client --version | grep -o "[0-9]*\.[0-9]*\.[0-9]*\.[0-9]*" | get_previous_release_tag) -echo $previous_release_tag + echo "Get previous release tag" + previous_release_tag=$(clickhouse-client --version | grep -o "[0-9]*\.[0-9]*\.[0-9]*\.[0-9]*" | get_previous_release_tag) + echo $previous_release_tag -echo "Clone previous release repository" -git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository + echo "Clone previous release repository" + git clone https://github.com/ClickHouse/ClickHouse.git --no-tags --progress --branch=$previous_release_tag --no-recurse-submodules --depth=1 previous_release_repository -echo "Download previous release server" -mkdir previous_release_package_folder + echo "Download previous release server" + mkdir previous_release_package_folder -echo $previous_release_tag | download_release_packets && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \ - || echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv + echo $previous_release_tag | download_release_packets && echo -e 'Download script exit code\tOK' >> /test_output/test_results.tsv \ + || echo -e 'Download script failed\tFAIL' >> /test_output/test_results.tsv -mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log -for table in query_log trace_log -do - clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz ||: -done - -tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: - -# Check if we cloned previous release repository successfully -if ! [ "$(ls -A previous_release_repository/tests/queries)" ] -then - echo -e "Backward compatibility check: Failed to clone previous release tests\tFAIL" >> /test_output/test_results.tsv -elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] -then - echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv -else - echo -e "Successfully cloned previous release tests\tOK" >> /test_output/test_results.tsv - echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/test_results.tsv - - # Uninstall current packages - dpkg --remove clickhouse-client - dpkg --remove clickhouse-server - dpkg --remove clickhouse-common-static-dbg - dpkg --remove clickhouse-common-static - - rm -rf /var/lib/clickhouse/* - - # Make BC check more funny by forcing Ordinary engine for system database - mkdir /var/lib/clickhouse/metadata - echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql - - # Install previous release packages - install_packages previous_release_package_folder - - # Start server from previous release - # Previous version may not be ready for fault injections - export ZOOKEEPER_FAULT_INJECTION=0 - configure - - # Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..." - rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||: - rm -f /etc/clickhouse-server/users.d/marks.xml ||: - - # Remove s3 related configs to avoid "there is no disk type `cache`" - rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||: - rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||: - - # Turn on after 22.12 - rm -f /etc/clickhouse-server/config.d/compressed_marks_and_index.xml ||: - # it uses recently introduced settings which previous versions may not have - rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||: - - start - - clickhouse-client --query="SELECT 'Server version: ', version()" - - # Install new package before running stress test because we should use new - # clickhouse-client and new clickhouse-test. - # - # But we should leave old binary in /usr/bin/ and debug symbols in - # /usr/lib/debug/usr/bin (if any) for gdb and internal DWARF parser, so it - # will print sane stacktraces and also to avoid possible crashes. - # - # FIXME: those files can be extracted directly from debian package, but - # actually better solution will be to use different PATH instead of playing - # games with files from packages. - mv /usr/bin/clickhouse previous_release_package_folder/ - mv /usr/lib/debug/usr/bin/clickhouse.debug previous_release_package_folder/ - install_packages package_folder - mv /usr/bin/clickhouse package_folder/ - mv /usr/lib/debug/usr/bin/clickhouse.debug package_folder/ - mv previous_release_package_folder/clickhouse /usr/bin/ - mv previous_release_package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug - - mkdir tmp_stress_output - - ./stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \ - && echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv - rm -rf tmp_stress_output - - clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" - - stop 1 - mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log - - # Start new server - mv package_folder/clickhouse /usr/bin/ - mv package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug - export ZOOKEEPER_FAULT_INJECTION=1 - configure - start 500 - clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \ - || (echo -e 'Backward compatibility check: Server failed to start\tFAIL' >> /test_output/test_results.tsv \ - && grep -a ".*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt) - - clickhouse-client --query="SELECT 'Server version: ', version()" - - # Let the server run for a while before checking log. - sleep 60 - - stop - mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.clean.log - - # Error messages (we should ignore some errors) - # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.") - # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") - # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. - # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") - # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'") - # NOTE Incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/39263, it's expected - # ("This engine is deprecated and is not supported in transactions", "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part") - # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 - bad mutation does not indicate backward incompatibility - echo "Check for Error messages in server log:" - zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ - -e "Code: 236. DB::Exception: Cancelled mutating parts" \ - -e "REPLICA_IS_ALREADY_ACTIVE" \ - -e "REPLICA_ALREADY_EXISTS" \ - -e "ALL_REPLICAS_LOST" \ - -e "DDLWorker: Cannot parse DDL task query" \ - -e "RaftInstance: failed to accept a rpc connection due to error 125" \ - -e "UNKNOWN_DATABASE" \ - -e "NETWORK_ERROR" \ - -e "UNKNOWN_TABLE" \ - -e "ZooKeeperClient" \ - -e "KEEPER_EXCEPTION" \ - -e "DirectoryMonitor" \ - -e "TABLE_IS_READ_ONLY" \ - -e "Code: 1000, e.code() = 111, Connection refused" \ - -e "UNFINISHED" \ - -e "NETLINK_ERROR" \ - -e "Renaming unexpected part" \ - -e "PART_IS_TEMPORARILY_LOCKED" \ - -e "and a merge is impossible: we didn't find" \ - -e "found in queue and some source parts for it was lost" \ - -e "is lost forever." \ - -e "Unknown index: idx." \ - -e "Cannot parse string 'Hello' as UInt64" \ - -e "} TCPHandler: Code:" \ - -e "} executeQuery: Code:" \ - -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \ - -e "This engine is deprecated and is not supported in transactions" \ - -e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \ - -e "The set of parts restored in place of" \ - -e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \ - -e "Code: 269. DB::Exception: Destination table is myself" \ - -e "Coordination::Exception: Connection loss" \ - -e "MutateFromLogEntryTask" \ - -e "No connection to ZooKeeper, cannot get shared table ID" \ - -e "Session expired" \ - /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ - && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv - - # Remove file bc_check_error_messages.txt if it's empty - [ -s /test_output/bc_check_error_messages.txt ] || rm /test_output/bc_check_error_messages.txt - - # Sanitizer asserts - zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp - zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp - zgrep -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \ - && echo -e 'Backward compatibility check: Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: No sanitizer asserts\tOK' >> /test_output/test_results.tsv - rm -f /test_output/tmp - - # OOM - zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ - && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv - - # Logical errors - echo "Check for Logical errors in server log:" - zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \ - && echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv - - # Remove file bc_check_logical_errors.txt if it's empty - [ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt - - # Crash - zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ - && echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv - - # It also checks for crash without stacktrace (printed by watchdog) - echo "Check for Fatal message in server log:" - zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ - && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ - || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv - - # Remove file bc_check_fatal_messages.txt if it's empty - [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt - - tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||: + mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.clean.log for table in query_log trace_log do - clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.backward.tsv.gz ||: + clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.tsv.gz ||: done + + tar -chf /test_output/coordination.tar /var/lib/clickhouse/coordination ||: + + # Check if we cloned previous release repository successfully + if ! [ "$(ls -A previous_release_repository/tests/queries)" ] + then + echo -e "Backward compatibility check: Failed to clone previous release tests\tFAIL" >> /test_output/test_results.tsv + elif ! [ "$(ls -A previous_release_package_folder/clickhouse-common-static_*.deb && ls -A previous_release_package_folder/clickhouse-server_*.deb)" ] + then + echo -e "Backward compatibility check: Failed to download previous release packets\tFAIL" >> /test_output/test_results.tsv + else + echo -e "Successfully cloned previous release tests\tOK" >> /test_output/test_results.tsv + echo -e "Successfully downloaded previous release packets\tOK" >> /test_output/test_results.tsv + + # Uninstall current packages + dpkg --remove clickhouse-client + dpkg --remove clickhouse-server + dpkg --remove clickhouse-common-static-dbg + dpkg --remove clickhouse-common-static + + rm -rf /var/lib/clickhouse/* + + # Make BC check more funny by forcing Ordinary engine for system database + mkdir /var/lib/clickhouse/metadata + echo "ATTACH DATABASE system ENGINE=Ordinary" > /var/lib/clickhouse/metadata/system.sql + + # Install previous release packages + install_packages previous_release_package_folder + + # Start server from previous release + # Previous version may not be ready for fault injections + export ZOOKEEPER_FAULT_INJECTION=0 + configure + + # Avoid "Setting s3_check_objects_after_upload is neither a builtin setting..." + rm -f /etc/clickhouse-server/users.d/enable_blobs_check.xml ||: + rm -f /etc/clickhouse-server/users.d/marks.xml ||: + + # Remove s3 related configs to avoid "there is no disk type `cache`" + rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||: + rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||: + + # Turn on after 22.12 + rm -f /etc/clickhouse-server/config.d/compressed_marks_and_index.xml ||: + # it uses recently introduced settings which previous versions may not have + rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||: + + start + + clickhouse-client --query="SELECT 'Server version: ', version()" + + # Install new package before running stress test because we should use new + # clickhouse-client and new clickhouse-test. + # + # But we should leave old binary in /usr/bin/ and debug symbols in + # /usr/lib/debug/usr/bin (if any) for gdb and internal DWARF parser, so it + # will print sane stacktraces and also to avoid possible crashes. + # + # FIXME: those files can be extracted directly from debian package, but + # actually better solution will be to use different PATH instead of playing + # games with files from packages. + mv /usr/bin/clickhouse previous_release_package_folder/ + mv /usr/lib/debug/usr/bin/clickhouse.debug previous_release_package_folder/ + install_packages package_folder + mv /usr/bin/clickhouse package_folder/ + mv /usr/lib/debug/usr/bin/clickhouse.debug package_folder/ + mv previous_release_package_folder/clickhouse /usr/bin/ + mv previous_release_package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug + + mkdir tmp_stress_output + + ./stress --test-cmd="/usr/bin/clickhouse-test --queries=\"previous_release_repository/tests/queries\"" --backward-compatibility-check --output-folder tmp_stress_output --global-time-limit=1200 \ + && echo -e 'Backward compatibility check: Test script exit code\tOK' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: Test script failed\tFAIL' >> /test_output/test_results.tsv + rm -rf tmp_stress_output + + clickhouse-client --query="SELECT 'Tables count:', count() FROM system.tables" + + stop 1 + mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.stress.log + + # Start new server + mv package_folder/clickhouse /usr/bin/ + mv package_folder/clickhouse.debug /usr/lib/debug/usr/bin/clickhouse.debug + export ZOOKEEPER_FAULT_INJECTION=1 + configure + start 500 + clickhouse-client --query "SELECT 'Backward compatibility check: Server successfully started', 'OK'" >> /test_output/test_results.tsv \ + || (echo -e 'Backward compatibility check: Server failed to start\tFAIL' >> /test_output/test_results.tsv \ + && grep -a ".*Application" /var/log/clickhouse-server/clickhouse-server.log >> /test_output/bc_check_application_errors.txt) + + clickhouse-client --query="SELECT 'Server version: ', version()" + + # Let the server run for a while before checking log. + sleep 60 + + stop + mv /var/log/clickhouse-server/clickhouse-server.log /var/log/clickhouse-server/clickhouse-server.backward.clean.log + + # Error messages (we should ignore some errors) + # FIXME https://github.com/ClickHouse/ClickHouse/issues/38643 ("Unknown index: idx.") + # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 ("Cannot parse string 'Hello' as UInt64") + # FIXME Not sure if it's expected, but some tests from BC check may not be finished yet when we restarting server. + # Let's just ignore all errors from queries ("} TCPHandler: Code:", "} executeQuery: Code:") + # FIXME https://github.com/ClickHouse/ClickHouse/issues/39197 ("Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'") + # NOTE Incompatibility was introduced in https://github.com/ClickHouse/ClickHouse/pull/39263, it's expected + # ("This engine is deprecated and is not supported in transactions", "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part") + # FIXME https://github.com/ClickHouse/ClickHouse/issues/39174 - bad mutation does not indicate backward incompatibility + echo "Check for Error messages in server log:" + zgrep -Fav -e "Code: 236. DB::Exception: Cancelled merging parts" \ + -e "Code: 236. DB::Exception: Cancelled mutating parts" \ + -e "REPLICA_IS_ALREADY_ACTIVE" \ + -e "REPLICA_ALREADY_EXISTS" \ + -e "ALL_REPLICAS_LOST" \ + -e "DDLWorker: Cannot parse DDL task query" \ + -e "RaftInstance: failed to accept a rpc connection due to error 125" \ + -e "UNKNOWN_DATABASE" \ + -e "NETWORK_ERROR" \ + -e "UNKNOWN_TABLE" \ + -e "ZooKeeperClient" \ + -e "KEEPER_EXCEPTION" \ + -e "DirectoryMonitor" \ + -e "TABLE_IS_READ_ONLY" \ + -e "Code: 1000, e.code() = 111, Connection refused" \ + -e "UNFINISHED" \ + -e "NETLINK_ERROR" \ + -e "Renaming unexpected part" \ + -e "PART_IS_TEMPORARILY_LOCKED" \ + -e "and a merge is impossible: we didn't find" \ + -e "found in queue and some source parts for it was lost" \ + -e "is lost forever." \ + -e "Unknown index: idx." \ + -e "Cannot parse string 'Hello' as UInt64" \ + -e "} TCPHandler: Code:" \ + -e "} executeQuery: Code:" \ + -e "Missing columns: 'v3' while processing query: 'v3, k, v1, v2, p'" \ + -e "This engine is deprecated and is not supported in transactions" \ + -e "[Queue = DB::MergeMutateRuntimeQueue]: Code: 235. DB::Exception: Part" \ + -e "The set of parts restored in place of" \ + -e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \ + -e "Code: 269. DB::Exception: Destination table is myself" \ + -e "Coordination::Exception: Connection loss" \ + -e "MutateFromLogEntryTask" \ + -e "No connection to ZooKeeper, cannot get shared table ID" \ + -e "Session expired" \ + /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ + && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv + + # Remove file bc_check_error_messages.txt if it's empty + [ -s /test_output/bc_check_error_messages.txt ] || rm /test_output/bc_check_error_messages.txt + + # Sanitizer asserts + zgrep -Fa "==================" /var/log/clickhouse-server/stderr.log >> /test_output/tmp + zgrep -Fa "WARNING" /var/log/clickhouse-server/stderr.log >> /test_output/tmp + zgrep -Fav -e "ASan doesn't fully support makecontext/swapcontext functions" -e "DB::Exception" /test_output/tmp > /dev/null \ + && echo -e 'Backward compatibility check: Sanitizer assert (in stderr.log)\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: No sanitizer asserts\tOK' >> /test_output/test_results.tsv + rm -f /test_output/tmp + + # OOM + zgrep -Fa " Application: Child process was terminated by signal 9" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ + && echo -e 'Backward compatibility check: OOM killer (or signal 9) in clickhouse-server.log\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: No OOM messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv + + # Logical errors + echo "Check for Logical errors in server log:" + zgrep -Fa -A20 "Code: 49, e.displayText() = DB::Exception:" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_logical_errors.txt \ + && echo -e 'Backward compatibility check: Logical error thrown (see clickhouse-server.log or bc_check_logical_errors.txt)\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: No logical errors\tOK' >> /test_output/test_results.tsv + + # Remove file bc_check_logical_errors.txt if it's empty + [ -s /test_output/bc_check_logical_errors.txt ] || rm /test_output/bc_check_logical_errors.txt + + # Crash + zgrep -Fa "########################################" /var/log/clickhouse-server/clickhouse-server.backward.*.log > /dev/null \ + && echo -e 'Backward compatibility check: Killed by signal (in clickhouse-server.log)\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: Not crashed\tOK' >> /test_output/test_results.tsv + + # It also checks for crash without stacktrace (printed by watchdog) + echo "Check for Fatal message in server log:" + zgrep -Fa " " /var/log/clickhouse-server/clickhouse-server.backward.*.log > /test_output/bc_check_fatal_messages.txt \ + && echo -e 'Backward compatibility check: Fatal message in clickhouse-server.log (see bc_check_fatal_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ + || echo -e 'Backward compatibility check: No fatal messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv + + # Remove file bc_check_fatal_messages.txt if it's empty + [ -s /test_output/bc_check_fatal_messages.txt ] || rm /test_output/bc_check_fatal_messages.txt + + tar -chf /test_output/coordination.backward.tar /var/lib/clickhouse/coordination ||: + for table in query_log trace_log + do + clickhouse-local --path /var/lib/clickhouse/ --only-system-tables -q "select * from system.$table format TSVWithNamesAndTypes" | pigz > /test_output/$table.backward.tsv.gz ||: + done + fi fi dmesg -T > /test_output/dmesg.log From 600bedbff4510589997e284765bbe953f460afd1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 24 Dec 2022 18:38:14 +0100 Subject: [PATCH 122/339] Add setting to disallow arguments in ReplicatedMergeTree constructor for DatabaseReplicated --- src/Core/Settings.h | 1 + src/Databases/DatabaseReplicated.cpp | 9 ++++++++- .../02232_allow_only_replicated_engine.sh | 2 ++ ...se_replicated_no_arguments_for_rmt.reference | 0 ..._database_replicated_no_arguments_for_rmt.sh | 17 +++++++++++++++++ 5 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.reference create mode 100755 tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 450304b2abd..b10760c8277 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -565,6 +565,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ M(Bool, database_replicated_always_detach_permanently, false, "Execute DETACH TABLE as DETACH TABLE PERMANENTLY if database engine is Replicated", 0) \ M(Bool, database_replicated_allow_only_replicated_engine, false, "Allow to create only Replicated tables in database with engine Replicated", 0) \ + M(Bool, database_replicated_allow_replicated_engine_arguments, true, "Allow to create only Replicated tables in database with engine Replicated with explicit arguments", 0) \ M(DistributedDDLOutputMode, distributed_ddl_output_mode, DistributedDDLOutputMode::THROW, "Format of distributed DDL query result", 0) \ M(UInt64, distributed_ddl_entry_format_version, 3, "Compatibility version of distributed DDL (ON CLUSTER) queries", 0) \ \ diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 2a9f06e77fc..7eeadf3feb9 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -584,7 +584,14 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, ContextPtr query_ bool enable_functional_tests_helper = getContext()->getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); if (!enable_functional_tests_helper) - LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + { + if (query_context->getSettingsRef().database_replicated_allow_replicated_engine_arguments) + LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + else + throw Exception(ErrorCodes::INCORRECT_QUERY, + "It's not allowed to specify explicit zookeeper_path and replica_name for ReplicatedMergeTree arguments in Replicated database. " + "If you really want to specify them explicitly, enable setting database_replicated_allow_replicated_engine_arguments."); + } if (maybe_shard_macros && maybe_replica_macros) return; diff --git a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh index 3ff2dabfa43..193d5fdb6d5 100755 --- a/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh +++ b/tests/queries/0_stateless/02232_allow_only_replicated_engine.sh @@ -16,3 +16,5 @@ ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none -n --query "CREATE TABLE ${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt (x UInt32) engine = ReplicatedMergeTree order by x;" ${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" ${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" + +${CLICKHOUSE_CLIENT} -q "drop table mute_stylecheck" diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.reference b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh new file mode 100755 index 00000000000..29af4882c1c --- /dev/null +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env sh + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "create table mute_stylecheck (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root', '1') order by x" + +${CLICKHOUSE_CLIENT} -q "CREATE USER user_${CLICKHOUSE_DATABASE} settings database_replicated_allow_replicated_engine_arguments=0" +${CLICKHOUSE_CLIENT} -q "GRANT CREATE TABLE ON ${CLICKHOUSE_DATABASE}_db.* TO user_${CLICKHOUSE_DATABASE}" +${CLICKHOUSE_CLIENT} --allow_experimental_database_replicated=1 --query "CREATE DATABASE ${CLICKHOUSE_DATABASE}_db engine = Replicated('/clickhouse/databases/${CLICKHOUSE_TEST_ZOOKEEPER_PREFIX}/${CLICKHOUSE_DATABASE}_db', '{shard}', '{replica}')" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_ok (x UInt32) engine = ReplicatedMergeTree order by x;" +${CLICKHOUSE_CLIENT} --distributed_ddl_output_mode=none --user "user_${CLICKHOUSE_DATABASE}" -n --query "CREATE TABLE ${CLICKHOUSE_DATABASE}_db.tab_rmt_fail (x UInt32) engine = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/root/{shard}', '{replica}') order by x; -- { serverError 80 }" +${CLICKHOUSE_CLIENT} --query "DROP DATABASE ${CLICKHOUSE_DATABASE}_db" +${CLICKHOUSE_CLIENT} -q "DROP USER user_${CLICKHOUSE_DATABASE}" + +${CLICKHOUSE_CLIENT} -q "drop table mute_stylecheck" From 705be441a145e748fbe0f4f2f31aaf20064459d6 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 24 Dec 2022 18:51:56 +0100 Subject: [PATCH 123/339] Shutup shellcheck --- .../02514_database_replicated_no_arguments_for_rmt.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index 29af4882c1c..7d1444964b2 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -1,4 +1,5 @@ -#!/usr/bin/env sh +#!/usr/bin/env bash +# Tags: replica CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 60408d65434407ece5d87b2f222824b0026966c7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 24 Dec 2022 19:53:47 +0100 Subject: [PATCH 124/339] Disable BC check for asan (just test) --- .github/workflows/pull_request.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index a81f52a9371..d4f813aa488 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3004,6 +3004,7 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=Stress test (asan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + DISABLE_BC_CHECK=1 EOF - name: Download json reports uses: actions/download-artifact@v2 From 4651a538a89d7e23cc5681d0b8a299d10005ccf4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Dec 2022 20:31:04 +0100 Subject: [PATCH 125/339] Fix extremely slow stack traces in debug build --- CMakeLists.txt | 6 +++--- contrib/libunwind-cmake/CMakeLists.txt | 5 ++++- src/Common/CMakeLists.txt | 3 +++ 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 99997db96a1..66fdaa6a765 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -377,15 +377,15 @@ set (DEBUG_INFO_FLAGS "-g -gdwarf-4") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS}") set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}") -set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} -fno-inline ${CMAKE_CXX_FLAGS_ADD}") +set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}") -set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} -fno-inline ${CMAKE_C_FLAGS_ADD}") +set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} ${CMAKE_C_FLAGS_ADD}") set (CMAKE_ASM_FLAGS "${CMAKE_ASM_FLAGS} ${COMPILER_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") set (CMAKE_ASM_FLAGS_RELWITHDEBINFO "${CMAKE_ASM_FLAGS_RELWITHDEBINFO} -O3 ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") -set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} -fno-inline ${CMAKE_ASM_FLAGS_ADD}") +set (CMAKE_ASM_FLAGS_DEBUG "${CMAKE_ASM_FLAGS_DEBUG} -O0 ${DEBUG_INFO_FLAGS} ${CMAKE_ASM_FLAGS_ADD}") if (COMPILER_CLANG) if (OS_DARWIN) diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index 155853a0bca..733f99d07f5 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -43,7 +43,10 @@ set_target_properties(unwind PROPERTIES FOLDER "contrib/libunwind-cmake") target_include_directories(unwind SYSTEM BEFORE PUBLIC $) target_compile_definitions(unwind PRIVATE -D_LIBUNWIND_NO_HEAP=1 -D_DEBUG -D_LIBUNWIND_IS_NATIVE_ONLY) -target_compile_options(unwind PRIVATE -fno-exceptions -funwind-tables -fno-sanitize=all $<$:-nostdinc++ -fno-rtti>) + +# We should enable optimizations (otherwise it will be too slow in debug) +# and disable sanitizers (otherwise infinite loop may happen) +target_compile_options(unwind PRIVATE -O3 -fno-exceptions -funwind-tables -fno-sanitize=all $<$:-nostdinc++ -fno-rtti>) check_c_compiler_flag(-Wunused-but-set-variable HAVE_WARNING_UNUSED_BUT_SET_VARIABLE) if (HAVE_WARNING_UNUSED_BUT_SET_VARIABLE) diff --git a/src/Common/CMakeLists.txt b/src/Common/CMakeLists.txt index e527b3dec43..db848b00ff6 100644 --- a/src/Common/CMakeLists.txt +++ b/src/Common/CMakeLists.txt @@ -11,3 +11,6 @@ endif() if (ENABLE_MYSQL) add_subdirectory (mysqlxx) endif () + +# Otherwise stack traces symbolization is painfully slow in debug build. +set_source_files_properties("Dwarf.cpp" PROPERTIES COMPILE_FLAGS "-O3") From d4864c7d38db04487c05b0d698afb029a4afe9be Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Dec 2022 20:40:23 +0100 Subject: [PATCH 126/339] Better command line argument name in `clickhouse-benchmark` --- programs/benchmark/Benchmark.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 017b28fe082..26099b352a3 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -683,7 +683,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) ("confidence", value()->default_value(5), "set the level of confidence for T-test [0=80%, 1=90%, 2=95%, 3=98%, 4=99%, 5=99.5%(default)") ("query_id", value()->default_value(""), "") ("max-consecutive-errors", value()->default_value(0), "set number of allowed consecutive errors") - ("continue_on_errors", "continue testing even if a query fails") + ("ignore-error,continue_on_errors", "continue testing even if a query fails") ("reconnect", "establish new connection for every query") ("client-side-time", "display the time including network communication instead of server-side time; note that for server versions before 22.8 we always display client-side time") ; @@ -738,7 +738,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv) options["query_id"].as(), options["query"].as(), options["max-consecutive-errors"].as(), - options.count("continue_on_errors"), + options.count("ignore-error"), options.count("reconnect"), options.count("client-side-time"), print_stacktrace, From cbf140633bccc63a9d0a803b910221422f23c81a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 09:30:20 +0100 Subject: [PATCH 127/339] Fix HDFS test --- tests/integration/test_merge_tree_hdfs/test.py | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index 8a6703be2dc..6ff990d43d2 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -213,9 +213,7 @@ def test_attach_detach_partition(cluster): assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)" wait_for_delete_empty_parts(node, "hdfs_test") wait_for_delete_inactive_parts(node, "hdfs_test") - - hdfs_objects = fs.listdir("/clickhouse") - assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 + wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2) node.query("ALTER TABLE hdfs_test ATTACH PARTITION '2020-01-03'") assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)" @@ -227,9 +225,7 @@ def test_attach_detach_partition(cluster): assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)" wait_for_delete_empty_parts(node, "hdfs_test") wait_for_delete_inactive_parts(node, "hdfs_test") - - hdfs_objects = fs.listdir("/clickhouse") - assert len(hdfs_objects) == FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE + wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE) node.query("ALTER TABLE hdfs_test DETACH PARTITION '2020-01-04'") node.query( @@ -239,9 +235,7 @@ def test_attach_detach_partition(cluster): assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(0)" wait_for_delete_empty_parts(node, "hdfs_test") wait_for_delete_inactive_parts(node, "hdfs_test") - - hdfs_objects = fs.listdir("/clickhouse") - assert len(hdfs_objects) == FILES_OVERHEAD + wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD) def test_move_partition_to_another_disk(cluster): @@ -307,9 +301,7 @@ def test_table_manipulations(cluster): assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(0)" wait_for_delete_empty_parts(node, "hdfs_test") wait_for_delete_inactive_parts(node, "hdfs_test") - - hdfs_objects = fs.listdir("/clickhouse") - assert len(hdfs_objects) == FILES_OVERHEAD + wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD) def test_move_replace_partition_to_another_table(cluster): @@ -376,7 +368,6 @@ def test_move_replace_partition_to_another_table(cluster): assert node.query("SELECT count(*) FROM hdfs_clone FORMAT Values") == "(8192)" # Wait for outdated partitions deletion. - print(1) wait_for_delete_hdfs_objects( cluster, FILES_OVERHEAD * 2 + FILES_OVERHEAD_PER_PART_WIDE * 4 ) From 19f05d4fb4e259915ec1f9f526661b5d24bc930d Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 25 Dec 2022 08:38:02 +0000 Subject: [PATCH 128/339] Automatic style fix --- tests/integration/test_merge_tree_hdfs/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_merge_tree_hdfs/test.py b/tests/integration/test_merge_tree_hdfs/test.py index 6ff990d43d2..3950077e619 100644 --- a/tests/integration/test_merge_tree_hdfs/test.py +++ b/tests/integration/test_merge_tree_hdfs/test.py @@ -213,7 +213,9 @@ def test_attach_detach_partition(cluster): assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(4096)" wait_for_delete_empty_parts(node, "hdfs_test") wait_for_delete_inactive_parts(node, "hdfs_test") - wait_for_delete_hdfs_objects(cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2) + wait_for_delete_hdfs_objects( + cluster, FILES_OVERHEAD + FILES_OVERHEAD_PER_PART_WIDE * 2 + ) node.query("ALTER TABLE hdfs_test ATTACH PARTITION '2020-01-03'") assert node.query("SELECT count(*) FROM hdfs_test FORMAT Values") == "(8192)" From f9f0f9446d0235ea22f07fdf1396a31c7a9fe8f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 09:46:51 +0100 Subject: [PATCH 129/339] Fix distributed_queries_stress --- tests/integration/test_distributed_queries_stress/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_distributed_queries_stress/test.py b/tests/integration/test_distributed_queries_stress/test.py index fce42b4e58b..65c8b28e8d3 100644 --- a/tests/integration/test_distributed_queries_stress/test.py +++ b/tests/integration/test_distributed_queries_stress/test.py @@ -24,7 +24,7 @@ def run_benchmark(payload, settings): [ "clickhouse", "benchmark", - "--concurrency=100", + "--concurrency=10", "--cumulative", "--delay=0", # NOTE: with current matrix even 3 seconds it huge... From fa722c6cd852a6e854735da210f9d5b3c1486a1a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 25 Dec 2022 10:50:42 +0100 Subject: [PATCH 130/339] Switch "contrib/sysroot" back to master. --- contrib/sysroot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/sysroot b/contrib/sysroot index e9fb375d0a1..0f41651860f 160000 --- a/contrib/sysroot +++ b/contrib/sysroot @@ -1 +1 @@ -Subproject commit e9fb375d0a1e5ebfd74c043f088f2342552103f8 +Subproject commit 0f41651860fa4a530ecd68b93a15b8fd77397adf From bb77ad8941b5e2f35f97f2283098a332af639477 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 25 Dec 2022 13:49:01 +0100 Subject: [PATCH 131/339] Disable BC check --- tests/ci/stress_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index b7f74c5aeb7..b90da97fa04 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -34,6 +34,7 @@ def get_run_command( "docker run --cap-add=SYS_PTRACE " # a static link, don't use S3_URL or S3_DOWNLOAD "-e S3_URL='https://s3.amazonaws.com/clickhouse-datasets' " + f"-e DISABLE_BC_CHECK={os.environ.get('DISABLE_BC_CHECK', '0')} " # For dmesg and sysctl "--privileged " f"--volume={build_path}:/package_folder " From b3b2b612817b8662ce30e213ded04e126431b823 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 25 Dec 2022 14:06:20 +0100 Subject: [PATCH 132/339] Disable test for replicated DB run --- .../02514_database_replicated_no_arguments_for_rmt.sh | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh index 7d1444964b2..ee51640488e 100755 --- a/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh +++ b/tests/queries/0_stateless/02514_database_replicated_no_arguments_for_rmt.sh @@ -1,5 +1,8 @@ #!/usr/bin/env bash -# Tags: replica +# Tags: replica, no-replicated-database +# I don't understand why this test fails in ReplicatedDatabase run +# but too many magic included in it, so I just disabled it for ReplicatedDatabase run becase +# here we explicitely create it and check is alright. CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From f22fc35900a674f7006c5f05fe178975fe64b6d4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 14:18:05 +0100 Subject: [PATCH 133/339] Minimal changes --- docker/test/stress/README.md | 4 ++-- src/Common/AsynchronousMetrics.cpp | 5 ----- .../MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h | 2 +- 3 files changed, 3 insertions(+), 8 deletions(-) diff --git a/docker/test/stress/README.md b/docker/test/stress/README.md index 96807b9f9a6..c22721fd7da 100644 --- a/docker/test/stress/README.md +++ b/docker/test/stress/README.md @@ -1,6 +1,6 @@ -Allow to run simple ClickHouse stress test in Docker from debian packages. +Allows to run simple ClickHouse stress test in Docker from debian packages. Actually it runs multiple copies of clickhouse-test (functional tests). -This allows to find problems like segmentation fault which cause shutdown of server. +This allows to find problems like failed assertions and memory safety issues. Usage: ``` diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index d4626d317c7..b68fcab2449 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -2,12 +2,7 @@ #include #include #include -#include #include -#include -#include -#include -#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h index 91f5824f8fc..05b3d656579 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.h @@ -17,7 +17,7 @@ struct ReplicatedMergeTreeLogEntryData; /// (so instead of doing exactly the same merge cluster-wise you can do merge once and fetch ready part) /// Fetches may be desirable for other operational reasons (backup replica without lot of CPU resources). /// -/// That class allow to take a decisions about preferred strategy for a concreate merge. +/// That class allow to take a decisions about preferred strategy for a concrete merge. /// /// Since that code is used in shouldExecuteLogEntry we need to be able to: /// 1) make decision fast From 71850f6913bb3a078d535d1e55537d7a6e0256a7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 16:06:34 +0100 Subject: [PATCH 134/339] Check for bad index_granularity --- src/Storages/MergeTree/MergeTreeData.cpp | 17 ----------------- .../MergeTreeDataPartWriterCompact.cpp | 3 ++- .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 10 +++++++--- src/Storages/MergeTree/MergeTreeSettings.cpp | 9 +++++++++ 4 files changed, 18 insertions(+), 21 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4089ba9adb8..a8cfc72ad22 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1,34 +1,23 @@ #include #include -#include #include #include #include #include #include -#include -#include -#include #include #include -#include #include #include #include -#include #include -#include #include #include #include -#include #include -#include #include -#include #include -#include #include #include #include @@ -56,7 +45,6 @@ #include #include #include -#include #include #include #include @@ -70,22 +58,17 @@ #include #include #include -#include #include #include #include -#include #include #include -#include #include -#include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 020121e59d7..4c1d117ac73 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -110,7 +110,7 @@ Granules getGranulesToWrite(const MergeTreeIndexGranularity & index_granularity, .is_complete = (rows_left_in_block >= expected_rows_in_mark) }); current_row += result.back().rows_to_write; - current_mark++; + ++current_mark; } return result; @@ -146,6 +146,7 @@ void MergeTreeDataPartWriterCompact::write(const Block & block, const IColumn::P if (compute_granularity) { size_t index_granularity_for_block = computeIndexGranularity(block); + assert(index_granularity_for_block >= 1); fillIndexGranularity(index_granularity_for_block, block.rows()); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index a887b0ee322..fbcf8cb241c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -135,7 +135,9 @@ static size_t computeIndexGranularityImpl( size_t rows_in_block = block.rows(); size_t index_granularity_for_block; if (!can_use_adaptive_index_granularity) + { index_granularity_for_block = fixed_index_granularity_rows; + } else { size_t block_size_in_memory = block.bytes(); @@ -152,11 +154,13 @@ static size_t computeIndexGranularityImpl( index_granularity_for_block = index_granularity_bytes / size_of_row_in_bytes; } } - if (index_granularity_for_block == 0) /// very rare case when index granularity bytes less then single row - index_granularity_for_block = 1; - /// We should be less or equal than fixed index granularity index_granularity_for_block = std::min(fixed_index_granularity_rows, index_granularity_for_block); + + /// very rare case when index granularity bytes less then single row + if (index_granularity_for_block == 0) + index_granularity_for_block = 1; + return index_granularity_for_block; } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 5b78a59687b..2bf717c883a 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -99,6 +99,15 @@ void MergeTreeSettings::sanityCheck(size_t background_pool_tasks) const background_pool_tasks); } + // Zero index_granularity is nonsensical. + if (index_granularity < 1) + { + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "index_granularity: value {} makes no sense", + index_granularity); + } + // The min_index_granularity_bytes value is 1024 b and index_granularity_bytes is 10 mb by default. // If index_granularity_bytes is not disabled i.e > 0 b, then always ensure that it's greater than // min_index_granularity_bytes. This is mainly a safeguard against accidents whereby a really low From a4895286de8f97a34b28b8244d8c90f561e5c338 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 16:08:20 +0100 Subject: [PATCH 135/339] Add a test --- .../0_stateless/02514_bad_index_granularity.reference | 0 tests/queries/0_stateless/02514_bad_index_granularity.sql | 7 +++++++ 2 files changed, 7 insertions(+) create mode 100644 tests/queries/0_stateless/02514_bad_index_granularity.reference create mode 100644 tests/queries/0_stateless/02514_bad_index_granularity.sql diff --git a/tests/queries/0_stateless/02514_bad_index_granularity.reference b/tests/queries/0_stateless/02514_bad_index_granularity.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02514_bad_index_granularity.sql b/tests/queries/0_stateless/02514_bad_index_granularity.sql new file mode 100644 index 00000000000..975af2d0728 --- /dev/null +++ b/tests/queries/0_stateless/02514_bad_index_granularity.sql @@ -0,0 +1,7 @@ +CREATE TABLE t +( + id Int64, + d String, + p Map(String, String) +) +ENGINE = ReplacingMergeTree order by id settings index_granularity = 0; -- { serverError BAD_ARGUMENTS } From 51503fc9e420cd73e6bb73be8f02a9ad5bd8a3f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 16:19:42 +0100 Subject: [PATCH 136/339] Fuzzer HTML: fix trash --- docker/test/fuzzer/run-fuzzer.sh | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index bd539ca978b..2b62ff5d2c6 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -352,17 +352,25 @@ th { cursor: pointer; }
-

AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}

+

AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}

- - + + + + + + + + + +
Test nameTest statusDescription
AST Fuzzer$(cat status.txt)$(cat description.txt)
Test nameTest statusDescription
AST Fuzzer$(cat status.txt)$(cat description.txt | clickhouse-local --input-format RawBLOB --output-format RawBLOB --query "SELECT encodeXMLComponent(*) FROM table")
From 31bd775073e40136cd0124d8410077c14ae2b978 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 19:53:26 +0300 Subject: [PATCH 137/339] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 2b62ff5d2c6..7d5875e6894 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -369,7 +369,7 @@ th { cursor: pointer; } AST Fuzzer $(cat status.txt) - $(cat description.txt | clickhouse-local --input-format RawBLOB --output-format RawBLOB --query "SELECT encodeXMLComponent(*) FROM table") + $(clickhouse-local --input-format RawBLOB --output-format RawBLOB --query "SELECT encodeXMLComponent(*) FROM table" < description.txt) From 211c6062bc5c305b1d950ea1265050185900537d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 25 Dec 2022 19:37:02 +0100 Subject: [PATCH 138/339] Fix --- docker/test/stress/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index d948a246275..984f35d5f27 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -334,7 +334,7 @@ zgrep -Fa "########################################" /test_output/* > /dev/null zgrep -Fa " received signal " /test_output/gdb.log > /dev/null \ && echo -e 'Found signal in gdb.log\tFAIL' >> /test_output/test_results.tsv -if [ -z "$DISABLE_BC_CHECK" ]; then +if [ "$DISABLE_BC_CHECK" -ne "1" ]; then echo -e "Backward compatibility check\n" echo "Get previous release tag" From fe73b0f64808e5670ecafc3a773f0f970b9dbb78 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Dec 2022 22:14:25 +0300 Subject: [PATCH 139/339] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 7d5875e6894..60f78886bab 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -75,6 +75,7 @@ function download ./clickhouse ||: ln -s ./clickhouse ./clickhouse-server ln -s ./clickhouse ./clickhouse-client + ln -s ./clickhouse ./clickhouse-local # clickhouse-server is in the current dir export PATH="$PWD:$PATH" From a06c8268ea9ccc5cf2de813a8b7f8a5f43130509 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 11:28:26 +0100 Subject: [PATCH 140/339] Remove the offender --- .../__init__.py | 0 .../configs/remote_servers.xml | 42 ------ .../test_distributed_queries_stress/test.py | 121 ------------------ 3 files changed, 163 deletions(-) delete mode 100644 tests/integration/test_distributed_queries_stress/__init__.py delete mode 100644 tests/integration/test_distributed_queries_stress/configs/remote_servers.xml delete mode 100644 tests/integration/test_distributed_queries_stress/test.py diff --git a/tests/integration/test_distributed_queries_stress/__init__.py b/tests/integration/test_distributed_queries_stress/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml b/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml deleted file mode 100644 index 79e71f1ecf3..00000000000 --- a/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml +++ /dev/null @@ -1,42 +0,0 @@ - - 1000 - - - - - true - - node1_r1 - 9000 - - - node1_r2 - 9000 - - - - - - true - - node1_r1 - 9000 - - - node1_r2 - 9000 - - - - - node2_r1 - 9000 - - - node2_r2 - 9000 - - - - - diff --git a/tests/integration/test_distributed_queries_stress/test.py b/tests/integration/test_distributed_queries_stress/test.py deleted file mode 100644 index 65c8b28e8d3..00000000000 --- a/tests/integration/test_distributed_queries_stress/test.py +++ /dev/null @@ -1,121 +0,0 @@ -# pylint: disable=redefined-outer-name -# pylint: disable=unused-argument -# pylint: disable=line-too-long - -import shlex -import itertools -import pytest -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) -node1_r1 = cluster.add_instance("node1_r1", main_configs=["configs/remote_servers.xml"]) -node2_r1 = cluster.add_instance("node2_r1", main_configs=["configs/remote_servers.xml"]) -node1_r2 = cluster.add_instance("node1_r2", main_configs=["configs/remote_servers.xml"]) -node2_r2 = cluster.add_instance("node2_r2", main_configs=["configs/remote_servers.xml"]) - - -def run_benchmark(payload, settings): - node1_r1.exec_in_container( - [ - "bash", - "-c", - "echo {} | ".format(shlex.quote(payload.strip())) - + " ".join( - [ - "clickhouse", - "benchmark", - "--concurrency=10", - "--cumulative", - "--delay=0", - # NOTE: with current matrix even 3 seconds it huge... - "--timelimit=3", - # tune some basic timeouts - "--hedged_connection_timeout_ms=200", - "--connect_timeout_with_failover_ms=200", - "--connections_with_failover_max_tries=5", - *settings, - ] - ), - ] - ) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - - for _, instance in cluster.instances.items(): - instance.query( - """ - create table if not exists data ( - key Int, - /* just to increase block size */ - v1 UInt64, - v2 UInt64, - v3 UInt64, - v4 UInt64, - v5 UInt64, - v6 UInt64, - v7 UInt64, - v8 UInt64, - v9 UInt64, - v10 UInt64, - v11 UInt64, - v12 UInt64 - ) Engine=MergeTree() order by key partition by key%5; - insert into data (key) select * from numbers(10); - - create table if not exists dist_one as data engine=Distributed(one_shard, currentDatabase(), data, key); - create table if not exists dist_one_over_dist as data engine=Distributed(one_shard, currentDatabase(), dist_one, kostikConsistentHash(key, 2)); - - create table if not exists dist_two as data engine=Distributed(two_shards, currentDatabase(), data, key); - create table if not exists dist_two_over_dist as data engine=Distributed(two_shards, currentDatabase(), dist_two, kostikConsistentHash(key, 2)); - """ - ) - yield cluster - finally: - cluster.shutdown() - - -@pytest.mark.parametrize( - "table,settings", - itertools.product( - [ # tables - "dist_one", - "dist_one_over_dist", - "dist_two", - "dist_two_over_dist", - ], - [ # settings - *list( - itertools.combinations( - [ - "", # defaults - "--prefer_localhost_replica=0", - "--async_socket_for_remote=0", - "--use_hedged_requests=0", - "--optimize_skip_unused_shards=1", - "--distributed_group_by_no_merge=2", - "--optimize_distributed_group_by_sharding_key=1", - # TODO: enlarge test matrix (but first those values to accept ms): - # - # - sleep_in_send_tables_status - # - sleep_in_send_data - ], - 2, - ) - ) - # TODO: more combinations that just 2 - ], - ), -) -def test_stress_distributed(table, settings, started_cluster): - payload = f""" - select * from {table} where key = 0; - select * from {table} where key = 1; - select * from {table} where key = 2; - select * from {table} where key = 3; - select * from {table}; - """ - run_benchmark(payload, settings) From e98f217b870864a421d04da4c41fe8a1f8ba432e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 12:15:58 +0100 Subject: [PATCH 141/339] Better diagnostics on stop --- programs/install/Install.cpp | 85 ++++++++++++++++++------------------ src/Daemon/BaseDaemon.cpp | 5 +-- src/Daemon/BaseDaemon.h | 1 - 3 files changed, 44 insertions(+), 47 deletions(-) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index de85572d5c6..d568012bb26 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -1051,18 +1051,12 @@ namespace return pid; } - int stop(const fs::path & pid_file, bool force, bool do_not_kill, unsigned max_tries) + bool sendSignalAndWaitForStop(const fs::path & pid_file, int signal, unsigned max_tries, unsigned wait_ms, const char * signal_name) { - if (force && do_not_kill) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible"); - int pid = isRunning(pid_file); if (!pid) - return 0; - - int signal = force ? SIGKILL : SIGTERM; - const char * signal_name = force ? "kill" : "terminate"; + return true; if (0 == kill(pid, signal)) fmt::print("Sent {} signal to process with pid {}.\n", signal_name, pid); @@ -1078,46 +1072,51 @@ namespace fmt::print("Server stopped\n"); break; } - sleepForSeconds(1); + sleepForMilliseconds(wait_ms); } - if (try_num == max_tries) + return try_num < max_tries; + } + + int stop(const fs::path & pid_file, bool force, bool do_not_kill, unsigned max_tries) + { + if (force && do_not_kill) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Specified flags are incompatible"); + + int signal = force ? SIGKILL : SIGTERM; + const char * signal_name = force ? "kill" : "terminate"; + + if (sendSignalAndWaitForStop(pid_file, signal, max_tries, 1000, signal_name)) + return 0; + + int pid = isRunning(pid_file); + if (!pid) + return 0; + + if (do_not_kill) { - if (do_not_kill) - { - fmt::print("Process (pid = {}) is still running. Will not try to kill it.\n", pid); - return 1; - } - - fmt::print("Will terminate forcefully (pid = {}).\n", pid); - if (0 == kill(pid, 9)) - fmt::print("Sent kill signal (pid = {}).\n", pid); - else - throwFromErrno("Cannot send kill signal", ErrorCodes::SYSTEM_ERROR); - - /// Wait for the process (100 seconds). - constexpr size_t num_kill_check_tries = 1000; - constexpr size_t kill_check_delay_ms = 100; - for (size_t i = 0; i < num_kill_check_tries; ++i) - { - fmt::print("Waiting for server to be killed\n"); - if (!isRunning(pid_file)) - { - fmt::print("Server exited\n"); - break; - } - sleepForMilliseconds(kill_check_delay_ms); - } - - if (isRunning(pid_file)) - { - throw Exception(ErrorCodes::CANNOT_KILL, - "The server process still exists after {} tries (delay: {} ms)", - num_kill_check_tries, kill_check_delay_ms); - } + fmt::print("Process (pid = {}) is still running. Will not try to kill it.\n", pid); + return 1; } - return 0; + /// Send termination signal again, the server will receive it and immediately terminate. + fmt::print("Will send the termination signal again to force the termination (pid = {}).\n", pid); + if (sendSignalAndWaitForStop(pid_file, signal, std::min(10U, max_tries), 1000, signal_name)) + return 0; + + /// Send kill signal. Total wait is 100 seconds. + constexpr size_t num_kill_check_tries = 1000; + constexpr size_t kill_check_delay_ms = 100; + fmt::print("Will terminate forcefully (pid = {}).\n", pid); + if (sendSignalAndWaitForStop(pid_file, SIGKILL, num_kill_check_tries, kill_check_delay_ms, signal_name)) + return 0; + + if (!isRunning(pid_file)) + return 0; + + throw Exception(ErrorCodes::CANNOT_KILL, + "The server process still exists after {} tries (delay: {} ms)", + num_kill_check_tries, kill_check_delay_ms); } } diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 6cd952bfa83..5e5ba41b1ba 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -958,7 +958,6 @@ void BaseDaemon::handleSignal(int signal_id) std::lock_guard lock(signal_handler_mutex); { ++terminate_signals_counter; - sigint_signals_counter += signal_id == SIGINT; signal_event.notify_all(); } @@ -973,9 +972,9 @@ void BaseDaemon::onInterruptSignals(int signal_id) is_cancelled = true; LOG_INFO(&logger(), "Received termination signal ({})", strsignal(signal_id)); // NOLINT(concurrency-mt-unsafe) // it is not thread-safe but ok in this context - if (sigint_signals_counter >= 2) + if (terminate_signals_counter >= 2) { - LOG_INFO(&logger(), "Received second signal Interrupt. Immediately terminate."); + LOG_INFO(&logger(), "This is the second termination signal. Immediately terminate."); call_default_signal_handler(signal_id); /// If the above did not help. _exit(128 + signal_id); diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index cb4aa0c2da6..d28f9403c16 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -162,7 +162,6 @@ protected: std::mutex signal_handler_mutex; std::condition_variable signal_event; std::atomic_size_t terminate_signals_counter{0}; - std::atomic_size_t sigint_signals_counter{0}; std::string config_path; DB::ConfigProcessor::LoadedConfig loaded_config; From d08e7e5f2d9a95576d7bc4ce609f288edfc6f487 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 13:30:13 +0100 Subject: [PATCH 142/339] The position of the log message about the server environment was wrong --- programs/server/Server.cpp | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 60adcb27d1d..d3c24c82780 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -703,6 +703,13 @@ try global_context->addWarningMessage("Server was built with sanitizer. It will work slowly."); #endif + const auto memory_amount = getMemoryAmount(); + + LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", + formatReadableSizeWithBinarySuffix(memory_amount), + getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores + std::thread::hardware_concurrency()); + sanityChecks(*this); // Initialize global thread pool. Do it before we fetch configs from zookeeper @@ -776,8 +783,6 @@ try Settings::checkNoSettingNamesAtTopLevel(config(), config_path); - const auto memory_amount = getMemoryAmount(); - #if defined(OS_LINUX) std::string executable_path = getExecutablePath(); @@ -1718,13 +1723,6 @@ try main_config_reloader->start(); access_control.startPeriodicReloading(); - { - LOG_INFO(log, "Available RAM: {}; physical cores: {}; logical cores: {}.", - formatReadableSizeWithBinarySuffix(memory_amount), - getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores - std::thread::hardware_concurrency()); - } - /// try to load dictionaries immediately, throw on error and die try { From b6bf942f70fbaa2b3bcf5df46819993e27bc427b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 13:35:44 +0100 Subject: [PATCH 143/339] Fix bad punctuation in log --- src/Daemon/BaseDaemon.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 6cd952bfa83..59050557665 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -909,7 +909,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() void BaseDaemon::logRevision() const { - Poco::Logger::root().information("Starting " + std::string{VERSION_FULL} + logger().information("Starting " + std::string{VERSION_FULL} + " (revision: " + std::to_string(ClickHouseRevision::getVersionRevision()) + ", git hash: " + (git_hash.empty() ? "" : git_hash) + ", build id: " + (build_id.empty() ? "" : build_id) + ")" From fec074fd1e7038dcce9fa706360a171197200db8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 13:51:59 +0100 Subject: [PATCH 144/339] Fix misleading log message --- src/Interpreters/Context.cpp | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index de29f4acefb..95afc6a17c2 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -763,13 +763,17 @@ try fs::directory_iterator dir_end; for (fs::directory_iterator it(path); it != dir_end; ++it) { - if (it->is_regular_file() && startsWith(it->path().filename(), "tmp")) + if (it->is_regular_file()) { - LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); - fs::remove(it->path()); + if (startsWith(it->path().filename(), "tmp")) + { + LOG_DEBUG(log, "Removing old temporary file {}", it->path().string()); + fs::remove(it->path()); + } + else + LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); } - else - LOG_DEBUG(log, "Found unknown file in temporary path {}", it->path().string()); + /// We skip directories (for example, 'http_buffers' - it's used for buffering of the results) and all other file types. } } catch (...) From b218d549bf948525ff0938cfb22e461f1615bbf0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 13:56:09 +0100 Subject: [PATCH 145/339] Fix bad log message --- programs/server/Server.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 60adcb27d1d..1acb53890a9 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1044,8 +1044,8 @@ try bool continue_if_corrupted = config().getBool("merge_tree_metadata_cache.continue_if_corrupted", false); try { - LOG_DEBUG( - log, "Initializing merge tree metadata cache lru_cache_size:{} continue_if_corrupted:{}", size, continue_if_corrupted); + LOG_DEBUG(log, "Initializing MergeTree metadata cache, lru_cache_size: {} continue_if_corrupted: {}", + ReadableSize(size), continue_if_corrupted); global_context->initializeMergeTreeMetadataCache(path_str + "/" + "rocksdb", size); } catch (...) From 4d17510fcac4bfc4b9bae3270bff19c3928929ed Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 26 Dec 2022 13:58:32 +0100 Subject: [PATCH 146/339] Use already written part of the query for fuzzy search (pass to skim) Signed-off-by: Azat Khuzhin --- base/base/ReplxxLineReader.cpp | 3 ++- rust/skim/include/skim.h | 2 +- rust/skim/src/lib.rs | 5 +++-- 3 files changed, 6 insertions(+), 4 deletions(-) diff --git a/base/base/ReplxxLineReader.cpp b/base/base/ReplxxLineReader.cpp index 9e29f7744fa..261f0e2a16a 100644 --- a/base/base/ReplxxLineReader.cpp +++ b/base/base/ReplxxLineReader.cpp @@ -402,10 +402,11 @@ ReplxxLineReader::ReplxxLineReader( words.push_back(hs.get().text()); } + std::string current_query(rx.get_state().text()); std::string new_query; try { - new_query = std::string(skim(words)); + new_query = std::string(skim(current_query, words)); } catch (const std::exception & e) { diff --git a/rust/skim/include/skim.h b/rust/skim/include/skim.h index 12cd257567b..8148474eba3 100644 --- a/rust/skim/include/skim.h +++ b/rust/skim/include/skim.h @@ -87,4 +87,4 @@ private: } // namespace cxxbridge1 } // namespace rust -::rust::String skim(::std::vector<::std::string> const &words); +::rust::String skim(::std::string const &prefix, ::std::vector<::std::string> const &words); diff --git a/rust/skim/src/lib.rs b/rust/skim/src/lib.rs index 29160329287..90f39cc8382 100644 --- a/rust/skim/src/lib.rs +++ b/rust/skim/src/lib.rs @@ -5,7 +5,7 @@ use cxx::{CxxString, CxxVector}; #[cxx::bridge] mod ffi { extern "Rust" { - fn skim(words: &CxxVector) -> Result; + fn skim(prefix: &CxxString, words: &CxxVector) -> Result; } } @@ -18,7 +18,7 @@ impl SkimItem for Item { } } -fn skim(words: &CxxVector) -> Result { +fn skim(prefix: &CxxString, words: &CxxVector) -> Result { // Let's check is terminal available. To avoid panic. if let Err(err) = TermInfo::from_env() { return Err(format!("{}", err)); @@ -26,6 +26,7 @@ fn skim(words: &CxxVector) -> Result { let options = SkimOptionsBuilder::default() .height(Some("30%")) + .query(Some(prefix.to_str().unwrap())) .tac(true) .tiebreak(Some("-score".to_string())) .build() From 6339f4f429975ed49cb4f78acb9f63a4698cc484 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 26 Dec 2022 16:14:29 +0100 Subject: [PATCH 147/339] Revert "Disable BC check for asan (just test)" This reverts commit 60408d65434407ece5d87b2f222824b0026966c7. --- .github/workflows/pull_request.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index d4f813aa488..a81f52a9371 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3004,7 +3004,6 @@ jobs: REPORTS_PATH=${{runner.temp}}/reports_dir CHECK_NAME=Stress test (asan) REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - DISABLE_BC_CHECK=1 EOF - name: Download json reports uses: actions/download-artifact@v2 From 6ba360be71cba402eb3ff23b0d522ea206de9976 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 26 Dec 2022 16:29:32 +0100 Subject: [PATCH 148/339] Rename runlog.log to run.log --- docker/test/fuzzer/run-fuzzer.sh | 2 +- tests/ci/ast_fuzzer_check.py | 8 ++++---- tests/ci/codebrowser_check.py | 2 +- tests/ci/docs_check.py | 2 +- tests/ci/docs_release.py | 2 +- tests/ci/fast_test_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/jepsen_check.py | 2 +- tests/ci/performance_comparison_check.py | 8 ++++---- tests/ci/sqlancer_check.py | 2 +- tests/ci/stress_check.py | 2 +- tests/ci/unit_tests_check.py | 2 +- 12 files changed, 18 insertions(+), 18 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index bd539ca978b..2b29b12df2c 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -354,7 +354,7 @@ th { cursor: pointer; }

AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}