From ed8b3b4a4e78210c42463ad0c445f255e9814e6d Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 14 Jan 2020 11:07:31 +0800 Subject: [PATCH 001/609] add WINDOW VIEW --- dbms/CMakeLists.txt | 1 + dbms/src/Common/ErrorCodes.cpp | 2 + dbms/src/Core/Defines.h | 1 + dbms/src/Core/Settings.h | 2 + .../PushingToViewsBlockOutputStream.cpp | 17 + dbms/src/Databases/DatabaseOnDisk.cpp | 2 +- .../Interpreters/InterpreterCreateQuery.cpp | 15 +- .../Interpreters/InterpreterWatchQuery.cpp | 4 +- dbms/src/Parsers/ASTAlterQuery.cpp | 6 + dbms/src/Parsers/ASTAlterQuery.h | 4 + dbms/src/Parsers/ASTCreateQuery.cpp | 2 + dbms/src/Parsers/ASTCreateQuery.h | 1 + dbms/src/Parsers/ParserAlterQuery.cpp | 25 +- dbms/src/Parsers/ParserAlterQuery.h | 8 +- dbms/src/Parsers/ParserCreateQuery.cpp | 130 +++- dbms/src/Parsers/ParserCreateQuery.h | 8 + dbms/src/Storages/StorageFactory.cpp | 16 + .../Storages/WindowView/StorageWindowView.cpp | 661 ++++++++++++++++++ .../Storages/WindowView/StorageWindowView.h | 141 ++++ .../WindowView/WindowViewBlockInputStream.h | 171 +++++ .../WindowViewBlocksBlockInputStream.h | 136 ++++ .../WindowView/WindowViewBlocksMetadata.h | 23 + .../WindowView/WindowViewProxyStorage.h | 70 ++ dbms/src/Storages/registerStorages.h | 2 + 24 files changed, 1432 insertions(+), 16 deletions(-) create mode 100644 dbms/src/Storages/WindowView/StorageWindowView.cpp create mode 100644 dbms/src/Storages/WindowView/StorageWindowView.h create mode 100644 dbms/src/Storages/WindowView/WindowViewBlockInputStream.h create mode 100644 dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h create mode 100644 dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h create mode 100644 dbms/src/Storages/WindowView/WindowViewProxyStorage.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 821cf3f6654..fcbe704f824 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -293,6 +293,7 @@ add_object_library(clickhouse_storages src/Storages) add_object_library(clickhouse_storages_distributed src/Storages/Distributed) add_object_library(clickhouse_storages_mergetree src/Storages/MergeTree) add_object_library(clickhouse_storages_liveview src/Storages/LiveView) +add_object_library(clickhouse_storages_windowview src/Storages/WindowView) add_object_library(clickhouse_client src/Client) add_object_library(clickhouse_formats src/Formats) add_object_library(clickhouse_processors src/Processors) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index dfb39f1624e..ea49aeba612 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -490,6 +490,8 @@ namespace ErrorCodes extern const int AUTHENTICATION_FAILED = 516; extern const int CANNOT_ASSIGN_ALTER = 517; + extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW = 666; + extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; extern const int STD_EXCEPTION = 1001; diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index ce3a8122ead..8dc1822de56 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -34,6 +34,7 @@ #define DEFAULT_MERGE_BLOCK_SIZE 8192 #define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 +#define DEFAULT_TEMPORARY_WINDOW_VIEW_TIMEOUT_SEC 5 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) #define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index ffc575e1eb6..f715b31f327 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -386,6 +386,8 @@ struct Settings : public SettingsCollection M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ + M(SettingBool, allow_experimental_window_view, false, "Enable WINDOW VIEW. Not mature enough.", 0) \ + M(SettingSeconds, temporary_window_view_timeout, DEFAULT_TEMPORARY_WINDOW_VIEW_TIMEOUT_SEC, "Timeout after which temporary window view is deleted.", 0) \ M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index b9a66f4899d..49bc1eac520 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include namespace DB @@ -82,6 +83,18 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( BlockIO io = interpreter.execute(); out = io.out; } + else if (dynamic_cast(dependent_table.get())) + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr(), true); + else if (dynamic_cast(dependent_table.get())) + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr(), true); + else + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr()); + + views.emplace_back(ViewInfo{std::move(query), database_table.first, database_table.second, std::move(out)}); + } else if (dynamic_cast(dependent_table.get())) out = std::make_shared(dependent_table, *views_context, ASTPtr(), true); else @@ -123,6 +136,10 @@ void PushingToViewsBlockOutputStream::write(const Block & block) { StorageLiveView::writeIntoLiveView(*live_view, block, context); } + else if (auto * window_view = dynamic_cast(storage.get())) + { + StorageWindowView::writeIntoWindowView(*window_view, block, context); + } else { if (output) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index 4ca866b006f..e8a7b4ace10 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -104,7 +104,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) create->replace_view = false; /// For views it is necessary to save the SELECT query itself, for the rest - on the contrary - if (!create->is_view && !create->is_materialized_view && !create->is_live_view) + if (!create->is_view && !create->is_materialized_view && !create->is_live_view && !create->is_window_view) create->select = nullptr; create->format = nullptr; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 850260dea91..3c651d6f108 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -482,7 +482,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const return; } - if (create.temporary) + if (create.temporary && !create.is_window_view) { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; @@ -512,6 +512,11 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const "Cannot CREATE a table AS " + qualified_name + ", it is a Live View", ErrorCodes::INCORRECT_QUERY); + if (as_create.is_window_view) + throw Exception( + "Cannot CREATE a table AS " + qualified_name + ", it is a Window View", + ErrorCodes::INCORRECT_QUERY); + if (as_create.is_dictionary) throw Exception( "Cannot CREATE a table AS " + qualified_name + ", it is a Dictionary", @@ -525,7 +530,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) { /// Temporary tables are created out of databases. - if (create.temporary && !create.database.empty()) + if (create.temporary && !create.database.empty() && !create.is_window_view) throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE); @@ -546,7 +551,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (!create.to_table.empty() && create.to_database.empty()) create.to_database = current_database; - if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view)) + if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view || create.is_window_view)) { AddDefaultDatabaseVisitor visitor(current_database); visitor.visit(*create.select); @@ -572,7 +577,7 @@ bool InterpreterCreateQuery::doCreateTable(const ASTCreateQuery & create, DatabasePtr database; const String & table_name = create.table; - bool need_add_to_database = !create.temporary; + bool need_add_to_database = !create.temporary || create.is_window_view; if (need_add_to_database) { database = context.getDatabase(create.database); @@ -647,7 +652,7 @@ BlockIO InterpreterCreateQuery::fillTableIfNeeded(const ASTCreateQuery & create) { /// If the query is a CREATE SELECT, insert the data into the table. if (create.select && !create.attach - && !create.is_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate)) + && !create.is_view && !create.is_live_view && !create.is_window_view && (!create.is_materialized_view || create.is_populate)) { auto insert = std::make_shared(); diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.cpp b/dbms/src/Interpreters/InterpreterWatchQuery.cpp index 40606899c53..8d92b555029 100644 --- a/dbms/src/Interpreters/InterpreterWatchQuery.cpp +++ b/dbms/src/Interpreters/InterpreterWatchQuery.cpp @@ -35,8 +35,8 @@ BlockInputStreamPtr InterpreterWatchQuery::executeImpl() BlockIO InterpreterWatchQuery::execute() { - if (!context.getSettingsRef().allow_experimental_live_view) - throw Exception("Experimental LIVE VIEW feature is not enabled (the setting 'allow_experimental_live_view')", ErrorCodes::SUPPORT_IS_DISABLED); + if (!context.getSettingsRef().allow_experimental_live_view && !context.getSettingsRef().allow_experimental_window_view) + throw Exception("Experimental LIVE VIEW or WINDOW VIEW feature is not enabled (the setting 'allow_experimental_live_view' or 'allow_experimental_window_view')", ErrorCodes::SUPPORT_IS_DISABLED); BlockIO res; const ASTWatchQuery & query = typeid_cast(*query_ptr); diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index ceb64aad6a3..10082197140 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -280,6 +280,10 @@ void ASTAlterCommand::formatImpl( { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); } + else if (type == ASTAlterCommand::WINDOW_VIEW_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); + } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } @@ -334,6 +338,8 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState if (is_live_view) settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE VIEW " << (settings.hilite ? hilite_none : ""); + else if (is_window_view) + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER WINDOW VIEW " << (settings.hilite ? hilite_none : ""); else settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index de36394a9c3..332fb2456fd 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -18,6 +18,8 @@ namespace DB * COMMENT_COLUMN col_name 'comment', * ALTER LIVE VIEW [db.]name_type * REFRESH + * ALTER WINDOW VIEW [db.]name_type + * REFRESH */ class ASTAlterCommand : public IAST @@ -57,6 +59,7 @@ public: NO_TYPE, LIVE_VIEW_REFRESH, + WINDOW_VIEW_REFRESH, }; Type type = NO_TYPE; @@ -186,6 +189,7 @@ class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCl { public: bool is_live_view{false}; /// true for ALTER LIVE VIEW + bool is_window_view{false}; /// true for ALTER WINDOW VIEW ASTAlterCommandList * command_list = nullptr; diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index ae72295d1de..5068fe7d5cf 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -224,6 +224,8 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat what = "MATERIALIZED VIEW"; if (is_live_view) what = "LIVE VIEW"; + if (is_window_view) + what = "WINDOW VIEW"; settings.ostr << (settings.hilite ? hilite_keyword : "") diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 23f3b63eb50..49f0c56e71d 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -58,6 +58,7 @@ public: bool is_view{false}; bool is_materialized_view{false}; bool is_live_view{false}; + bool is_window_view{false}; bool is_populate{false}; bool is_dictionary{false}; /// CREATE DICTIONARY bool replace_view{false}; /// CREATE OR REPLACE VIEW diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 6df5bd15b68..3f48d56b097 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -103,6 +103,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected else return false; } + else if (is_window_view) + { + if (s_refresh.ignore(pos, expected)) + { + command->type = ASTAlterCommand::WINDOW_VIEW_REFRESH; + } + else + return false; + } else { if (s_add_column.ignore(pos, expected)) @@ -511,7 +520,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command(is_live_view); + ParserAlterCommand p_command(is_live_view, is_window_view); do { @@ -561,19 +570,25 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_alter_table("ALTER TABLE"); ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); + ParserKeyword s_alter_window_view("ALTER WINDOW VIEW"); bool is_live_view = false; + bool is_window_view = false; if (!s_alter_table.ignore(pos, expected)) { - if (!s_alter_live_view.ignore(pos, expected)) - return false; - else + if (s_alter_live_view.ignore(pos, expected)) is_live_view = true; + else if (s_alter_window_view.ignore(pos, expected)) + is_window_view = true; + else + return false; } if (is_live_view) query->is_live_view = true; + else if(is_window_view) + query->is_window_view = true; if (!parseDatabaseAndTableName(pos, expected, query->database, query->table)) return false; @@ -586,7 +601,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } query->cluster = cluster_str; - ParserAlterCommandList p_command_list(is_live_view); + ParserAlterCommandList p_command_list(is_live_view, is_window_view); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 61a25b9b387..3d118d0bdb8 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -22,6 +22,8 @@ namespace DB * [UPDATE col_name = expr, ... WHERE ...] * ALTER LIVE VIEW [db.name] * [REFRESH] + * ALTER WINDOW VIEW [db.name] + * [REFRESH] */ class ParserAlterQuery : public IParserBase @@ -40,8 +42,9 @@ protected: public: bool is_live_view; + bool is_window_view; - ParserAlterCommandList(bool is_live_view_ = false) : is_live_view(is_live_view_) {} + ParserAlterCommandList(bool is_live_view_ = false, bool is_window_view_ = false) : is_live_view(is_live_view_), is_window_view(is_window_view_) {} }; @@ -53,8 +56,9 @@ protected: public: bool is_live_view; + bool is_window_view; - ParserAlterCommand(bool is_live_view_ = false) : is_live_view(is_live_view_) {} + ParserAlterCommand(bool is_live_view_ = false, bool is_window_view_ = false) : is_live_view(is_live_view_), is_window_view(is_window_view_) {} }; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 0a03a5ca342..04a7228b513 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -615,6 +615,132 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e return true; } +bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_create("CREATE"); + ParserKeyword s_temporary("TEMPORARY"); + ParserKeyword s_attach("ATTACH"); + ParserKeyword s_if_not_exists("IF NOT EXISTS"); + ParserKeyword s_as("AS"); + ParserKeyword s_view("VIEW"); + ParserKeyword s_window("WINDOW"); + ParserToken s_dot(TokenType::Dot); + ParserToken s_lparen(TokenType::OpeningRoundBracket); + ParserToken s_rparen(TokenType::ClosingRoundBracket); + ParserStorage storage_p; + ParserIdentifier name_p; + ParserTablePropertiesDeclarationList table_properties_p; + ParserSelectWithUnionQuery select_p; + + ASTPtr database; + ASTPtr table; + ASTPtr columns_list; + ASTPtr to_database; + ASTPtr to_table; + ASTPtr storage; + ASTPtr as_database; + ASTPtr as_table; + ASTPtr select; + + String cluster_str; + bool attach = false; + bool if_not_exists = false; + bool is_temporary = false; + + if (!s_create.ignore(pos, expected)) + { + if (s_attach.ignore(pos, expected)) + attach = true; + else + return false; + } + + if (s_temporary.ignore(pos, expected)) + { + is_temporary = true; + } + + if (!s_window.ignore(pos, expected)) + return false; + + if (!s_view.ignore(pos, expected)) + return false; + + if (s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!name_p.parse(pos, table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + database = table; + if (!name_p.parse(pos, table, expected)) + return false; + } + + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + // TO [db.]table + if (ParserKeyword{"TO"}.ignore(pos, expected)) + { + if (!name_p.parse(pos, to_table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + to_database = to_table; + if (!name_p.parse(pos, to_table, expected)) + return false; + } + } + + /// Optional - a list of columns can be specified. It must fully comply with SELECT. + if (s_lparen.ignore(pos, expected)) + { + if (!table_properties_p.parse(pos, columns_list, expected)) + return false; + + if (!s_rparen.ignore(pos, expected)) + return false; + } + + /// AS SELECT ... + if (!s_as.ignore(pos, expected)) + return false; + + if (!select_p.parse(pos, select, expected)) + return false; + + + auto query = std::make_shared(); + node = query; + + query->attach = attach; + query->if_not_exists = if_not_exists; + query->is_window_view = true; + query->temporary = is_temporary; + + tryGetIdentifierNameInto(database, query->database); + tryGetIdentifierNameInto(table, query->table); + query->cluster = cluster_str; + + tryGetIdentifierNameInto(to_database, query->to_database); + tryGetIdentifierNameInto(to_table, query->to_table); + + query->set(query->columns_list, columns_list); + + tryGetIdentifierNameInto(as_database, query->as_database); + tryGetIdentifierNameInto(as_table, query->as_table); + query->set(query->select, select); + + return true; +} + bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_create("CREATE"); @@ -916,12 +1042,14 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserCreateViewQuery view_p; ParserCreateDictionaryQuery dictionary_p; ParserCreateLiveViewQuery live_view_p; + ParserCreateWindowViewQuery window_view_p; return table_p.parse(pos, node, expected) || database_p.parse(pos, node, expected) || view_p.parse(pos, node, expected) || dictionary_p.parse(pos, node, expected) - || live_view_p.parse(pos, node, expected); + || live_view_p.parse(pos, node, expected) + || window_view_p.parse(pos, node, expected); } } diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 6ec55e42ec8..e2ee27dbf41 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -320,6 +320,14 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] AS SELECT ... +class ParserCreateWindowViewQuery : public IParserBase +{ +protected: + const char * getName() const { return "CREATE WINDOW VIEW query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + /// CREATE|ATTACH DATABASE db [ENGINE = engine] class ParserCreateDatabaseQuery : public IParserBase { diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 8008b9fe39d..2c2486c6f53 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -68,6 +68,14 @@ StoragePtr StorageFactory::get( name = "LiveView"; } + else if (query.is_window_view) + { + + if (query.storage) + throw Exception("Specifying ENGINE is not allowed for a WindowView", ErrorCodes::INCORRECT_QUERY); + + name = "WindowView"; + } else { /// Check for some special types, that are not allowed to be stored in tables. Example: NULL data type. @@ -112,6 +120,14 @@ StoragePtr StorageFactory::get( "Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement", ErrorCodes::INCORRECT_QUERY); } + else if (name == "WindowView") + { + throw Exception( + "Direct creation of tables with ENGINE WindowView is not supported, use CREATE WINDOW VIEW statement", + ErrorCodes::INCORRECT_QUERY); + } + } + } auto it = storages.find(name); if (it == storages.end()) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp new file mode 100644 index 00000000000..163bbf75d70 --- /dev/null +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -0,0 +1,661 @@ +#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 +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; + extern const int TABLE_WAS_NOT_DROPPED; + extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW; + extern const int SUPPORT_IS_DISABLED; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + const auto RESCHEDULE_MS = 500; + + class ParserStageMergeableOneMatcher + { + public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + ASTPtr window_function; + String window_column_name; + // String window_column_name_or_alias; + bool is_tumble = false; + bool is_hop = false; + }; + + static bool needChildVisit(ASTPtr & node, const ASTPtr &) + { + if (node->as()) + return false; + return true; + } + + static void visit(ASTPtr & ast, Data & data) + { + if (const auto * t = ast->as()) + visit(*t, ast, data); + } + + private: + static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data & data) + { + if (node.name == "TUMBLE") + { + if(!data.window_function) + { + data.is_tumble = true; + data.window_column_name = node.getColumnName(); + data.window_function = node.clone(); + } + else if (serializeAST(node) != serializeAST(*data.window_function)) + throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + } + else if (node.name == "HOP") + { + if(!data.window_function) + { + data.is_hop = true; + data.window_function = node.clone(); + auto ptr_ = node.clone(); + std::static_pointer_cast(ptr_)->setAlias(""); + auto arrayJoin = makeASTFunction("arrayJoin", ptr_); + arrayJoin->alias = node.alias; + data.window_column_name = arrayJoin->getColumnName(); + node_ptr = arrayJoin; + } + else if(serializeAST(node) != serializeAST(*data.window_function)) + throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + } + } + }; +} + +static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) +{ + auto db_and_table = getDatabaseAndTable(query, 0); + ASTPtr subquery = extractTableExpression(query, 0); + + if (!db_and_table && !subquery) + return; + + if (db_and_table) + { + select_table_name = db_and_table->table; + + if (db_and_table->database.empty()) + { + db_and_table->database = select_database_name; + AddDefaultDatabaseVisitor visitor(select_database_name); + visitor.visit(query); + } + else + select_database_name = db_and_table->database; + } + else if (auto * ast_select = subquery->as()) + { + if (ast_select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for WINDOW VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + + auto & inner_query = ast_select->list_of_selects->children.at(0); + + extractDependentTable(inner_query->as(), select_database_name, select_table_name); + } + else + throw Exception( + "Logical error while creating StorageWindowView." + " Could not retrieve table name from select query.", + DB::ErrorCodes::LOGICAL_ERROR); +} + +void StorageWindowView::checkTableCanBeDropped() const +{ + Dependencies dependencies = global_context.getDependencies(database_name, table_name); + if (!dependencies.empty()) + { + DatabaseAndTableName database_and_table_name = dependencies.front(); + throw Exception( + "Table has dependency " + database_and_table_name.first + "." + database_and_table_name.second, + ErrorCodes::TABLE_WAS_NOT_DROPPED); + } +} + +void StorageWindowView::drop(TableStructureWriteLockHolder &) +{ + global_context.removeDependency( + DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); + + std::lock_guard lock(mutex); + is_dropped = true; + condition.notify_all(); +} + +inline void StorageWindowView::flushToTable() +{ + //write into dependent table + StoragePtr target_table = getTargetTable(); + auto _blockInputStreamPtr = getNewBlocksInputStreamPtr(); + auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); + auto stream = target_table->write(getInnerQuery(), global_context); + copyData(*_blockInputStreamPtr, *stream); +} + +UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) +{ + switch (window_kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: \ + { \ + UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ + return AddTime::execute(start, window_num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second); + CASE_WINDOW_KIND(Minute); + CASE_WINDOW_KIND(Hour); + CASE_WINDOW_KIND(Day); + CASE_WINDOW_KIND(Week); + CASE_WINDOW_KIND(Month); + CASE_WINDOW_KIND(Quarter); + CASE_WINDOW_KIND(Year); +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); +} + +void StorageWindowView::threadFuncToTable() +{ + while (!shutdown_called && has_target_table) + { + std::unique_lock lock(flushTableMutex); + UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); + UInt64 w_end = static_cast(getWindowUpperBound((UInt32)(timestamp_usec / 1000000))) * 1000000; + condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); + try + { + if (refreshBlockStatus()) + flushToTable(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + break; + } + } + if (!shutdown_called) + toTableTask->scheduleAfter(RESCHEDULE_MS); +} + +BlockInputStreams StorageWindowView::watch( + const Names & /*column_names*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + if (has_target_table) + throw Exception("WATCH query is disabled for " + getName() + " when constructed with 'TO' clause.", ErrorCodes::INCORRECT_QUERY); + + if(active_ptr.use_count() > 1) + throw Exception("WATCH query is already attached, WINDOW VIEW only supports attaching one watch query.", ErrorCodes::INCORRECT_QUERY); + + ASTWatchQuery & query = typeid_cast(*query_info.query); + + bool has_limit = false; + UInt64 limit = 0; + + if (query.limit_length) + { + has_limit = true; + limit = safeGet(typeid_cast(*query.limit_length).value); + } + + auto reader = std::make_shared( + std::static_pointer_cast(shared_from_this()), + active_ptr, + has_limit, + limit, + context.getSettingsRef().temporary_window_view_timeout.totalSeconds()); + + { + std::lock_guard no_users_thread_lock(no_users_thread_mutex); + if (no_users_thread.joinable()) + { + std::lock_guard lock(no_users_thread_wakeup_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.notify_one(); + } + } + + processed_stage = QueryProcessingStage::Complete; + + return {reader}; +} + +Block StorageWindowView::getHeader() const +{ + if (!sample_block) + { + auto storage = global_context.getTable(select_database_name, select_table_name); + sample_block = InterpreterSelectQuery( + getInnerQuery(), global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)) + .getSampleBlock(); + for (size_t i = 0; i < sample_block.columns(); ++i) + sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); + } + + return sample_block; +} + +StoragePtr & StorageWindowView::getParentStorage() +{ + if (!parent_storage) + parent_storage = global_context.getTable(getSelectDatabaseName(), getSelectTableName()); + return parent_storage; +} + +StorageWindowView::StorageWindowView( + const String & table_name_, + const String & database_name_, + Context & local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns_) + : table_name(table_name_) + , database_name(database_name_) + , global_context(local_context.getGlobalContext()) + , time_zone(DateLUT::instance()) + , log(&Poco::Logger::get("StorageWindowView")) +{ + setColumns(columns_); + + if (!query.select) + throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); + + /// Default value, if only table name exist in the query + select_database_name = local_context.getCurrentDatabase(); + if (query.select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + + auto inner_query_ = query.select->list_of_selects->children.at(0); + + ASTSelectQuery & select_query = typeid_cast(*inner_query_); + extractDependentTable(select_query, select_database_name, select_table_name); + inner_query = innerQueryParser(select_query); + + /// If the table is not specified - use the table `system.one` + if (select_table_name.empty()) + { + select_database_name = "system"; + select_table_name = "one"; + } + + global_context.addDependency( + DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); + + if (!query.to_table.empty()) + { + has_target_table = true; + target_database_name = query.to_database; + target_table_name = query.to_table; + } + + is_temporary = query.temporary; + temporary_window_view_timeout = local_context.getSettingsRef().temporary_window_view_timeout.totalSeconds(); + + mergeable_blocks = std::make_shared>(); + + active_ptr = std::make_shared(true); + + toTableTask = global_context.getSchedulePool().createTask(log->name(), [this] { threadFuncToTable(); }); + toTableTask->deactivate(); +} + + +ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) +{ + if (!query.groupBy()) + throw Exception("GROUP BY query is required for " + getName(), ErrorCodes::INCORRECT_QUERY); + + // parse stage mergeable + ASTPtr result = query.clone(); + ASTPtr expr_list = result; + ParserStageMergeableOneMatcher::Data stageMergeableOneData; + ParserStageMergeableOneMatcher::Visitor(stageMergeableOneData).visit(expr_list); + if (!stageMergeableOneData.is_tumble && !stageMergeableOneData.is_hop) + throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); + window_column_name = stageMergeableOneData.window_column_name; + + // parser window function + ASTFunction & window_function = typeid_cast(*stageMergeableOneData.window_function); + ASTExpressionList &window_function_args = typeid_cast(*window_function.arguments); + const auto & children = window_function_args.children; + const auto & interval_p1 = std::static_pointer_cast(children.at(1)); + if(!interval_p1 || !startsWith(interval_p1->name, "toInterval")) + throw Exception("Illegal type of last argument of function " + interval_p1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + + String interval_str = interval_p1->name.substr(10); + if(interval_str == "Second") + window_kind = IntervalKind::Second; + else if(interval_str == "Minute") + window_kind = IntervalKind::Minute; + else if(interval_str == "Hour") + window_kind = IntervalKind::Hour; + else if(interval_str == "Day") + window_kind = IntervalKind::Day; + else if(interval_str == "Week") + window_kind = IntervalKind::Week; + else if(interval_str == "Month") + window_kind = IntervalKind::Month; + else if(interval_str == "Quarter") + window_kind = IntervalKind::Quarter; + else if(interval_str == "Year") + window_kind = IntervalKind::Year; + + const auto & interval_units_p1 = std::static_pointer_cast(interval_p1->children.front()->children.front()); + + window_num_units = stoi(interval_units_p1->value.get()); + return result; +} + +void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) +{ + BlockInputStreams streams = {std::make_shared(block)}; + auto window_proxy_storage = std::make_shared( + window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); + InterpreterSelectQuery select_block( + window_view.getInnerQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); + + auto data_mergeable_stream = std::make_shared(select_block.execute().in); + + BlocksListPtr new_mergeable_blocks = std::make_shared(); + while (Block block_ = data_mergeable_stream->read()) + { + const ColumnTuple * column_tuple = checkAndGetColumn(block_.getByName(window_view.window_column_name).column.get()); + block_.insert( + {ColumnUInt8::create(block_.rows(), WINDOW_VIEW_FIRE_STATUS::WAITING), std::make_shared(), "____fire_status"}); + block_.insert({column_tuple->getColumnPtr(1), std::make_shared(), "____w_end"}); + new_mergeable_blocks->push_back(block_); + } + + if (new_mergeable_blocks->empty()) + return; + + { + std::unique_lock lock(window_view.mutex); + window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); + } + window_view.condition.notify_all(); +} + +StoragePtr StorageWindowView::getTargetTable() const +{ + return global_context.getTable(target_database_name, target_table_name); +} + +StoragePtr StorageWindowView::tryGetTargetTable() const +{ + return global_context.tryGetTable(target_database_name, target_table_name); +} + +void StorageWindowView::startup() +{ + // Start the working thread + if (has_target_table) + toTableTask->activateAndSchedule(); + startNoUsersThread(temporary_window_view_timeout); +} + +void StorageWindowView::shutdown() +{ + bool expected = false; + if (!shutdown_called.compare_exchange_strong(expected, true)) + return; + toTableTask->deactivate(); + { + std::lock_guard no_users_thread_lock(no_users_thread_mutex); + if (no_users_thread.joinable()) + { + { + std::lock_guard lock(no_users_thread_wakeup_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.notify_one(); + } + } + } +} + +StorageWindowView::~StorageWindowView() +{ + shutdown(); + + { + std::lock_guard lock(no_users_thread_mutex); + if (no_users_thread.joinable()) + no_users_thread.detach(); + } +} + +bool StorageWindowView::refreshBlockStatus() +{ + UInt32 timestamp_now = std::time(nullptr); + for (BlocksListPtr mergeable_block : *mergeable_blocks) + { + for (Block & block : *mergeable_block) + { + auto & col_wend = block.getByName("____w_end").column; + const auto & wend_data = static_cast(*col_wend).getData(); + auto & col_status = block.getByName("____fire_status").column; + const auto & col_status_data = static_cast(*col_status).getData(); + + for (size_t i = 0; i < col_wend->size(); ++i) + { + if (wend_data[i] < timestamp_now && col_status_data[i] == WINDOW_VIEW_FIRE_STATUS::WAITING) + return true; + } + } + } + return false; +} + +BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() +{ + if (mergeable_blocks->empty()) + return {std::make_shared(getHeader())}; + { + std::lock_guard lock(mutex); + //delete fired blocks + for (BlocksListPtr mergeable_block : *mergeable_blocks) + { + mergeable_block->remove_if([](Block & block_) { + auto & column_ = block_.getByName("____fire_status").column; + const auto & data = static_cast(*column_).getData(); + for (size_t i = 0; i < column_->size(); ++i) + { + if (data[i] != WINDOW_VIEW_FIRE_STATUS::RETIRED) + return false; + } + return true; + }); + } + mergeable_blocks->remove_if([](BlocksListPtr & ptr) { return ptr->size() == 0; }); + + if (mergeable_blocks->empty()) + return {std::make_shared(getHeader())}; + + // mark blocks can be fired + UInt32 timestamp_now = std::time(nullptr); + for (BlocksListPtr mergeable_block : *mergeable_blocks) + { + for (Block & block : *mergeable_block) + { + auto & col_wend = block.getByName("____w_end").column; + const auto & wend_data = static_cast(*col_wend).getData(); + auto & col_status = block.getByName("____fire_status").column; + auto col_status_mutable = col_status->assumeMutable(); + auto & col_status_data = static_cast(*col_status_mutable).getData(); + + for (size_t i = 0; i < col_wend->size(); ++i) + { + if (wend_data[i] < timestamp_now && col_status_data[i] == WINDOW_VIEW_FIRE_STATUS::WAITING) + col_status_data[i] = WINDOW_VIEW_FIRE_STATUS::READY; + } + col_status = std::move(col_status_mutable); + } + } + } + + BlockInputStreams from; + auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); //TODO: 改为全局 + BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, mutex); + from.push_back(std::move(stream)); + auto proxy_storage = std::make_shared( + getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + return data; +} + +BlocksPtr StorageWindowView::getNewBlocks() +{ + auto res = getNewBlocksInputStreamPtr(); + BlocksPtr blocks = std::make_shared(); + while (Block this_block = res->read()) + blocks->push_back(this_block); + return blocks; +} + +void StorageWindowView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) +{ + bool drop_table = false; + + if (storage->shutdown_called) + return; + + { + while (1) + { + std::unique_lock lock(storage->no_users_thread_wakeup_mutex); + if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) + { + storage->no_users_thread_wakeup = false; + if (storage->shutdown_called) + return; + if (!storage->global_context.getDependencies(storage->database_name, storage->table_name).empty()) + continue; + drop_table = true; + } + break; + } + } + + if (drop_table) + { + if (storage->global_context.tryGetTable(storage->database_name, storage->table_name)) + { + try + { + /// We create and execute `drop` query for this table + auto drop_query = std::make_shared(); + drop_query->database = storage->database_name; + drop_query->table = storage->table_name; + drop_query->kind = ASTDropQuery::Kind::Drop; + ASTPtr ast_drop_query = drop_query; + InterpreterDropQuery drop_interpreter(ast_drop_query, storage->global_context); + drop_interpreter.execute(); + } + catch (...) + { + } + } + } +} + +void StorageWindowView::startNoUsersThread(const UInt64 & timeout) +{ + bool expected = false; + if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) + return; + + if (is_temporary) + { + std::lock_guard no_users_thread_lock(no_users_thread_mutex); + + if (shutdown_called) + return; + + if (no_users_thread.joinable()) + { + { + std::lock_guard lock(no_users_thread_wakeup_mutex); + no_users_thread_wakeup = true; + no_users_thread_condition.notify_one(); + } + no_users_thread.join(); + } + { + std::lock_guard lock(no_users_thread_wakeup_mutex); + no_users_thread_wakeup = false; + } + if (!is_dropped) + no_users_thread = std::thread(&StorageWindowView::noUsersThread, + std::static_pointer_cast(shared_from_this()), timeout); + } + + start_no_users_thread_called = false; +} + + +void registerStorageWindowView(StorageFactory & factory) +{ + factory.registerStorage("WindowView", [](const StorageFactory::Arguments & args) { + if (!args.attach && !args.local_context.getSettingsRef().allow_experimental_window_view) + throw Exception( + "Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')", + ErrorCodes::SUPPORT_IS_DISABLED); + return StorageWindowView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); + }); +} +} \ No newline at end of file diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h new file mode 100644 index 00000000000..534cc1afa1e --- /dev/null +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -0,0 +1,141 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +class IAST; +using ASTPtr = std::shared_ptr; +using BlocksListPtr = std::shared_ptr; +using BlocksListPtrs = std::shared_ptr>; + +class StorageWindowView : public ext::shared_ptr_helper, public IStorage +{ + friend struct ext::shared_ptr_helper; + friend class WindowViewBlockInputStream; + +public: + ~StorageWindowView() override; + String getName() const override { return "WindowView"; } + String getTableName() const override { return table_name; } + String getDatabaseName() const override { return database_name; } + String getSelectDatabaseName() const { return select_database_name; } + String getSelectTableName() const { return select_table_name; } + + ASTPtr getInnerQuery() const { return inner_query->clone(); } + + /// It is passed inside the query and solved at its level. + bool supportsSampling() const override { return true; } + bool supportsFinal() const override { return true; } + + bool isTemporary() { return is_temporary; } + + /// Check we have any active readers + /// must be called with mutex locked + bool hasActiveUsers() { return active_ptr.use_count() > 1; } + + /// Background thread for temporary tables + /// which drops this table if there are no users + void startNoUsersThread(const UInt64 & timeout); + std::mutex no_users_thread_wakeup_mutex; + bool no_users_thread_wakeup{false}; + std::condition_variable no_users_thread_condition; + + void checkTableCanBeDropped() const override; + + StoragePtr getTargetTable() const; + StoragePtr tryGetTargetTable() const; + + void drop(TableStructureWriteLockHolder &) override; + + void startup() override; + void shutdown() override; + + BlockInputStreams watch( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + BlocksListPtrs getMergeableBlocksList() { return mergeable_blocks; } + std::shared_ptr getActivePtr() { return active_ptr; } + + /// Read new data blocks that store query result + BlockInputStreamPtr getNewBlocksInputStreamPtr(); + + BlocksPtr getNewBlocks(); + + Block getHeader() const; + + StoragePtr & getParentStorage(); + + static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); + + ASTPtr innerQueryParser(ASTSelectQuery & inner_query); + + inline UInt32 getWindowUpperBound(UInt32 time_sec); + +private: + String select_database_name; + String select_table_name; + String table_name; + String database_name; + ASTPtr inner_query; + String window_column_name; + String window_end_column_alias; + Context & global_context; + StoragePtr parent_storage; + bool is_temporary{false}; + mutable Block sample_block; + + /// Mutex for the blocks and ready condition + std::mutex mutex; + std::mutex flushTableMutex; + /// New blocks ready condition to broadcast to readers + /// that new blocks are available + std::condition_variable condition; + + /// Active users + std::shared_ptr active_ptr; + BlocksListPtrs mergeable_blocks; + + IntervalKind::Kind window_kind; + Int64 window_num_units; + const DateLUTImpl & time_zone; + + std::atomic has_target_table{false}; + String target_database_name; + String target_table_name; + + static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); + inline void flushToTable(); + void threadFuncToTable(); + bool refreshBlockStatus(); + std::mutex no_users_thread_mutex; + std::thread no_users_thread; + std::atomic shutdown_called{false}; + std::atomic start_no_users_thread_called{false}; + UInt64 temporary_window_view_timeout; + + Poco::Logger * log; + Poco::Timestamp timestamp; + + BackgroundSchedulePool::TaskHolder toTableTask; + BackgroundSchedulePool::TaskHolder toTableTask_preprocess; + + StorageWindowView( + const String & table_name_, + const String & database_name_, + Context & local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns); +}; +} \ No newline at end of file diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h new file mode 100644 index 00000000000..1345419189c --- /dev/null +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -0,0 +1,171 @@ +#pragma once + +#include + + +namespace DB +{ +/** Implements WINDOW VIEW table WATCH input stream. + * Keeps stream alive by outputing blocks with no rows + * based on period specified by the heartbeat interval. + */ +class WindowViewBlockInputStream : public IBlockInputStream +{ +public: + ~WindowViewBlockInputStream() override + { + /// Start storage no users thread + /// if we are the last active user + if (!storage->is_dropped) + storage->startNoUsersThread(temporary_window_view_timeout_sec); + } + + WindowViewBlockInputStream( + std::shared_ptr storage_, + std::shared_ptr active_ptr_, + const bool has_limit_, + const UInt64 limit_, + // const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_window_view_timeout_sec_) + : storage(std::move(storage_)) + // , active_ptr(std::move(active_ptr_)) + , active(std::move(active_ptr_)) + , has_limit(has_limit_) + , limit(limit_) + , temporary_window_view_timeout_sec(temporary_window_view_timeout_sec_) + { + /// grab active pointer + // active = active_ptr.lock(); + } + + String getName() const override { return "WindowViewBlockInputStream"; } + + void cancel(bool kill) override + { + if (isCancelled() || storage->is_dropped) + return; + IBlockInputStream::cancel(kill); + std::lock_guard lock(storage->mutex); + storage->condition.notify_all(); + } + + Block getHeader() const override { return storage->getHeader(); } + +protected: + Block readImpl() override + { + /// try reading + return tryReadImpl(); + } + + /** tryRead method attempts to read a block in either blocking + * or non-blocking mode. If blocking is set to false + * then method return empty block with flag set to false + * to indicate that method would block to get the next block. + */ + Block tryReadImpl() + { + Block res; + + if (has_limit && num_updates == static_cast(limit)) + { + return Block(); + } + /// If blocks were never assigned get blocks + if (!blocks) + { + if (!active) + return Block(); + // std::unique_lock lock(storage->mutex); + blocks = storage->getNewBlocks(); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + if (isCancelled() || storage->is_dropped) + { + return Block(); + } + + if (it == end) + { + if (!active) + return Block(); + if (storage->refreshBlockStatus()) + { + // std::unique_lock lock(storage->mutex); + blocks = storage->getNewBlocks(); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + /// No new blocks available wait for new ones + else + { + std::unique_lock lock(storage->flushTableMutex); + // std::unique_lock lock(storage->mutex); + if (!end_of_blocks) + { + end_of_blocks = true; + return getHeader(); + } + while (true) + { + UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); + UInt64 w_end = static_cast(storage->getWindowUpperBound((UInt32)(timestamp_usec / 1000000))) * 1000000; + storage->condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); + + if (isCancelled() || storage->is_dropped) + { + std::cout << "AAAAAAAAAAAAAAAA Cancelled:" << std::endl; + return Block(); + } + if (storage->refreshBlockStatus()) + { + break; + } + else + { + return getHeader(); + } + } + } + return tryReadImpl(); + } + + res = *it; + + ++it; + + if (it == end) + { + end_of_blocks = false; + num_updates += 1; + } + + // last_event_timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); + // last_event_timestamp_usec = timestamp_usec; + return res; + } + +private: + std::shared_ptr storage; + // std::weak_ptr active_ptr; + std::shared_ptr active; + BlocksPtr blocks; + + // std::mutex mutex; + Blocks::iterator it; + Blocks::iterator end; + Blocks::iterator begin; + const bool has_limit; + const UInt64 limit; + Int64 num_updates = 0; + bool end_of_blocks = false; + // UInt64 heartbeat_interval_usec; + UInt64 temporary_window_view_timeout_sec; + // UInt64 last_event_timestamp_usec = 0; + // UInt64 reschedule_us; +}; +} diff --git a/dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h new file mode 100644 index 00000000000..036cb7b921d --- /dev/null +++ b/dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h @@ -0,0 +1,136 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +/** A stream of blocks from a shared vector of blocks with metadata + */ +using BlocksListPtr = std::shared_ptr; +using BlocksListPtrs = std::shared_ptr>; + +class WindowViewBlocksBlockInputStream : public IBlockInputStream +{ +public: + /// Acquires shared ownership of the blocks vector + WindowViewBlocksBlockInputStream(BlocksListPtrs blocks_ptr_, Block header_, std::mutex &mutex_) + : blocks(blocks_ptr_), mutex(mutex_), header(std::move(header_)) + { + it_blocks = blocks->begin(); + end_blocks = blocks->end(); + if (it_blocks != end_blocks) + { + it = (*it_blocks)->begin(); + end = (*it_blocks)->end(); + } + } + + String getName() const override { return "MetadataBlocks"; } + + Block getHeader() const override { return header; } + +protected: + Block readImpl() override + { + while (it_blocks != end_blocks) + { + while (it != end) + { + Block &block = *it; + size_t columns = block.columns(); + + //generate filter + auto & column_status = block.getByName("____fire_status").column; + auto column_status_mutable = column_status->assumeMutable(); + IColumn::Filter filter(column_status->size(), 0); + auto & data = static_cast(*column_status_mutable).getData(); + { + std::unique_lock lock(mutex); + for (size_t i = 0; i < column_status->size(); ++i) + { + if (data[i] == WINDOW_VIEW_FIRE_STATUS::READY) + { + filter[i] = 1; + data[i] = WINDOW_VIEW_FIRE_STATUS::RETIRED; + } + } + } + + //filter block + /** Let's find out how many rows will be in result. + * To do this, we filter out the first non-constant column + * or calculate number of set bytes in the filter. + */ + size_t first_non_constant_column = 0; + for (size_t i = 0; i < columns; ++i) + { + if (!isColumnConst(*block.safeGetByPosition(i).column)) + { + first_non_constant_column = i; + break; + } + } + + Block res = block.cloneEmpty(); + size_t filtered_rows = 0; + { + ColumnWithTypeAndName & current_column = block.safeGetByPosition(first_non_constant_column); + ColumnWithTypeAndName & filtered_column = res.safeGetByPosition(first_non_constant_column); + filtered_column.column = current_column.column->filter(filter, -1); + filtered_rows = filtered_column.column->size(); + } + + /// If the current block is completely filtered out, let's move on to the next one. + if (filtered_rows == 0) + { + ++it; + continue; + } + + /// If all the rows pass through the filter. + if (filtered_rows == filter.size()) + { + ++it; + return block; + } + + /// Filter the rest of the columns. + for (size_t i = 0; i < columns; ++i) + { + ColumnWithTypeAndName & current_column = block.safeGetByPosition(i); + ColumnWithTypeAndName & filtered_column = res.safeGetByPosition(i); + + if (i == first_non_constant_column) + continue; + + if (isColumnConst(*current_column.column)) + filtered_column.column = current_column.column->cut(0, filtered_rows); + else + filtered_column.column = current_column.column->filter(filter, -1); + } + ++it; + return res; + } + ++it_blocks; + if (it_blocks != end_blocks) + { + it = (*it_blocks)->begin(); + end = (*it_blocks)->end(); + } + } + return Block(); + } + +private: + BlocksListPtrs blocks; + std::list::iterator it_blocks; + std::list::iterator end_blocks; + BlocksList::iterator it; + BlocksList::iterator end; + std::mutex & mutex; + Block header; +}; +} diff --git a/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h b/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h new file mode 100644 index 00000000000..e3210a38f70 --- /dev/null +++ b/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +namespace DB +{ +enum WINDOW_VIEW_FIRE_STATUS +{ + WAITING, + READY, + RETIRED +}; + +// struct WindowViewBlocksMetadata +// { +// // String hash; +// // std::set window_ids; +// UInt64 max_window_id = 0; +// WINDOW_VIEW_FIRE_STATUS fired_status = WINDOW_VIEW_FIRE_STATUS::WAITING; +// // bool retired; +// }; + +} \ No newline at end of file diff --git a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h new file mode 100644 index 00000000000..79b069c3745 --- /dev/null +++ b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h @@ -0,0 +1,70 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +class WindowViewProxyStorage : public IStorage +{ +public: + WindowViewProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) + : storage(std::move(storage_)) + , streams(std::move(streams_)) + , to_stage(to_stage_) {} + +public: + std::string getName() const override { return "WindowViewProxyStorage(" + storage->getName() + ")"; } + std::string getTableName() const override { return storage->getTableName(); } + + bool isRemote() const override { return storage->isRemote(); } + bool supportsSampling() const override { return storage->supportsSampling(); } + bool supportsFinal() const override { return storage->supportsFinal(); } + bool supportsPrewhere() const override { return storage->supportsPrewhere(); } + bool supportsReplication() const override { return storage->supportsReplication(); } + bool supportsDeduplication() const override { return storage->supportsDeduplication(); } + + QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; } + + BlockInputStreams read( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) override + { + return streams; + } + + bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); } + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override { return storage->mayBenefitFromIndexForIn(left_in_operand, query_context); } + ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); } + ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); } + ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); } + ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); } + Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); } + Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); } + Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); } + Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); } + Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); } + + const ColumnsDescription & getColumns() const override { return storage->getColumns(); } + + void setColumns(ColumnsDescription columns_) override { return storage->setColumns(columns_); } + + NameAndTypePair getColumn(const String & column_name) const override { return storage->getColumn(column_name); } + + bool hasColumn(const String & column_name) const override { return storage->hasColumn(column_name); } + +private: + StoragePtr storage; + BlockInputStreams streams; + QueryProcessingStage::Enum to_stage; + NameAndTypePair column_end; +}; +} diff --git a/dbms/src/Storages/registerStorages.h b/dbms/src/Storages/registerStorages.h index 63a758f5b38..55ead6bbfcb 100644 --- a/dbms/src/Storages/registerStorages.h +++ b/dbms/src/Storages/registerStorages.h @@ -24,6 +24,8 @@ void registerStorageView(StorageFactory & factory); void registerStorageMaterializedView(StorageFactory & factory); void registerStorageLiveView(StorageFactory & factory); void registerStorageGenerateRandom(StorageFactory & factory); +void registerStorageGenerate(StorageFactory & factory); +void registerStorageWindowView(StorageFactory & factory); #if USE_AWS_S3 void registerStorageS3(StorageFactory & factory); From d85ca7576c9d8c3dafac834056ee5756728a1cc6 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 14 Jan 2020 11:08:54 +0800 Subject: [PATCH 002/609] add window function --- dbms/src/Functions/FunctionsWindow.cpp | 15 + dbms/src/Functions/FunctionsWindow.h | 598 +++++++++++++++++++++++ dbms/src/Functions/registerFunctions.cpp | 2 + 3 files changed, 615 insertions(+) create mode 100644 dbms/src/Functions/FunctionsWindow.cpp create mode 100644 dbms/src/Functions/FunctionsWindow.h diff --git a/dbms/src/Functions/FunctionsWindow.cpp b/dbms/src/Functions/FunctionsWindow.cpp new file mode 100644 index 00000000000..968002f7b79 --- /dev/null +++ b/dbms/src/Functions/FunctionsWindow.cpp @@ -0,0 +1,15 @@ +#include +#include + +namespace DB +{ +void registerFunctionsWindow(FunctionFactory& factory) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} +} \ No newline at end of file diff --git a/dbms/src/Functions/FunctionsWindow.h b/dbms/src/Functions/FunctionsWindow.h new file mode 100644 index 00000000000..3b1757db853 --- /dev/null +++ b/dbms/src/Functions/FunctionsWindow.h @@ -0,0 +1,598 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "IFunctionImpl.h" + +namespace DB +{ + +/** Window functions: + * + * TUMBLE(time_attr, interval) + * + * TUMBLE_START(window_id) + * + * TUMBLE_START(time_attr, interval) + * + * TUMBLE_END(window_id) + * + * TUMBLE_END(time_attr, interval) + * + * HOP(time_attr, hop_interval, window_interval) + * + * HOP_START(window_id) + * + * HOP_START(time_attr, hop_interval, window_interval) + * + * HOP_END(window_id) + * + * HOP_END(time_attr, hop_interval, window_interval) + * + */ +enum FunctionName +{ + TUMBLE, + TUMBLE_START, + TUMBLE_END, + HOP, + HOP_START, + HOP_END +}; +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; +} + +template +struct ToStartOfTransform; + +#define TRANSFORM_DATE(INTERVAL_KIND) \ + template <> \ + struct ToStartOfTransform \ + { \ + static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ + { \ + return time_zone.toStartOf##INTERVAL_KIND##Interval(time_zone.toDayNum(t), delta); \ + } \ + }; + TRANSFORM_DATE(Year) + TRANSFORM_DATE(Quarter) + TRANSFORM_DATE(Month) + TRANSFORM_DATE(Week) + TRANSFORM_DATE(Day) +#undef TRANSFORM_DATE + +#define TRANSFORM_TIME(INTERVAL_KIND) \ + template <> \ + struct ToStartOfTransform \ + { \ + static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ + { \ + return time_zone.toStartOf##INTERVAL_KIND##Interval(t, delta); \ + } \ + }; + TRANSFORM_TIME(Hour) + TRANSFORM_TIME(Minute) + TRANSFORM_TIME(Second) +#undef TRANSFORM_DATE + + template + struct AddTime; + +#define ADD_DATE(INTERVAL_KIND) \ + template <> \ + struct AddTime \ + { \ + static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone) { return time_zone.add##INTERVAL_KIND##s(t, delta); } \ + }; + ADD_DATE(Year) + ADD_DATE(Quarter) + ADD_DATE(Month) + ADD_DATE(Week) + ADD_DATE(Day) +#undef ADD_DATE + +#define ADD_TIME(INTERVAL_KIND, INTERVAL) \ + template <> \ + struct AddTime \ + { \ + static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) { return t + INTERVAL * delta; } \ + }; + ADD_TIME(Hour, 3600) + ADD_TIME(Minute, 60) + ADD_TIME(Second, 1) +#undef ADD_TIME + +namespace +{ + static std::tuple + dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) + { + const auto * interval_type = checkAndGetDataType(interval_column.type.get()); + if (!interval_type) + throw Exception( + "Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN); + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); + if (!interval_column_const_int64) + throw Exception( + "Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN); + Int64 num_units = interval_column_const_int64->getValue(); + if (num_units <= 0) + throw Exception( + "Value for column " + interval_column.name + " of function " + function_name + " must be positive.", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + return {interval_type->getKind(), num_units}; + } + + static ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) + { + // const auto & first_column = block.getByPosition(column_index); + if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) + { + if (!checkColumn>(*col_tuple->getColumnPtr(index))) + throw Exception( + "Illegal column for first argument of function " + function_name + ". Must be a Tuple(DataTime, DataTime)", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return col_tuple->getColumnPtr(index); + } + else if (const ColumnArray * col_array = checkAndGetColumn(column.get()); col_array) + { + const ColumnTuple * col_tuple_ = checkAndGetColumn(&col_array->getData()); + if (!col_tuple_) + throw Exception( + "Illegal column for first argument of function " + function_name + ". Must be a Tuple or Array(Tuple)", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto & bound_column = col_tuple_->getColumn(index); + const ColumnUInt32::Container & bound_data = static_cast(bound_column).getData(); + const auto & column_offsets = col_array->getOffsets(); + auto res = ColumnUInt32::create(); + ColumnUInt32::Container & res_data = res->getData(); + res_data.reserve(column_offsets.size()); + if (index == 0) // lower bound of hop window + { + IColumn::Offset current_offset = 0; + for (size_t i = 0; i < column_offsets.size(); ++i) + { + res_data.push_back(bound_data[current_offset]); + current_offset = column_offsets[i]; + } + } + else // upper bound of hop window + { + for (size_t i = 0; i < column_offsets.size(); ++i) + res_data.push_back(bound_data[column_offsets[i] - 1]); + } + return res; + } + else + { + throw Exception( + "Illegal column for first argument of function " + function_name + ". Must be a Tuple or Array(Tuple)", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + struct TumbleImpl + { + static constexpr auto name = "TUMBLE"; + static constexpr auto isVariadic = false; + static constexpr auto numberOfArguments = 2; + static constexpr auto type = FunctionName::TUMBLE; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() != 2) + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[1].type).isInterval()) + throw Exception( + "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + return std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto & interval_column = block.getByPosition(arguments[1]); + const auto & from_datatype = *time_column.type.get(); + const auto which_type = WhichDataType(from_datatype); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + if (!which_type.isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto interval = dispatchForIntervalColumns(interval_column, function_name); + + switch (std::get<0>(interval)) + { + case IntervalKind::Second: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Minute: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Hour: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Day: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Week: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Month: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Quarter: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Year: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnUInt32::create(size); + auto end = ColumnUInt32::create(size); + ColumnUInt32::Container & start_data = start->getData(); + ColumnUInt32::Container & end_data = end->getData(); + for (size_t i = 0; i != size; ++i) + { + UInt32 wid = (UInt32)ToStartOfTransform::execute(time_data[i], num_units, time_zone); + start_data[i] = wid; + end_data[i] = AddTime::execute(wid, num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } + }; + + struct TumbleStartImpl + { + static constexpr auto name = "TUMBLE_START"; + static constexpr auto isVariadic = true; + static constexpr auto numberOfArguments = 0; + static constexpr auto type = FunctionName::TUMBLE_START; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 1) + { + if (!WhichDataType(arguments[0].type).isTuple()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be tuple", ErrorCodes::ILLEGAL_COLUMN); + return std::make_shared(); + } + else if (arguments.size() == 2) + { + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[1].type).isInterval()) + throw Exception( + "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + return std::make_shared(); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should not larger than 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = TumbleImpl::dispatchForColumns(block, arguments, time_zone, function_name); + else + result_column_ = block.getByPosition(arguments[0]).column; + return executeWindowBound(result_column_, 0, function_name); + } + }; + + + struct TumbleEndImpl + { + static constexpr auto name = "TUMBLE_END"; + static constexpr auto isVariadic = true; + static constexpr auto numberOfArguments = 0; + static constexpr auto type = FunctionName::TUMBLE_END; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return TumbleStartImpl::getReturnType(arguments, function_name); + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = TumbleImpl::dispatchForColumns(block, arguments, time_zone, function_name); + else + result_column_ = block.getByPosition(arguments[0]).column; + return executeWindowBound(result_column_, 1, function_name); + } + }; + + struct HopImpl + { + static constexpr auto name = "HOP"; + static constexpr auto isVariadic = false; + static constexpr auto numberOfArguments = 3; + static constexpr auto type = FunctionName::HOP; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() != 3) + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 3.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[1].type).isInterval()) + throw Exception( + "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[2].type).isInterval()) + throw Exception( + "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + return std::make_shared( + std::make_shared(DataTypes{std::make_shared(), std::make_shared()})); + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto & hop_interval_column = block.getByPosition(arguments[1]); + const auto & window_interval_column = block.getByPosition(arguments[2]); + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<0>(hop_interval) != std::get<0>(window_interval)) + throw Exception( + "Interval type of window and hop column of function " + function_name + ", must be same.", ErrorCodes::ILLEGAL_COLUMN); + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval.", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) + { + case IntervalKind::Second: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr + execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + int max_wid_nums = window_num_units / hop_num_units + (window_num_units % hop_num_units != 0); + + auto column_offsets = ColumnArray::ColumnOffsets::create(size); + IColumn::Offsets & out_offsets = column_offsets->getData(); + + auto start = ColumnUInt32::create(); + auto end = ColumnUInt32::create(); + ColumnUInt32::Container & start_data = start->getData(); + ColumnUInt32::Container & end_data = end->getData(); + start_data.reserve(max_wid_nums * size); + end_data.reserve(max_wid_nums * size); + out_offsets.reserve(size); + IColumn::Offset current_offset = 0; + for (size_t i = 0; i < size; ++i) + { + UInt32 wstart = (UInt32)ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + UInt32 wend = AddTime::execute(wstart, hop_num_units, time_zone); + wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); + + UInt32 wend_ = wend; + UInt32 wend_latest; + + do + { + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); + } while (wend_ > time_data[i]); + + UInt32 wstart_ = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); + wend_ = wend_latest; + + while (wstart_ <= time_data[i]) + { + start_data.push_back(wstart_); + end_data.push_back(wend_); + wstart_ = AddTime::execute(wstart_, hop_num_units, time_zone); + wend_ = AddTime::execute(wstart_, window_num_units, time_zone); + ++current_offset; + } + out_offsets[i] = current_offset; + } + MutableColumns tuple_columns; + tuple_columns.emplace_back(std::move(start)); + tuple_columns.emplace_back(std::move(end)); + return ColumnArray::create(ColumnTuple::create(std::move(tuple_columns)), std::move(column_offsets)); + } + }; + + struct HopStartImpl + { + static constexpr auto name = "HOP_START"; + static constexpr auto isVariadic = true; + static constexpr auto numberOfArguments = 0; + static constexpr auto type = FunctionName::HOP_START; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 1) + { + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isArray()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be tuple or array", + ErrorCodes::ILLEGAL_COLUMN); + return std::make_shared(); + } + else if (arguments.size() == 3) + { + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[1].type).isInterval()) + throw Exception( + "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[2].type).isInterval()) + throw Exception( + "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + return std::make_shared(); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 1 or 3.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = HopImpl::dispatchForColumns(block, arguments, time_zone, function_name); + else + result_column_ = block.getByPosition(arguments[0]).column; + return executeWindowBound(result_column_, 0, function_name); + } + }; + + struct HopEndImpl + { + static constexpr auto name = "HOP_END"; + static constexpr auto isVariadic = true; + static constexpr auto numberOfArguments = 0; + static constexpr auto type = FunctionName::HOP_END; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return HopStartImpl::getReturnType(arguments, function_name); + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = HopImpl::dispatchForColumns(block, arguments, time_zone, function_name); + else + result_column_ = block.getByPosition(arguments[0]).column; + return executeWindowBound(result_column_, 1, function_name); + } + }; +}; + +template +class FunctionWindow : public IFunction +{ +public: + static constexpr auto name = Impl::name; + static FunctionPtr create(const Context &) { return std::make_shared(); } + String getName() const override { return name; } + bool isVariadic() const override { return Impl::isVariadic; } + size_t getNumberOfArguments() const override { return Impl::numberOfArguments; } + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return Impl::getReturnType(arguments, name); } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override + { + const DateLUTImpl & time_zone = DateLUT::instance(); + auto result_column = Impl::dispatchForColumns(block, arguments, time_zone, name); + block.getByPosition(result).column = std::move(result_column); + } +}; + +using FunctionTumble = FunctionWindow; +using FunctionTumbleStart = FunctionWindow; +using FunctionTumbleEnd = FunctionWindow; +using FunctionHop = FunctionWindow; +using FunctionHopStart = FunctionWindow; +using FunctionHopEnd = FunctionWindow; +} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index 233018c7f16..2888483fd6e 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -37,6 +37,7 @@ void registerFunctionsIntrospection(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsJSON(FunctionFactory &); void registerFunctionsConsistentHashing(FunctionFactory & factory); +void registerFunctionsWindow(FunctionFactory &); void registerFunctions() @@ -76,6 +77,7 @@ void registerFunctions() registerFunctionsJSON(factory); registerFunctionsIntrospection(factory); registerFunctionsConsistentHashing(factory); + registerFunctionsWindow(factory); } } From 68666b11a7f9dada77b9ee8869d8afade7985055 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 14 Jan 2020 11:09:48 +0800 Subject: [PATCH 003/609] add window view test --- .../01049_window_functions.reference | 12 +++++++ .../0_stateless/01049_window_functions.sql | 13 ++++++++ .../01052_window_view_watch_tumble.reference | 6 ++++ .../01052_window_view_watch_tumble.sql | 25 +++++++++++++++ .../01053_window_view_watch_hop.reference | 29 +++++++++++++++++ .../01053_window_view_watch_hop.sql | 22 +++++++++++++ .../01054_window_view_to_tumble.reference | 7 +++++ .../01054_window_view_to_tumble.sql | 23 ++++++++++++++ .../01055_window_view_to_hop.reference | 31 +++++++++++++++++++ .../0_stateless/01055_window_view_to_hop.sql | 23 ++++++++++++++ 10 files changed, 191 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01049_window_functions.reference create mode 100644 dbms/tests/queries/0_stateless/01049_window_functions.sql create mode 100644 dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.reference b/dbms/tests/queries/0_stateless/01049_window_functions.reference new file mode 100644 index 00000000000..a7f7395f572 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01049_window_functions.reference @@ -0,0 +1,12 @@ +--TUMBLE-- +('2020-01-09 00:00:00','2020-01-10 00:00:00') +2020-01-09 00:00:00 +2020-01-09 00:00:00 +2020-01-10 00:00:00 +2020-01-10 00:00:00 +--HOP-- +[('2020-01-07 00:00:00','2020-01-10 00:00:00'),('2020-01-08 00:00:00','2020-01-11 00:00:00'),('2020-01-09 00:00:00','2020-01-12 00:00:00')] +2020-01-07 00:00:00 +2020-01-07 00:00:00 +2020-01-12 00:00:00 +2020-01-12 00:00:00 diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.sql b/dbms/tests/queries/0_stateless/01049_window_functions.sql new file mode 100644 index 00000000000..7a8be290803 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01049_window_functions.sql @@ -0,0 +1,13 @@ +SELECT '--TUMBLE--'; +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); +SELECT TUMBLE_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); +SELECT TUMBLE_START(TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY)); +SELECT TUMBLE_END(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); +SELECT TUMBLE_END(TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY)); + +SELECT '--HOP--'; +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); +SELECT HOP_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); +SELECT HOP_START(HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY)); +SELECT HOP_END(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); +SELECT HOP_END(HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY)); \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference new file mode 100644 index 00000000000..600c6521c54 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference @@ -0,0 +1,6 @@ +1 2020-01-09 12:00:00 2020-01-09 12:00:05 +1 2020-01-09 12:00:05 2020-01-09 12:00:10 +1 2020-01-09 12:00:15 2020-01-09 12:00:20 +1 2020-01-09 12:00:10 2020-01-09 12:00:15 +1 2020-01-09 12:00:20 2020-01-09 12:00:25 +1 2020-01-09 12:00:35 2020-01-09 12:00:40 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql new file mode 100644 index 00000000000..fbe36708992 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); +CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) as w_start, TUMBLE_END(wid) as w_end FROM test.mt group by TUMBLE(timestamp, INTERVAL '5' SECOND) as wid; + +INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); +INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); + +WATCH test.wv LIMIT 1; + +INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); +INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); + +WATCH test.wv LIMIT 1; + +INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); +INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); + +WATCH test.wv LIMIT 1; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference new file mode 100644 index 00000000000..cdaddb1b34c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference @@ -0,0 +1,29 @@ +1 2020-01-09 11:59:59 2020-01-09 12:00:04 +1 2020-01-09 12:00:00 2020-01-09 12:00:05 +1 2020-01-09 12:00:04 2020-01-09 12:00:09 +1 2020-01-09 11:59:58 2020-01-09 12:00:03 +1 2020-01-09 11:59:57 2020-01-09 12:00:02 +1 2020-01-09 12:00:05 2020-01-09 12:00:10 +1 2020-01-09 12:00:02 2020-01-09 12:00:07 +2 2020-01-09 12:00:01 2020-01-09 12:00:06 +1 2020-01-09 12:00:03 2020-01-09 12:00:08 +1 2020-01-09 12:00:15 2020-01-09 12:00:20 +1 2020-01-09 12:00:08 2020-01-09 12:00:13 +1 2020-01-09 12:00:12 2020-01-09 12:00:17 +1 2020-01-09 12:00:07 2020-01-09 12:00:12 +1 2020-01-09 12:00:10 2020-01-09 12:00:15 +1 2020-01-09 12:00:14 2020-01-09 12:00:19 +1 2020-01-09 12:00:09 2020-01-09 12:00:14 +1 2020-01-09 12:00:13 2020-01-09 12:00:18 +1 2020-01-09 12:00:11 2020-01-09 12:00:16 +1 2020-01-09 12:00:06 2020-01-09 12:00:11 +1 2020-01-09 12:00:32 2020-01-09 12:00:37 +1 2020-01-09 12:00:20 2020-01-09 12:00:25 +1 2020-01-09 12:00:16 2020-01-09 12:00:21 +1 2020-01-09 12:00:31 2020-01-09 12:00:36 +1 2020-01-09 12:00:34 2020-01-09 12:00:39 +1 2020-01-09 12:00:33 2020-01-09 12:00:38 +1 2020-01-09 12:00:35 2020-01-09 12:00:40 +1 2020-01-09 12:00:18 2020-01-09 12:00:23 +1 2020-01-09 12:00:19 2020-01-09 12:00:24 +1 2020-01-09 12:00:17 2020-01-09 12:00:22 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql new file mode 100644 index 00000000000..566aea23886 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql @@ -0,0 +1,22 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); +CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid; + +INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); +INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); +WATCH test.wv LIMIT 1; + +INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); +INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); +WATCH test.wv LIMIT 1; + +INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); +INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); +WATCH test.wv LIMIT 1; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference b/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference new file mode 100644 index 00000000000..72b7c66ee1d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference @@ -0,0 +1,7 @@ +0 +1 2020-01-09 12:00:00 2020-01-09 12:00:05 +1 2020-01-09 12:00:05 2020-01-09 12:00:10 +1 2020-01-09 12:00:10 2020-01-09 12:00:15 +1 2020-01-09 12:00:15 2020-01-09 12:00:20 +1 2020-01-09 12:00:20 2020-01-09 12:00:25 +1 2020-01-09 12:00:35 2020-01-09 12:00:40 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql b/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql new file mode 100644 index 00000000000..8a4270ec361 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql @@ -0,0 +1,23 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); +CREATE TABLE test.dst(count UInt64, w_start DateTime, w_end DateTime) Engine=MergeTree order by tuple(); +CREATE WINDOW VIEW test.wv to test.dst AS SELECT count(a) as count, TUMBLE_START(wid) as w_start, TUMBLE_END(wid) as w_end FROM test.mt group by TUMBLE(timestamp, INTERVAL '5' SECOND) as wid; + +INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); +INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); +INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); +INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); +INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); +INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); + +SELECT sleep(1); + +SELECT count, w_start, w_end FROM test.dst order by w_start; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference b/dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference new file mode 100644 index 00000000000..7353189c8c1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference @@ -0,0 +1,31 @@ +0 +1 2020-01-09 11:59:57 2020-01-09 12:00:02 +1 2020-01-09 11:59:58 2020-01-09 12:00:03 +1 2020-01-09 11:59:59 2020-01-09 12:00:04 +1 2020-01-09 12:00:00 2020-01-09 12:00:05 +1 2020-01-09 12:00:01 2020-01-09 12:00:06 +1 2020-01-09 12:00:01 2020-01-09 12:00:06 +1 2020-01-09 12:00:02 2020-01-09 12:00:07 +1 2020-01-09 12:00:03 2020-01-09 12:00:08 +1 2020-01-09 12:00:04 2020-01-09 12:00:09 +1 2020-01-09 12:00:05 2020-01-09 12:00:10 +1 2020-01-09 12:00:06 2020-01-09 12:00:11 +1 2020-01-09 12:00:07 2020-01-09 12:00:12 +1 2020-01-09 12:00:08 2020-01-09 12:00:13 +1 2020-01-09 12:00:09 2020-01-09 12:00:14 +1 2020-01-09 12:00:10 2020-01-09 12:00:15 +1 2020-01-09 12:00:11 2020-01-09 12:00:16 +1 2020-01-09 12:00:12 2020-01-09 12:00:17 +1 2020-01-09 12:00:13 2020-01-09 12:00:18 +1 2020-01-09 12:00:14 2020-01-09 12:00:19 +1 2020-01-09 12:00:15 2020-01-09 12:00:20 +1 2020-01-09 12:00:16 2020-01-09 12:00:21 +1 2020-01-09 12:00:17 2020-01-09 12:00:22 +1 2020-01-09 12:00:18 2020-01-09 12:00:23 +1 2020-01-09 12:00:19 2020-01-09 12:00:24 +1 2020-01-09 12:00:20 2020-01-09 12:00:25 +1 2020-01-09 12:00:31 2020-01-09 12:00:36 +1 2020-01-09 12:00:32 2020-01-09 12:00:37 +1 2020-01-09 12:00:33 2020-01-09 12:00:38 +1 2020-01-09 12:00:34 2020-01-09 12:00:39 +1 2020-01-09 12:00:35 2020-01-09 12:00:40 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql b/dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql new file mode 100644 index 00000000000..15aef70956b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql @@ -0,0 +1,23 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); +CREATE TABLE test.dst(count UInt64, w_start DateTime, w_end DateTime) Engine=MergeTree order by tuple(); +CREATE WINDOW VIEW test.wv to test.dst AS SELECT count(a) as count, HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid; + +INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); +INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); +INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); +INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); +INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); +INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); + +SELECT sleep(1); + +SELECT * FROM test.dst order by w_start; + +DROP TABLE test.wv; +DROP TABLE test.mt; From 0b0bde7da407cab85d82f934e4a6e46ea43f5ea1 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 14 Jan 2020 17:04:47 +0800 Subject: [PATCH 004/609] build fix --- dbms/src/Functions/FunctionsWindow.h | 4 ++-- dbms/src/Storages/WindowView/StorageWindowView.cpp | 2 +- dbms/src/Storages/WindowView/WindowViewBlockInputStream.h | 2 +- dbms/src/Storages/registerStorages.cpp | 2 ++ 4 files changed, 6 insertions(+), 4 deletions(-) diff --git a/dbms/src/Functions/FunctionsWindow.h b/dbms/src/Functions/FunctionsWindow.h index 3b1757db853..ce6112832d0 100644 --- a/dbms/src/Functions/FunctionsWindow.h +++ b/dbms/src/Functions/FunctionsWindow.h @@ -261,7 +261,7 @@ namespace ColumnUInt32::Container & end_data = end->getData(); for (size_t i = 0; i != size; ++i) { - UInt32 wid = (UInt32)ToStartOfTransform::execute(time_data[i], num_units, time_zone); + UInt32 wid = static_cast(ToStartOfTransform::execute(time_data[i], num_units, time_zone)); start_data[i] = wid; end_data[i] = AddTime::execute(wid, num_units, time_zone); } @@ -453,7 +453,7 @@ namespace IColumn::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - UInt32 wstart = (UInt32)ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + UInt32 wstart = static_cast(ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone)); UInt32 wend = AddTime::execute(wstart, hop_num_units, time_zone); wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 163bbf75d70..97c6697ad46 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -211,7 +211,7 @@ void StorageWindowView::threadFuncToTable() { std::unique_lock lock(flushTableMutex); UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - UInt64 w_end = static_cast(getWindowUpperBound((UInt32)(timestamp_usec / 1000000))) * 1000000; + UInt64 w_end = static_cast(getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); try { diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 1345419189c..91f8fc0f685 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -113,7 +113,7 @@ protected: while (true) { UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - UInt64 w_end = static_cast(storage->getWindowUpperBound((UInt32)(timestamp_usec / 1000000))) * 1000000; + UInt64 w_end = static_cast(storage->getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; storage->condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); if (isCancelled() || storage->is_dropped) diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index f5fab52285d..3badb8cdf03 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -30,6 +30,8 @@ void registerStorages() registerStorageMaterializedView(factory); registerStorageLiveView(factory); registerStorageGenerateRandom(factory); + registerStorageGenerate(factory); + registerStorageWindowView(factory); #if USE_AWS_S3 registerStorageS3(factory); From 7b0e4cdcb78de0e327f6c93a56492c58380c7e84 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 14 Jan 2020 17:53:52 +0800 Subject: [PATCH 005/609] update code style --- dbms/src/Parsers/ParserAlterQuery.cpp | 2 +- .../Storages/WindowView/StorageWindowView.cpp | 32 ++++++++++--------- 2 files changed, 18 insertions(+), 16 deletions(-) diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 3f48d56b097..d8ffd470cea 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -587,7 +587,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (is_live_view) query->is_live_view = true; - else if(is_window_view) + else if (is_window_view) query->is_window_view = true; if (!parseDatabaseAndTableName(pos, expected, query->database, query->table)) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 97c6697ad46..7357066058a 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -84,7 +84,7 @@ namespace { if (node.name == "TUMBLE") { - if(!data.window_function) + if (!data.window_function) { data.is_tumble = true; data.window_column_name = node.getColumnName(); @@ -95,7 +95,7 @@ namespace } else if (node.name == "HOP") { - if(!data.window_function) + if (!data.window_function) { data.is_hop = true; data.window_function = node.clone(); @@ -106,7 +106,7 @@ namespace data.window_column_name = arrayJoin->getColumnName(); node_ptr = arrayJoin; } - else if(serializeAST(node) != serializeAST(*data.window_function)) + else if (serializeAST(node) != serializeAST(*data.window_function)) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); } } @@ -239,7 +239,7 @@ BlockInputStreams StorageWindowView::watch( if (has_target_table) throw Exception("WATCH query is disabled for " + getName() + " when constructed with 'TO' clause.", ErrorCodes::INCORRECT_QUERY); - if(active_ptr.use_count() > 1) + if (active_ptr.use_count() > 1) throw Exception("WATCH query is already attached, WINDOW VIEW only supports attaching one watch query.", ErrorCodes::INCORRECT_QUERY); ASTWatchQuery & query = typeid_cast(*query_info.query); @@ -373,25 +373,25 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) ASTExpressionList &window_function_args = typeid_cast(*window_function.arguments); const auto & children = window_function_args.children; const auto & interval_p1 = std::static_pointer_cast(children.at(1)); - if(!interval_p1 || !startsWith(interval_p1->name, "toInterval")) + if (!interval_p1 || !startsWith(interval_p1->name, "toInterval")) throw Exception("Illegal type of last argument of function " + interval_p1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); String interval_str = interval_p1->name.substr(10); - if(interval_str == "Second") + if (interval_str == "Second") window_kind = IntervalKind::Second; - else if(interval_str == "Minute") + else if (interval_str == "Minute") window_kind = IntervalKind::Minute; - else if(interval_str == "Hour") + else if (interval_str == "Hour") window_kind = IntervalKind::Hour; - else if(interval_str == "Day") + else if (interval_str == "Day") window_kind = IntervalKind::Day; - else if(interval_str == "Week") + else if (interval_str == "Week") window_kind = IntervalKind::Week; - else if(interval_str == "Month") + else if (interval_str == "Month") window_kind = IntervalKind::Month; - else if(interval_str == "Quarter") + else if (interval_str == "Quarter") window_kind = IntervalKind::Quarter; - else if(interval_str == "Year") + else if (interval_str == "Year") window_kind = IntervalKind::Year; const auto & interval_units_p1 = std::static_pointer_cast(interval_p1->children.front()->children.front()); @@ -509,7 +509,8 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() //delete fired blocks for (BlocksListPtr mergeable_block : *mergeable_blocks) { - mergeable_block->remove_if([](Block & block_) { + mergeable_block->remove_if([](Block & block_) + { auto & column_ = block_.getByName("____fire_status").column; const auto & data = static_cast(*column_).getData(); for (size_t i = 0; i < column_->size(); ++i) @@ -650,7 +651,8 @@ void StorageWindowView::startNoUsersThread(const UInt64 & timeout) void registerStorageWindowView(StorageFactory & factory) { - factory.registerStorage("WindowView", [](const StorageFactory::Arguments & args) { + factory.registerStorage("WindowView", [](const StorageFactory::Arguments & args) + { if (!args.attach && !args.local_context.getSettingsRef().allow_experimental_window_view) throw Exception( "Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')", From 98eacca0b644bef1619ef7b7bc7a0c357b4e1131 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 15 Jan 2020 00:24:26 +0800 Subject: [PATCH 006/609] fix build in clang-8 --- dbms/src/Storages/WindowView/StorageWindowView.cpp | 2 +- dbms/src/Storages/WindowView/StorageWindowView.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 7357066058a..b9275768be4 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -660,4 +660,4 @@ void registerStorageWindowView(StorageFactory & factory) return StorageWindowView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); }); } -} \ No newline at end of file +} diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 534cc1afa1e..94c900acf28 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -138,4 +138,4 @@ private: const ASTCreateQuery & query, const ColumnsDescription & columns); }; -} \ No newline at end of file +} From c6db98e0b5942f5a22e0cdca29dfb7a348c8624c Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 15 Jan 2020 00:24:26 +0800 Subject: [PATCH 007/609] fix build in clang-8 --- dbms/src/Functions/FunctionsWindow.cpp | 2 +- dbms/src/Functions/FunctionsWindow.h | 86 ++++++++++--------- .../Storages/WindowView/StorageWindowView.cpp | 16 ++-- .../WindowView/WindowViewBlocksMetadata.h | 12 +-- 4 files changed, 56 insertions(+), 60 deletions(-) diff --git a/dbms/src/Functions/FunctionsWindow.cpp b/dbms/src/Functions/FunctionsWindow.cpp index 968002f7b79..b99183044ce 100644 --- a/dbms/src/Functions/FunctionsWindow.cpp +++ b/dbms/src/Functions/FunctionsWindow.cpp @@ -12,4 +12,4 @@ void registerFunctionsWindow(FunctionFactory& factory) factory.registerFunction(); factory.registerFunction(); } -} \ No newline at end of file +} diff --git a/dbms/src/Functions/FunctionsWindow.h b/dbms/src/Functions/FunctionsWindow.h index ce6112832d0..1706427fecc 100644 --- a/dbms/src/Functions/FunctionsWindow.h +++ b/dbms/src/Functions/FunctionsWindow.h @@ -40,7 +40,7 @@ namespace DB * HOP_END(time_attr, hop_interval, window_interval) * */ -enum FunctionName +enum WindowFunctionName { TUMBLE, TUMBLE_START, @@ -188,14 +188,20 @@ namespace } } - struct TumbleImpl + template + struct WindowImpl + { + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); + }; + + template <> + struct WindowImpl { static constexpr auto name = "TUMBLE"; static constexpr auto isVariadic = false; static constexpr auto numberOfArguments = 2; - static constexpr auto type = FunctionName::TUMBLE; - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { if (arguments.size() != 2) { @@ -213,7 +219,7 @@ namespace return std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); } - static ColumnPtr + [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); @@ -272,12 +278,12 @@ namespace } }; - struct TumbleStartImpl + template <> + struct WindowImpl { static constexpr auto name = "TUMBLE_START"; static constexpr auto isVariadic = true; static constexpr auto numberOfArguments = 0; - static constexpr auto type = FunctionName::TUMBLE_START; static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -307,14 +313,14 @@ namespace } } - static ColumnPtr + [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = TumbleImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 0, function_name); @@ -322,40 +328,40 @@ namespace }; - struct TumbleEndImpl + template <> + struct WindowImpl { static constexpr auto name = "TUMBLE_END"; static constexpr auto isVariadic = true; static constexpr auto numberOfArguments = 0; - static constexpr auto type = FunctionName::TUMBLE_END; - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - return TumbleStartImpl::getReturnType(arguments, function_name); + return WindowImpl::getReturnType(arguments, function_name); } - static ColumnPtr + [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = TumbleImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 1, function_name); } }; - struct HopImpl + template <> + struct WindowImpl { static constexpr auto name = "HOP"; static constexpr auto isVariadic = false; static constexpr auto numberOfArguments = 3; - static constexpr auto type = FunctionName::HOP; - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { if (arguments.size() != 3) { @@ -486,12 +492,12 @@ namespace } }; - struct HopStartImpl + template <> + struct WindowImpl { static constexpr auto name = "HOP_START"; static constexpr auto isVariadic = true; static constexpr auto numberOfArguments = 0; - static constexpr auto type = FunctionName::HOP_START; static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -526,40 +532,40 @@ namespace } } - static ColumnPtr + [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = HopImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 0, function_name); } }; - struct HopEndImpl + template <> + struct WindowImpl { static constexpr auto name = "HOP_END"; static constexpr auto isVariadic = true; static constexpr auto numberOfArguments = 0; - static constexpr auto type = FunctionName::HOP_END; - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - return HopStartImpl::getReturnType(arguments, function_name); + return WindowImpl::getReturnType(arguments, function_name); } - static ColumnPtr + [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = HopImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 1, function_name); @@ -567,32 +573,32 @@ namespace }; }; -template +template class FunctionWindow : public IFunction { public: - static constexpr auto name = Impl::name; + static constexpr auto name = WindowImpl::name; static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } - bool isVariadic() const override { return Impl::isVariadic; } - size_t getNumberOfArguments() const override { return Impl::numberOfArguments; } + bool isVariadic() const override { return WindowImpl::isVariadic; } + size_t getNumberOfArguments() const override { return WindowImpl::numberOfArguments; } bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return Impl::getReturnType(arguments, name); } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return WindowImpl::getReturnType(arguments, name); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { const DateLUTImpl & time_zone = DateLUT::instance(); - auto result_column = Impl::dispatchForColumns(block, arguments, time_zone, name); + auto result_column = WindowImpl::dispatchForColumns(block, arguments, time_zone, name); block.getByPosition(result).column = std::move(result_column); } }; -using FunctionTumble = FunctionWindow; -using FunctionTumbleStart = FunctionWindow; -using FunctionTumbleEnd = FunctionWindow; -using FunctionHop = FunctionWindow; -using FunctionHopStart = FunctionWindow; -using FunctionHopEnd = FunctionWindow; +using FunctionTumble = FunctionWindow; +using FunctionTumbleStart = FunctionWindow; +using FunctionTumbleEnd = FunctionWindow; +using FunctionHop = FunctionWindow; +using FunctionHopStart = FunctionWindow; +using FunctionHopEnd = FunctionWindow; } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index b9275768be4..8ff4155c3d7 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -192,14 +192,14 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ return AddTime::execute(start, window_num_units, time_zone); \ } - CASE_WINDOW_KIND(Second); - CASE_WINDOW_KIND(Minute); - CASE_WINDOW_KIND(Hour); - CASE_WINDOW_KIND(Day); - CASE_WINDOW_KIND(Week); - CASE_WINDOW_KIND(Month); - CASE_WINDOW_KIND(Quarter); - CASE_WINDOW_KIND(Year); + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) #undef CASE_WINDOW_KIND } __builtin_unreachable(); diff --git a/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h b/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h index e3210a38f70..40dab479518 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h +++ b/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h @@ -10,14 +10,4 @@ enum WINDOW_VIEW_FIRE_STATUS READY, RETIRED }; - -// struct WindowViewBlocksMetadata -// { -// // String hash; -// // std::set window_ids; -// UInt64 max_window_id = 0; -// WINDOW_VIEW_FIRE_STATUS fired_status = WINDOW_VIEW_FIRE_STATUS::WAITING; -// // bool retired; -// }; - -} \ No newline at end of file +} From 35f5f8888951d6a75643d978cee170ac997bd56e Mon Sep 17 00:00:00 2001 From: LB Date: Wed, 15 Jan 2020 17:20:19 +0800 Subject: [PATCH 008/609] update tests --- .../01052_window_view_watch_tumble.reference | 3 -- .../01052_window_view_watch_tumble.sql | 6 ---- .../01053_window_view_watch_hop.reference | 16 +--------- .../01053_window_view_watch_hop.sql | 3 -- .../01054_window_view_to_tumble.reference | 7 ----- .../01054_window_view_to_tumble.sql | 23 -------------- .../01055_window_view_to_hop.reference | 31 ------------------- .../0_stateless/01055_window_view_to_hop.sql | 23 -------------- 8 files changed, 1 insertion(+), 111 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference index 600c6521c54..02d1dc8ad0f 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference @@ -1,6 +1,3 @@ 1 2020-01-09 12:00:00 2020-01-09 12:00:05 -1 2020-01-09 12:00:05 2020-01-09 12:00:10 -1 2020-01-09 12:00:15 2020-01-09 12:00:20 1 2020-01-09 12:00:10 2020-01-09 12:00:15 1 2020-01-09 12:00:20 2020-01-09 12:00:25 -1 2020-01-09 12:00:35 2020-01-09 12:00:40 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql index fbe36708992..39240597856 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql @@ -7,18 +7,12 @@ CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tupl CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) as w_start, TUMBLE_END(wid) as w_end FROM test.mt group by TUMBLE(timestamp, INTERVAL '5' SECOND) as wid; INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); -INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); - WATCH test.wv LIMIT 1; INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); -INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); - WATCH test.wv LIMIT 1; INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); -INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); - WATCH test.wv LIMIT 1; DROP TABLE test.wv; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference index cdaddb1b34c..630096bc32b 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference @@ -1,29 +1,15 @@ 1 2020-01-09 11:59:59 2020-01-09 12:00:04 1 2020-01-09 12:00:00 2020-01-09 12:00:05 -1 2020-01-09 12:00:04 2020-01-09 12:00:09 1 2020-01-09 11:59:58 2020-01-09 12:00:03 1 2020-01-09 11:59:57 2020-01-09 12:00:02 -1 2020-01-09 12:00:05 2020-01-09 12:00:10 -1 2020-01-09 12:00:02 2020-01-09 12:00:07 -2 2020-01-09 12:00:01 2020-01-09 12:00:06 -1 2020-01-09 12:00:03 2020-01-09 12:00:08 -1 2020-01-09 12:00:15 2020-01-09 12:00:20 +1 2020-01-09 12:00:01 2020-01-09 12:00:06 1 2020-01-09 12:00:08 2020-01-09 12:00:13 -1 2020-01-09 12:00:12 2020-01-09 12:00:17 1 2020-01-09 12:00:07 2020-01-09 12:00:12 1 2020-01-09 12:00:10 2020-01-09 12:00:15 -1 2020-01-09 12:00:14 2020-01-09 12:00:19 1 2020-01-09 12:00:09 2020-01-09 12:00:14 -1 2020-01-09 12:00:13 2020-01-09 12:00:18 -1 2020-01-09 12:00:11 2020-01-09 12:00:16 1 2020-01-09 12:00:06 2020-01-09 12:00:11 -1 2020-01-09 12:00:32 2020-01-09 12:00:37 1 2020-01-09 12:00:20 2020-01-09 12:00:25 1 2020-01-09 12:00:16 2020-01-09 12:00:21 -1 2020-01-09 12:00:31 2020-01-09 12:00:36 -1 2020-01-09 12:00:34 2020-01-09 12:00:39 -1 2020-01-09 12:00:33 2020-01-09 12:00:38 -1 2020-01-09 12:00:35 2020-01-09 12:00:40 1 2020-01-09 12:00:18 2020-01-09 12:00:23 1 2020-01-09 12:00:19 2020-01-09 12:00:24 1 2020-01-09 12:00:17 2020-01-09 12:00:22 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql index 566aea23886..ed8a7ce456c 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql @@ -7,15 +7,12 @@ CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tupl CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid; INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); -INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); WATCH test.wv LIMIT 1; INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); -INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); WATCH test.wv LIMIT 1; INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); -INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); WATCH test.wv LIMIT 1; DROP TABLE test.wv; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference b/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference deleted file mode 100644 index 72b7c66ee1d..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.reference +++ /dev/null @@ -1,7 +0,0 @@ -0 -1 2020-01-09 12:00:00 2020-01-09 12:00:05 -1 2020-01-09 12:00:05 2020-01-09 12:00:10 -1 2020-01-09 12:00:10 2020-01-09 12:00:15 -1 2020-01-09 12:00:15 2020-01-09 12:00:20 -1 2020-01-09 12:00:20 2020-01-09 12:00:25 -1 2020-01-09 12:00:35 2020-01-09 12:00:40 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql b/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql deleted file mode 100644 index 8a4270ec361..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_to_tumble.sql +++ /dev/null @@ -1,23 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); -CREATE TABLE test.dst(count UInt64, w_start DateTime, w_end DateTime) Engine=MergeTree order by tuple(); -CREATE WINDOW VIEW test.wv to test.dst AS SELECT count(a) as count, TUMBLE_START(wid) as w_start, TUMBLE_END(wid) as w_end FROM test.mt group by TUMBLE(timestamp, INTERVAL '5' SECOND) as wid; - -INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); -INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); -INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); -INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); -INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); -INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); - -SELECT sleep(1); - -SELECT count, w_start, w_end FROM test.dst order by w_start; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference b/dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference deleted file mode 100644 index 7353189c8c1..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_to_hop.reference +++ /dev/null @@ -1,31 +0,0 @@ -0 -1 2020-01-09 11:59:57 2020-01-09 12:00:02 -1 2020-01-09 11:59:58 2020-01-09 12:00:03 -1 2020-01-09 11:59:59 2020-01-09 12:00:04 -1 2020-01-09 12:00:00 2020-01-09 12:00:05 -1 2020-01-09 12:00:01 2020-01-09 12:00:06 -1 2020-01-09 12:00:01 2020-01-09 12:00:06 -1 2020-01-09 12:00:02 2020-01-09 12:00:07 -1 2020-01-09 12:00:03 2020-01-09 12:00:08 -1 2020-01-09 12:00:04 2020-01-09 12:00:09 -1 2020-01-09 12:00:05 2020-01-09 12:00:10 -1 2020-01-09 12:00:06 2020-01-09 12:00:11 -1 2020-01-09 12:00:07 2020-01-09 12:00:12 -1 2020-01-09 12:00:08 2020-01-09 12:00:13 -1 2020-01-09 12:00:09 2020-01-09 12:00:14 -1 2020-01-09 12:00:10 2020-01-09 12:00:15 -1 2020-01-09 12:00:11 2020-01-09 12:00:16 -1 2020-01-09 12:00:12 2020-01-09 12:00:17 -1 2020-01-09 12:00:13 2020-01-09 12:00:18 -1 2020-01-09 12:00:14 2020-01-09 12:00:19 -1 2020-01-09 12:00:15 2020-01-09 12:00:20 -1 2020-01-09 12:00:16 2020-01-09 12:00:21 -1 2020-01-09 12:00:17 2020-01-09 12:00:22 -1 2020-01-09 12:00:18 2020-01-09 12:00:23 -1 2020-01-09 12:00:19 2020-01-09 12:00:24 -1 2020-01-09 12:00:20 2020-01-09 12:00:25 -1 2020-01-09 12:00:31 2020-01-09 12:00:36 -1 2020-01-09 12:00:32 2020-01-09 12:00:37 -1 2020-01-09 12:00:33 2020-01-09 12:00:38 -1 2020-01-09 12:00:34 2020-01-09 12:00:39 -1 2020-01-09 12:00:35 2020-01-09 12:00:40 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql b/dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql deleted file mode 100644 index 15aef70956b..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_to_hop.sql +++ /dev/null @@ -1,23 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); -CREATE TABLE test.dst(count UInt64, w_start DateTime, w_end DateTime) Engine=MergeTree order by tuple(); -CREATE WINDOW VIEW test.wv to test.dst AS SELECT count(a) as count, HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid; - -INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); -INSERT INTO test.mt VALUES (2, toDateTime('2020-01-09 12:00:05')); -INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); -INSERT INTO test.mt VALUES (4, toDateTime('2020-01-09 12:00:15')); -INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); -INSERT INTO test.mt VALUES (6, toDateTime('2020-01-09 12:00:35')); - -SELECT sleep(1); - -SELECT * FROM test.dst order by w_start; - -DROP TABLE test.wv; -DROP TABLE test.mt; From 17a31771daa18595f0e80efa0f8af4f4c3ae480c Mon Sep 17 00:00:00 2001 From: LB Date: Thu, 16 Jan 2020 05:39:30 +0300 Subject: [PATCH 009/609] update tests --- .../01053_window_view_watch_hop.reference | 25 ++++++++----------- .../01053_window_view_watch_hop.sql | 9 +++---- 2 files changed, 13 insertions(+), 21 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference index 630096bc32b..62c2e82a1d4 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference @@ -1,15 +1,10 @@ -1 2020-01-09 11:59:59 2020-01-09 12:00:04 -1 2020-01-09 12:00:00 2020-01-09 12:00:05 -1 2020-01-09 11:59:58 2020-01-09 12:00:03 -1 2020-01-09 11:59:57 2020-01-09 12:00:02 -1 2020-01-09 12:00:01 2020-01-09 12:00:06 -1 2020-01-09 12:00:08 2020-01-09 12:00:13 -1 2020-01-09 12:00:07 2020-01-09 12:00:12 -1 2020-01-09 12:00:10 2020-01-09 12:00:15 -1 2020-01-09 12:00:09 2020-01-09 12:00:14 -1 2020-01-09 12:00:06 2020-01-09 12:00:11 -1 2020-01-09 12:00:20 2020-01-09 12:00:25 -1 2020-01-09 12:00:16 2020-01-09 12:00:21 -1 2020-01-09 12:00:18 2020-01-09 12:00:23 -1 2020-01-09 12:00:19 2020-01-09 12:00:24 -1 2020-01-09 12:00:17 2020-01-09 12:00:22 +1 1990-01-01 11:59:57 1990-01-01 12:00:02 +1 1990-01-01 11:59:58 1990-01-01 12:00:03 +1 1990-01-01 11:59:59 1990-01-01 12:00:04 +1 1990-01-01 12:00:00 1990-01-01 12:00:05 +1 1990-01-01 12:00:01 1990-01-01 12:00:06 +1 1990-01-01 12:00:06 1990-01-01 12:00:11 +1 1990-01-01 12:00:07 1990-01-01 12:00:12 +1 1990-01-01 12:00:08 1990-01-01 12:00:13 +1 1990-01-01 12:00:09 1990-01-01 12:00:14 +1 1990-01-01 12:00:10 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql index ed8a7ce456c..7c5634c08f3 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql @@ -4,15 +4,12 @@ DROP TABLE IF EXISTS test.wv; DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); -CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid; +CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid order by w_start; -INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); +INSERT INTO test.mt VALUES (1, toDateTime('1990-01-01 12:00:01')); WATCH test.wv LIMIT 1; -INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); -WATCH test.wv LIMIT 1; - -INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); +INSERT INTO test.mt VALUES (3, toDateTime('1990-01-01 12:00:10')); WATCH test.wv LIMIT 1; DROP TABLE test.wv; From 4c2f5e1c8da47559860bf3f7ab3f4eaf6a209afe Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 24 Jan 2020 05:45:45 +0300 Subject: [PATCH 010/609] build fix --- .../PushingToViewsBlockOutputStream.cpp | 12 +-- .../Storages/WindowView/StorageWindowView.cpp | 73 ++++++++----------- .../Storages/WindowView/StorageWindowView.h | 19 +---- .../WindowView/WindowViewProxyStorage.h | 6 +- 4 files changed, 40 insertions(+), 70 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 49bc1eac520..66ce5e9f9ee 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -84,18 +84,8 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( out = io.out; } else if (dynamic_cast(dependent_table.get())) - out = std::make_shared( - database_table.first, database_table.second, dependent_table, *views_context, ASTPtr(), true); + out = std::make_shared(dependent_table, *views_context, ASTPtr(), true); else if (dynamic_cast(dependent_table.get())) - out = std::make_shared( - database_table.first, database_table.second, dependent_table, *views_context, ASTPtr(), true); - else - out = std::make_shared( - database_table.first, database_table.second, dependent_table, *views_context, ASTPtr()); - - views.emplace_back(ViewInfo{std::move(query), database_table.first, database_table.second, std::move(out)}); - } - else if (dynamic_cast(dependent_table.get())) out = std::make_shared(dependent_table, *views_context, ASTPtr(), true); else out = std::make_shared(dependent_table, *views_context, ASTPtr()); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 8ff4155c3d7..55aa3fa8426 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -152,20 +152,19 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa void StorageWindowView::checkTableCanBeDropped() const { - Dependencies dependencies = global_context.getDependencies(database_name, table_name); + auto table_id = getStorageID(); + Dependencies dependencies = global_context.getDependencies(table_id); if (!dependencies.empty()) { - DatabaseAndTableName database_and_table_name = dependencies.front(); - throw Exception( - "Table has dependency " + database_and_table_name.first + "." + database_and_table_name.second, - ErrorCodes::TABLE_WAS_NOT_DROPPED); + StorageID dependent_table_id = dependencies.front(); + throw Exception("Table has dependency " + dependent_table_id.getNameForLogs(), ErrorCodes::TABLE_WAS_NOT_DROPPED); } } void StorageWindowView::drop(TableStructureWriteLockHolder &) { - global_context.removeDependency( - DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); + auto table_id = getStorageID(); + global_context.removeDependency(select_table_id, table_id); std::lock_guard lock(mutex); is_dropped = true; @@ -207,7 +206,7 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) void StorageWindowView::threadFuncToTable() { - while (!shutdown_called && has_target_table) + while (!shutdown_called && !target_table_id.empty()) { std::unique_lock lock(flushTableMutex); UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); @@ -236,7 +235,7 @@ BlockInputStreams StorageWindowView::watch( size_t /*max_block_size*/, const unsigned /*num_streams*/) { - if (has_target_table) + if (!target_table_id.empty()) throw Exception("WATCH query is disabled for " + getName() + " when constructed with 'TO' clause.", ErrorCodes::INCORRECT_QUERY); if (active_ptr.use_count() > 1) @@ -279,9 +278,8 @@ Block StorageWindowView::getHeader() const { if (!sample_block) { - auto storage = global_context.getTable(select_database_name, select_table_name); - sample_block = InterpreterSelectQuery( - getInnerQuery(), global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)) + auto storage = global_context.getTable(select_table_id); + sample_block = InterpreterSelectQuery(getInnerQuery(), global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)) .getSampleBlock(); for (size_t i = 0; i < sample_block.columns(); ++i) sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); @@ -290,24 +288,14 @@ Block StorageWindowView::getHeader() const return sample_block; } -StoragePtr & StorageWindowView::getParentStorage() -{ - if (!parent_storage) - parent_storage = global_context.getTable(getSelectDatabaseName(), getSelectTableName()); - return parent_storage; -} - StorageWindowView::StorageWindowView( - const String & table_name_, - const String & database_name_, + const StorageID & table_id_, Context & local_context, const ASTCreateQuery & query, const ColumnsDescription & columns_) - : table_name(table_name_) - , database_name(database_name_) + : IStorage(table_id_) , global_context(local_context.getGlobalContext()) , time_zone(DateLUT::instance()) - , log(&Poco::Logger::get("StorageWindowView")) { setColumns(columns_); @@ -315,14 +303,16 @@ StorageWindowView::StorageWindowView( throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); /// Default value, if only table name exist in the query - select_database_name = local_context.getCurrentDatabase(); if (query.select->list_of_selects->children.size() != 1) throw Exception("UNION is not supported for Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); auto inner_query_ = query.select->list_of_selects->children.at(0); ASTSelectQuery & select_query = typeid_cast(*inner_query_); + String select_database_name = local_context.getCurrentDatabase(); + String select_table_name; extractDependentTable(select_query, select_database_name, select_table_name); + select_table_id = StorageID(select_database_name, select_table_name); inner_query = innerQueryParser(select_query); /// If the table is not specified - use the table `system.one` @@ -332,14 +322,13 @@ StorageWindowView::StorageWindowView( select_table_name = "one"; } - global_context.addDependency( - DatabaseAndTableName(select_database_name, select_table_name), DatabaseAndTableName(database_name, table_name)); + global_context.addDependency(select_table_id, table_id_); + + parent_storage = local_context.getTable(select_table_id); if (!query.to_table.empty()) { - has_target_table = true; - target_database_name = query.to_database; - target_table_name = query.to_table; + target_table_id = StorageID(query.to_database, query.to_table); } is_temporary = query.temporary; @@ -349,7 +338,7 @@ StorageWindowView::StorageWindowView( active_ptr = std::make_shared(true); - toTableTask = global_context.getSchedulePool().createTask(log->name(), [this] { threadFuncToTable(); }); + toTableTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncToTable(); }); toTableTask->deactivate(); } @@ -403,7 +392,7 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { BlockInputStreams streams = {std::make_shared(block)}; - auto window_proxy_storage = std::make_shared( + auto window_proxy_storage = std::make_shared(StorageID("", "WindowViewProxyStorage"), window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); InterpreterSelectQuery select_block( window_view.getInnerQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); @@ -432,18 +421,18 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con StoragePtr StorageWindowView::getTargetTable() const { - return global_context.getTable(target_database_name, target_table_name); + return global_context.getTable(target_table_id); } StoragePtr StorageWindowView::tryGetTargetTable() const { - return global_context.tryGetTable(target_database_name, target_table_name); + return global_context.tryGetTable(target_table_id); } void StorageWindowView::startup() { // Start the working thread - if (has_target_table) + if (!target_table_id.empty()) toTableTask->activateAndSchedule(); startNoUsersThread(temporary_window_view_timeout); } @@ -553,7 +542,7 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, mutex); from.push_back(std::move(stream)); auto proxy_storage = std::make_shared( - getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); + StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); return data; @@ -575,6 +564,7 @@ void StorageWindowView::noUsersThread(std::shared_ptr storage if (storage->shutdown_called) return; + auto table_id = storage->getStorageID(); { while (1) { @@ -584,7 +574,7 @@ void StorageWindowView::noUsersThread(std::shared_ptr storage storage->no_users_thread_wakeup = false; if (storage->shutdown_called) return; - if (!storage->global_context.getDependencies(storage->database_name, storage->table_name).empty()) + if (!storage->global_context.getDependencies(table_id).empty()) continue; drop_table = true; } @@ -594,14 +584,14 @@ void StorageWindowView::noUsersThread(std::shared_ptr storage if (drop_table) { - if (storage->global_context.tryGetTable(storage->database_name, storage->table_name)) + if (storage->global_context.tryGetTable(table_id)) { try { /// We create and execute `drop` query for this table auto drop_query = std::make_shared(); - drop_query->database = storage->database_name; - drop_query->table = storage->table_name; + drop_query->database = table_id.database_name; + drop_query->table = table_id.table_name; drop_query->kind = ASTDropQuery::Kind::Drop; ASTPtr ast_drop_query = drop_query; InterpreterDropQuery drop_interpreter(ast_drop_query, storage->global_context); @@ -657,7 +647,8 @@ void registerStorageWindowView(StorageFactory & factory) throw Exception( "Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')", ErrorCodes::SUPPORT_IS_DISABLED); - return StorageWindowView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); + + return StorageWindowView::create(args.table_id, args.local_context, args.query, args.columns); }); } } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 94c900acf28..c60f62822c2 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -23,10 +23,6 @@ class StorageWindowView : public ext::shared_ptr_helper, publ public: ~StorageWindowView() override; String getName() const override { return "WindowView"; } - String getTableName() const override { return table_name; } - String getDatabaseName() const override { return database_name; } - String getSelectDatabaseName() const { return select_database_name; } - String getSelectTableName() const { return select_table_name; } ASTPtr getInnerQuery() const { return inner_query->clone(); } @@ -75,7 +71,7 @@ public: Block getHeader() const; - StoragePtr & getParentStorage(); + StoragePtr & getParentStorage() { return parent_storage; } static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); @@ -84,10 +80,7 @@ public: inline UInt32 getWindowUpperBound(UInt32 time_sec); private: - String select_database_name; - String select_table_name; - String table_name; - String database_name; + StorageID select_table_id = StorageID::createEmpty(); ASTPtr inner_query; String window_column_name; String window_end_column_alias; @@ -111,9 +104,7 @@ private: Int64 window_num_units; const DateLUTImpl & time_zone; - std::atomic has_target_table{false}; - String target_database_name; - String target_table_name; + StorageID target_table_id = StorageID::createEmpty(); static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); inline void flushToTable(); @@ -125,15 +116,13 @@ private: std::atomic start_no_users_thread_called{false}; UInt64 temporary_window_view_timeout; - Poco::Logger * log; Poco::Timestamp timestamp; BackgroundSchedulePool::TaskHolder toTableTask; BackgroundSchedulePool::TaskHolder toTableTask_preprocess; StorageWindowView( - const String & table_name_, - const String & database_name_, + const StorageID & table_id_, Context & local_context, const ASTCreateQuery & query, const ColumnsDescription & columns); diff --git a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h index 79b069c3745..377b04c6713 100644 --- a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h +++ b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h @@ -12,14 +12,14 @@ namespace DB class WindowViewProxyStorage : public IStorage { public: - WindowViewProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) - : storage(std::move(storage_)) + WindowViewProxyStorage(const StorageID & table_id_, StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) + : IStorage(table_id_) + , storage(std::move(storage_)) , streams(std::move(streams_)) , to_stage(to_stage_) {} public: std::string getName() const override { return "WindowViewProxyStorage(" + storage->getName() + ")"; } - std::string getTableName() const override { return storage->getTableName(); } bool isRemote() const override { return storage->isRemote(); } bool supportsSampling() const override { return storage->supportsSampling(); } From 2d7b0545fed172c7fa7cd1e60eb2da555b2d9476 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 24 Jan 2020 20:14:31 +0300 Subject: [PATCH 011/609] try to fix code style check --- dbms/src/Storages/WindowView/StorageWindowView.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 55aa3fa8426..9bc9d7b0788 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -392,8 +392,8 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { BlockInputStreams streams = {std::make_shared(block)}; - auto window_proxy_storage = std::make_shared(StorageID("", "WindowViewProxyStorage"), - window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); + auto window_proxy_storage = std::make_shared( + StorageID("", "WindowViewProxyStorage"), window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); InterpreterSelectQuery select_block( window_view.getInnerQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); From c8d079a0ee762631749b7114b8090ed2c3712720 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 2 Feb 2020 06:54:54 +0300 Subject: [PATCH 012/609] timezone support in window functionns --- dbms/src/Functions/FunctionsWindow.h | 89 ++++++++++--------- .../01049_window_functions.reference | 8 ++ .../0_stateless/01049_window_functions.sql | 8 ++ 3 files changed, 65 insertions(+), 40 deletions(-) diff --git a/dbms/src/Functions/FunctionsWindow.h b/dbms/src/Functions/FunctionsWindow.h index 1706427fecc..740144fa67d 100644 --- a/dbms/src/Functions/FunctionsWindow.h +++ b/dbms/src/Functions/FunctionsWindow.h @@ -4,13 +4,14 @@ #include #include #include +#include #include #include #include -#include -#include #include +#include #include +#include #include "IFunctionImpl.h" @@ -19,25 +20,25 @@ namespace DB /** Window functions: * - * TUMBLE(time_attr, interval) + * TUMBLE(time_attr, interval [, timezone]) * * TUMBLE_START(window_id) * - * TUMBLE_START(time_attr, interval) + * TUMBLE_START(time_attr, interval [, timezone]) * * TUMBLE_END(window_id) * - * TUMBLE_END(time_attr, interval) + * TUMBLE_END(time_attr, interval [, timezone]) * - * HOP(time_attr, hop_interval, window_interval) + * HOP(time_attr, hop_interval, window_interval [, timezone]) * * HOP_START(window_id) * - * HOP_START(time_attr, hop_interval, window_interval) + * HOP_START(time_attr, hop_interval, window_interval [, timezone]) * * HOP_END(window_id) * - * HOP_END(time_attr, hop_interval, window_interval) + * HOP_END(time_attr, hop_interval, window_interval [, timezone]) * */ enum WindowFunctionName @@ -141,7 +142,6 @@ namespace static ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) { - // const auto & first_column = block.getByPosition(column_index); if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) { if (!checkColumn>(*col_tuple->getColumnPtr(index))) @@ -198,12 +198,10 @@ namespace struct WindowImpl { static constexpr auto name = "TUMBLE"; - static constexpr auto isVariadic = false; - static constexpr auto numberOfArguments = 2; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - if (arguments.size() != 2) + if (arguments.size() != 2 && arguments.size() != 3) { throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) @@ -216,17 +214,23 @@ namespace if (!WhichDataType(arguments[1].type).isInterval()) throw Exception( "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString()) + throw Exception( + "Illegal type " + arguments[2].type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); } [[maybe_unused]] static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto & interval_column = block.getByPosition(arguments[1]); const auto & from_datatype = *time_column.type.get(); const auto which_type = WhichDataType(from_datatype); const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 2, 0); if (!which_type.isDateTime() || !time_column_vec) throw Exception( "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", @@ -282,8 +286,6 @@ namespace struct WindowImpl { static constexpr auto name = "TUMBLE_START"; - static constexpr auto isVariadic = true; - static constexpr auto numberOfArguments = 0; static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -294,7 +296,7 @@ namespace "Illegal type of first argument of function " + function_name + " should be tuple", ErrorCodes::ILLEGAL_COLUMN); return std::make_shared(); } - else if (arguments.size() == 2) + else if (arguments.size() == 2 || arguments.size() == 3) { if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception( @@ -302,6 +304,11 @@ namespace if (!WhichDataType(arguments[1].type).isInterval()) throw Exception( "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString()) + throw Exception( + "Illegal type " + arguments[2].type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); } else @@ -314,13 +321,13 @@ namespace } [[maybe_unused]] static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 0, function_name); @@ -332,8 +339,6 @@ namespace struct WindowImpl { static constexpr auto name = "TUMBLE_END"; - static constexpr auto isVariadic = true; - static constexpr auto numberOfArguments = 0; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -341,13 +346,13 @@ namespace } [[maybe_unused]] static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 1, function_name); @@ -358,12 +363,10 @@ namespace struct WindowImpl { static constexpr auto name = "HOP"; - static constexpr auto isVariadic = false; - static constexpr auto numberOfArguments = 3; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - if (arguments.size() != 3) + if (arguments.size() != 3 && arguments.size() != 4) { throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) @@ -379,18 +382,24 @@ namespace if (!WhichDataType(arguments[2].type).isInterval()) throw Exception( "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) + throw Exception( + "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared( std::make_shared(DataTypes{std::make_shared(), std::make_shared()})); } static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto & hop_interval_column = block.getByPosition(arguments[1]); const auto & window_interval_column = block.getByPosition(arguments[2]); const auto & from_datatype = *time_column.type.get(); const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 3, 0); if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) throw Exception( "Illegal column " + time_column.name + " argument of function " + function_name @@ -496,8 +505,6 @@ namespace struct WindowImpl { static constexpr auto name = "HOP_START"; - static constexpr auto isVariadic = true; - static constexpr auto numberOfArguments = 0; static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -510,7 +517,7 @@ namespace ErrorCodes::ILLEGAL_COLUMN); return std::make_shared(); } - else if (arguments.size() == 3) + else if (arguments.size() == 3 || arguments.size() == 4) { if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception( @@ -521,6 +528,11 @@ namespace if (!WhichDataType(arguments[2].type).isInterval()) throw Exception( "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) + throw Exception( + "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return std::make_shared(); } else @@ -533,13 +545,13 @@ namespace } [[maybe_unused]] static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 0, function_name); @@ -550,8 +562,6 @@ namespace struct WindowImpl { static constexpr auto name = "HOP_END"; - static constexpr auto isVariadic = true; - static constexpr auto numberOfArguments = 0; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -559,13 +569,13 @@ namespace } [[maybe_unused]] static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const DateLUTImpl & time_zone, const String & function_name) + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(block, arguments, time_zone, function_name); + result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); else result_column_ = block.getByPosition(arguments[0]).column; return executeWindowBound(result_column_, 1, function_name); @@ -580,17 +590,16 @@ public: static constexpr auto name = WindowImpl::name; static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } - bool isVariadic() const override { return WindowImpl::isVariadic; } - size_t getNumberOfArguments() const override { return WindowImpl::numberOfArguments; } + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return WindowImpl::getReturnType(arguments, name); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { - const DateLUTImpl & time_zone = DateLUT::instance(); - auto result_column = WindowImpl::dispatchForColumns(block, arguments, time_zone, name); + auto result_column = WindowImpl::dispatchForColumns(block, arguments, name); block.getByPosition(result).column = std::move(result_column); } }; diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.reference b/dbms/tests/queries/0_stateless/01049_window_functions.reference index a7f7395f572..e69e82162a5 100644 --- a/dbms/tests/queries/0_stateless/01049_window_functions.reference +++ b/dbms/tests/queries/0_stateless/01049_window_functions.reference @@ -1,12 +1,20 @@ --TUMBLE-- ('2020-01-09 00:00:00','2020-01-10 00:00:00') 2020-01-09 00:00:00 +2020-01-08 00:00:00 +2020-01-09 00:00:00 +2020-01-09 00:00:00 +2020-01-10 00:00:00 2020-01-09 00:00:00 2020-01-10 00:00:00 2020-01-10 00:00:00 --HOP-- [('2020-01-07 00:00:00','2020-01-10 00:00:00'),('2020-01-08 00:00:00','2020-01-11 00:00:00'),('2020-01-09 00:00:00','2020-01-12 00:00:00')] 2020-01-07 00:00:00 +2020-01-06 00:00:00 +2020-01-07 00:00:00 2020-01-07 00:00:00 2020-01-12 00:00:00 +2020-01-11 00:00:00 +2020-01-12 00:00:00 2020-01-12 00:00:00 diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.sql b/dbms/tests/queries/0_stateless/01049_window_functions.sql index 7a8be290803..09a47d41a96 100644 --- a/dbms/tests/queries/0_stateless/01049_window_functions.sql +++ b/dbms/tests/queries/0_stateless/01049_window_functions.sql @@ -1,13 +1,21 @@ SELECT '--TUMBLE--'; SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); SELECT TUMBLE_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); +SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY), 'US/Samoa'); SELECT TUMBLE_START(TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY)); SELECT TUMBLE_END(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); +SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY), 'US/Samoa'); SELECT TUMBLE_END(TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY)); SELECT '--HOP--'; SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); SELECT HOP_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); +SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY), 'US/Samoa'); SELECT HOP_START(HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY)); SELECT HOP_END(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); +SELECT toDateTime(HOP_END(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY), 'US/Samoa'); SELECT HOP_END(HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY)); \ No newline at end of file From 2519637564bb539ee71b967574637f92dabedc84 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 2 Feb 2020 11:52:15 +0300 Subject: [PATCH 013/609] build fix --- dbms/src/Parsers/ParserCreateQuery.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index e2ee27dbf41..9fcda44b8a8 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -324,8 +324,8 @@ protected: class ParserCreateWindowViewQuery : public IParserBase { protected: - const char * getName() const { return "CREATE WINDOW VIEW query"; } - bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + const char * getName() const override { return "CREATE WINDOW VIEW query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; /// CREATE|ATTACH DATABASE db [ENGINE = engine] From 772d9cb222b76f01db9e2c53fcd25b2b8e81cd66 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 4 Feb 2020 07:14:31 +0300 Subject: [PATCH 014/609] remove unused code --- dbms/src/Parsers/ASTAlterQuery.cpp | 6 ------ dbms/src/Parsers/ASTAlterQuery.h | 6 +----- dbms/src/Parsers/ParserAlterQuery.cpp | 19 ++----------------- dbms/src/Parsers/ParserAlterQuery.h | 8 ++------ .../Storages/WindowView/StorageWindowView.cpp | 2 +- .../Storages/WindowView/StorageWindowView.h | 4 ---- 6 files changed, 6 insertions(+), 39 deletions(-) diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 10082197140..ceb64aad6a3 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -280,10 +280,6 @@ void ASTAlterCommand::formatImpl( { settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); } - else if (type == ASTAlterCommand::WINDOW_VIEW_REFRESH) - { - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); - } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } @@ -338,8 +334,6 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState if (is_live_view) settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE VIEW " << (settings.hilite ? hilite_none : ""); - else if (is_window_view) - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER WINDOW VIEW " << (settings.hilite ? hilite_none : ""); else settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index 332fb2456fd..44774dcb408 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -18,8 +18,6 @@ namespace DB * COMMENT_COLUMN col_name 'comment', * ALTER LIVE VIEW [db.]name_type * REFRESH - * ALTER WINDOW VIEW [db.]name_type - * REFRESH */ class ASTAlterCommand : public IAST @@ -58,8 +56,7 @@ public: NO_TYPE, - LIVE_VIEW_REFRESH, - WINDOW_VIEW_REFRESH, + LIVE_VIEW_REFRESH }; Type type = NO_TYPE; @@ -189,7 +186,6 @@ class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCl { public: bool is_live_view{false}; /// true for ALTER LIVE VIEW - bool is_window_view{false}; /// true for ALTER WINDOW VIEW ASTAlterCommandList * command_list = nullptr; diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index d8ffd470cea..f991d82a6fc 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -103,15 +103,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected else return false; } - else if (is_window_view) - { - if (s_refresh.ignore(pos, expected)) - { - command->type = ASTAlterCommand::WINDOW_VIEW_REFRESH; - } - else - return false; - } else { if (s_add_column.ignore(pos, expected)) @@ -520,7 +511,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command(is_live_view, is_window_view); + ParserAlterCommand p_command(is_live_view); do { @@ -570,25 +561,19 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_alter_table("ALTER TABLE"); ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); - ParserKeyword s_alter_window_view("ALTER WINDOW VIEW"); bool is_live_view = false; - bool is_window_view = false; if (!s_alter_table.ignore(pos, expected)) { if (s_alter_live_view.ignore(pos, expected)) is_live_view = true; - else if (s_alter_window_view.ignore(pos, expected)) - is_window_view = true; else return false; } if (is_live_view) query->is_live_view = true; - else if (is_window_view) - query->is_window_view = true; if (!parseDatabaseAndTableName(pos, expected, query->database, query->table)) return false; @@ -601,7 +586,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } query->cluster = cluster_str; - ParserAlterCommandList p_command_list(is_live_view, is_window_view); + ParserAlterCommandList p_command_list(is_live_view); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 3d118d0bdb8..61a25b9b387 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -22,8 +22,6 @@ namespace DB * [UPDATE col_name = expr, ... WHERE ...] * ALTER LIVE VIEW [db.name] * [REFRESH] - * ALTER WINDOW VIEW [db.name] - * [REFRESH] */ class ParserAlterQuery : public IParserBase @@ -42,9 +40,8 @@ protected: public: bool is_live_view; - bool is_window_view; - ParserAlterCommandList(bool is_live_view_ = false, bool is_window_view_ = false) : is_live_view(is_live_view_), is_window_view(is_window_view_) {} + ParserAlterCommandList(bool is_live_view_ = false) : is_live_view(is_live_view_) {} }; @@ -56,9 +53,8 @@ protected: public: bool is_live_view; - bool is_window_view; - ParserAlterCommand(bool is_live_view_ = false, bool is_window_view_ = false) : is_live_view(is_live_view_), is_window_view(is_window_view_) {} + ParserAlterCommand(bool is_live_view_ = false) : is_live_view(is_live_view_) {} }; diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 9bc9d7b0788..63ec64c81b3 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -538,7 +538,7 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() } BlockInputStreams from; - auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); //TODO: 改为全局 + auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, mutex); from.push_back(std::move(stream)); auto proxy_storage = std::make_shared( diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index c60f62822c2..c7da45c2c26 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -32,12 +32,8 @@ public: bool isTemporary() { return is_temporary; } - /// Check we have any active readers - /// must be called with mutex locked bool hasActiveUsers() { return active_ptr.use_count() > 1; } - /// Background thread for temporary tables - /// which drops this table if there are no users void startNoUsersThread(const UInt64 & timeout); std::mutex no_users_thread_wakeup_mutex; bool no_users_thread_wakeup{false}; From 4658732e16609f73c528cfbb18ccf0a1990762a0 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 13 Feb 2020 01:39:57 +0800 Subject: [PATCH 015/609] inner table support --- dbms/src/Core/Defines.h | 2 +- dbms/src/Core/Settings.h | 2 +- dbms/src/Parsers/ParserCreateQuery.cpp | 4 + dbms/src/Parsers/ParserCreateQuery.h | 2 +- dbms/src/Storages/StorageFactory.cpp | 6 - ...kInputStream.h => BlocksListInputStream.h} | 24 +- .../Storages/WindowView/StorageWindowView.cpp | 511 ++++++++++-------- .../Storages/WindowView/StorageWindowView.h | 48 +- .../WindowView/WindowViewBlockInputStream.h | 123 ++--- .../WindowView/WindowViewProxyStorage.h | 51 +- ...50_window_view_sql_parser_tumble.reference | 5 + .../01050_window_view_sql_parser_tumble.sql | 28 + ...01051_window_view_sql_parser_hop.reference | 5 + .../01051_window_view_sql_parser_hop.sql | 28 + .../01052_window_view_watch_tumble.reference | 5 +- .../01052_window_view_watch_tumble.sql | 12 +- .../01053_window_view_watch_hop.reference | 12 +- .../01053_window_view_watch_hop.sql | 9 +- ...4_window_view_inner_watch_tumble.reference | 2 + .../01054_window_view_inner_watch_tumble.sql | 13 + ...1055_window_view_inner_watch_hop.reference | 2 + .../01055_window_view_inner_watch_hop.sql | 13 + .../01056_window_view_to_tumble.reference | 2 + .../01056_window_view_to_tumble.sql | 18 + .../01057_window_view_to_hop.reference | 3 + .../0_stateless/01057_window_view_to_hop.sql | 18 + ...1058_window_view_inner_to_tumble.reference | 2 + .../01058_window_view_inner_to_tumble.sql | 18 + .../01059_window_view_inner_to_hop.reference | 3 + .../01059_window_view_inner_to_hop.sql | 18 + 30 files changed, 591 insertions(+), 398 deletions(-) rename dbms/src/Storages/WindowView/{WindowViewBlocksBlockInputStream.h => BlocksListInputStream.h} (78%) create mode 100644 dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql create mode 100644 dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql create mode 100644 dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 8dc1822de56..c84ab12a6b4 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -34,7 +34,7 @@ #define DEFAULT_MERGE_BLOCK_SIZE 8192 #define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 -#define DEFAULT_TEMPORARY_WINDOW_VIEW_TIMEOUT_SEC 5 +#define DEFAULT_WINDOW_VIEW_INNER_TABLE_CLEAN_INTERVAL_SEC 5 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) #define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index f715b31f327..e27a9442a3e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -387,7 +387,7 @@ struct Settings : public SettingsCollection M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ M(SettingBool, allow_experimental_window_view, false, "Enable WINDOW VIEW. Not mature enough.", 0) \ - M(SettingSeconds, temporary_window_view_timeout, DEFAULT_TEMPORARY_WINDOW_VIEW_TIMEOUT_SEC, "Timeout after which temporary window view is deleted.", 0) \ + M(SettingSeconds, window_view_inner_table_clean_interval, DEFAULT_WINDOW_VIEW_INNER_TABLE_CLEAN_INTERVAL_SEC, "The clean interval of window view inner table in seconds to free outdated data.", 0) \ M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 04a7228b513..d0938e7f0c0 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -709,6 +709,9 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } + /// Internal ENGINE for WINDOW VIEW + storage_p.parse(pos, storage, expected); + /// AS SELECT ... if (!s_as.ignore(pos, expected)) return false; @@ -733,6 +736,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & tryGetIdentifierNameInto(to_table, query->to_table); query->set(query->columns_list, columns_list); + query->set(query->storage, storage); tryGetIdentifierNameInto(as_database, query->as_database); tryGetIdentifierNameInto(as_table, query->as_table); diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index 9fcda44b8a8..c0fa8d7aa3e 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -320,7 +320,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] AS SELECT ... +/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [ENGINE [db.]name] [TO [db.]name] AS SELECT ... class ParserCreateWindowViewQuery : public IParserBase { protected: diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 2c2486c6f53..c45812be2d0 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -70,10 +70,6 @@ StoragePtr StorageFactory::get( } else if (query.is_window_view) { - - if (query.storage) - throw Exception("Specifying ENGINE is not allowed for a WindowView", ErrorCodes::INCORRECT_QUERY); - name = "WindowView"; } else @@ -126,8 +122,6 @@ StoragePtr StorageFactory::get( "Direct creation of tables with ENGINE WindowView is not supported, use CREATE WINDOW VIEW statement", ErrorCodes::INCORRECT_QUERY); } - } - } auto it = storages.find(name); if (it == storages.end()) diff --git a/dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h b/dbms/src/Storages/WindowView/BlocksListInputStream.h similarity index 78% rename from dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h rename to dbms/src/Storages/WindowView/BlocksListInputStream.h index 036cb7b921d..f2fbb37f164 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlocksBlockInputStream.h +++ b/dbms/src/Storages/WindowView/BlocksListInputStream.h @@ -12,12 +12,12 @@ namespace DB using BlocksListPtr = std::shared_ptr; using BlocksListPtrs = std::shared_ptr>; -class WindowViewBlocksBlockInputStream : public IBlockInputStream +class BlocksListInputStream : public IBlockInputStream { public: /// Acquires shared ownership of the blocks vector - WindowViewBlocksBlockInputStream(BlocksListPtrs blocks_ptr_, Block header_, std::mutex &mutex_) - : blocks(blocks_ptr_), mutex(mutex_), header(std::move(header_)) + BlocksListInputStream(BlocksListPtrs blocks_ptr_, Block header_, std::mutex &mutex_, UInt32 window_upper_bound_) + : blocks(blocks_ptr_), mutex(mutex_), window_upper_bound(window_upper_bound_), header(std::move(header_)) { it_blocks = blocks->begin(); end_blocks = blocks->end(); @@ -28,7 +28,7 @@ public: } } - String getName() const override { return "MetadataBlocks"; } + String getName() const override { return "BlocksListInputStream"; } Block getHeader() const override { return header; } @@ -42,28 +42,19 @@ protected: Block &block = *it; size_t columns = block.columns(); - //generate filter - auto & column_status = block.getByName("____fire_status").column; - auto column_status_mutable = column_status->assumeMutable(); + auto & column_status = block.getByName("____w_end").column; IColumn::Filter filter(column_status->size(), 0); - auto & data = static_cast(*column_status_mutable).getData(); + auto & data = static_cast(*column_status).getData(); { std::unique_lock lock(mutex); for (size_t i = 0; i < column_status->size(); ++i) { - if (data[i] == WINDOW_VIEW_FIRE_STATUS::READY) - { + if (data[i] == window_upper_bound) filter[i] = 1; - data[i] = WINDOW_VIEW_FIRE_STATUS::RETIRED; - } } } //filter block - /** Let's find out how many rows will be in result. - * To do this, we filter out the first non-constant column - * or calculate number of set bytes in the filter. - */ size_t first_non_constant_column = 0; for (size_t i = 0; i < columns; ++i) { @@ -131,6 +122,7 @@ private: BlocksList::iterator it; BlocksList::iterator end; std::mutex & mutex; + UInt32 window_upper_bound; Block header; }; } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 63ec64c81b3..729d26c5c9c 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -2,6 +2,8 @@ #include #include #include +#include +#include #include #include #include @@ -11,13 +13,18 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include +#include +#include +#include #include #include #include @@ -32,7 +39,7 @@ #include #include -#include +#include #include @@ -61,7 +68,6 @@ namespace { ASTPtr window_function; String window_column_name; - // String window_column_name_or_alias; bool is_tumble = false; bool is_hop = false; }; @@ -111,6 +117,50 @@ namespace } } }; + + static inline String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } + + static ASTPtr generateDeleteRetiredQuery(StorageID inner_table_id, UInt32 timestamp) + { + auto function_equal = makeASTFunction( + "less", std::make_shared("____w_end"), std::make_shared(timestamp)); + + auto alterCommand = std::make_shared(); + alterCommand->type = ASTAlterCommand::DELETE; + alterCommand->predicate = function_equal; + alterCommand->children.push_back(alterCommand->predicate); + + auto alterCommandList = std::make_shared(); + alterCommandList->add(alterCommand); + + auto alterQuery = std::make_shared(); + alterQuery->database = inner_table_id.database_name; + alterQuery->table = inner_table_id.table_name; + alterQuery->set(alterQuery->command_list, alterCommandList); + return alterQuery; + } + + static std::shared_ptr generateFetchColumnsQuery(const StorageID & inner_storage) + { + auto res_query = std::make_shared(); + auto select = std::make_shared(); + select->children.push_back(std::make_shared()); + res_query->setExpression(ASTSelectQuery::Expression::SELECT, select); + + auto tableInSelectQuery = std::make_shared(); + auto tableInSelectQueryElement = std::make_shared(); + res_query->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); + auto tables = res_query->tables(); + auto tables_elem = std::make_shared(); + auto table_expr = std::make_shared(); + tables->children.push_back(tables_elem); + tables_elem->table_expression = table_expr; + tables_elem->children.push_back(table_expr); + table_expr->database_and_table_name = createTableIdentifier(inner_storage.database_name, inner_storage.table_name); + table_expr->children.push_back(table_expr->database_and_table_name); + + return res_query; + } } static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) @@ -161,16 +211,66 @@ void StorageWindowView::checkTableCanBeDropped() const } } +static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id) +{ + if (global_context.tryGetTable(target_table_id)) + { + /// We create and execute `drop` query for internal table. + auto drop_query = std::make_shared(); + drop_query->database = target_table_id.database_name; + drop_query->table = target_table_id.table_name; + drop_query->kind = kind; + ASTPtr ast_drop_query = drop_query; + InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); + drop_interpreter.execute(); + } +} + void StorageWindowView::drop(TableStructureWriteLockHolder &) { auto table_id = getStorageID(); global_context.removeDependency(select_table_id, table_id); + if (has_inner_table) + executeDropQuery(ASTDropQuery::Kind::Drop, global_context, inner_table_id); + std::lock_guard lock(mutex); is_dropped = true; condition.notify_all(); } +inline void StorageWindowView::clearInnerTable() +{ + //delete fired blocks + UInt32 timestamp_now = std::time(nullptr); + UInt32 w_lower_bound = getWindowLowerBound(timestamp_now, -1); + if (has_inner_table) + { + auto sql = generateDeleteRetiredQuery(inner_table_id, w_lower_bound); + InterpreterAlterQuery alt_query(sql, global_context); + alt_query.execute(); + } + else + { + std::lock_guard lock(mutex); + for (BlocksListPtr mergeable_block : *mergeable_blocks) + { + mergeable_block->remove_if([&w_lower_bound](Block & block_) + { + auto & column_ = block_.getByName("____w_end").column; + const auto & data = static_cast(*column_).getData(); + for (size_t i = 0; i < column_->size(); ++i) + { + if (data[i] >= w_lower_bound) + return false; + } + return true; + }); + } + mergeable_blocks->remove_if([](BlocksListPtr & ptr) { return ptr->size() == 0; }); + } +} + inline void StorageWindowView::flushToTable() { //write into dependent table @@ -181,15 +281,55 @@ inline void StorageWindowView::flushToTable() copyData(*_blockInputStreamPtr, *stream); } -UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) +std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name) +{ + /// We will create a query to create an internal table. + auto manual_create_query = std::make_shared(); + manual_create_query->database = database_name; + manual_create_query->table = table_name; + + auto new_columns_list = std::make_shared(); + + auto storage = getParentStorage(); + auto sample_block_ + = InterpreterSelectQuery(getInnerQuery(), global_context, storage, SelectQueryOptions(QueryProcessingStage::WithMergeableState)) + .getSampleBlock(); + + auto columns_list = std::make_shared(); + for (auto & column_ : sample_block_.getColumnsWithTypeAndName()) + { + ParserIdentifierWithOptionalParameters parser; + String sql = column_.type->getName(); + ASTPtr ast = parseQuery(parser, sql.data(), sql.data() + sql.size(), "data type", 0); + auto column_dec = std::make_shared(); + column_dec->name = column_.name; + column_dec->type = ast; + columns_list->children.push_back(column_dec); + } + auto column_fire_status = std::make_shared(); + column_fire_status->name = "____w_end"; + column_fire_status->type = std::make_shared("DateTime"); + columns_list->children.push_back(column_fire_status); + + new_columns_list->set(new_columns_list->columns, columns_list); + manual_create_query->set(manual_create_query->columns_list, new_columns_list); + manual_create_query->set(manual_create_query->storage, inner_create_query.storage->ptr()); + + return manual_create_query; +} + +UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec, int window_id_skew) { switch (window_kind) { #define CASE_WINDOW_KIND(KIND) \ case IntervalKind::KIND: \ { \ - UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ - return AddTime::execute(start, window_num_units, time_zone); \ + UInt32 res = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ + if (window_id_skew != 0) \ + return AddTime::execute(res, window_id_skew * window_num_units, time_zone); \ + else \ + return res; \ } CASE_WINDOW_KIND(Second) CASE_WINDOW_KIND(Minute) @@ -204,6 +344,52 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) __builtin_unreachable(); } +UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window_id_skew) +{ + switch (window_kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: \ + { \ + UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ + UInt32 res = AddTime::execute(start, window_num_units, time_zone); \ + if (window_id_skew != 0) \ + return AddTime::execute(res, window_id_skew * window_num_units, time_zone); \ + else \ + return res; \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); +} + +void StorageWindowView::threadFuncClearInnerTable() +{ + while (!shutdown_called) + { + try + { + clearInnerTable(); + sleep(inner_table_clear_interval); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + break; + } + } + if (!shutdown_called) + innerTableClearTask->scheduleAfter(RESCHEDULE_MS); +} + void StorageWindowView::threadFuncToTable() { while (!shutdown_called && !target_table_id.empty()) @@ -214,8 +400,7 @@ void StorageWindowView::threadFuncToTable() condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); try { - if (refreshBlockStatus()) - flushToTable(); + flushToTable(); } catch (...) { @@ -230,7 +415,7 @@ void StorageWindowView::threadFuncToTable() BlockInputStreams StorageWindowView::watch( const Names & /*column_names*/, const SelectQueryInfo & query_info, - const Context & context, + const Context & /*context*/, QueryProcessingStage::Enum & processed_stage, size_t /*max_block_size*/, const unsigned /*num_streams*/) @@ -256,18 +441,7 @@ BlockInputStreams StorageWindowView::watch( std::static_pointer_cast(shared_from_this()), active_ptr, has_limit, - limit, - context.getSettingsRef().temporary_window_view_timeout.totalSeconds()); - - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } + limit); processed_stage = QueryProcessingStage::Complete; @@ -292,7 +466,8 @@ StorageWindowView::StorageWindowView( const StorageID & table_id_, Context & local_context, const ASTCreateQuery & query, - const ColumnsDescription & columns_) + const ColumnsDescription & columns_, + bool attach_) : IStorage(table_id_) , global_context(local_context.getGlobalContext()) , time_zone(DateLUT::instance()) @@ -324,22 +499,46 @@ StorageWindowView::StorageWindowView( global_context.addDependency(select_table_id, table_id_); - parent_storage = local_context.getTable(select_table_id); - if (!query.to_table.empty()) - { target_table_id = StorageID(query.to_database, query.to_table); - } is_temporary = query.temporary; - temporary_window_view_timeout = local_context.getSettingsRef().temporary_window_view_timeout.totalSeconds(); + inner_table_clear_interval = local_context.getSettingsRef().window_view_inner_table_clean_interval.totalSeconds(); mergeable_blocks = std::make_shared>(); active_ptr = std::make_shared(true); + has_inner_table = query.storage; + + if (query.storage) + { + if (attach_) + { + inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); + } + else + { + if(query.storage->engine->name != "MergeTree") + throw Exception( + "The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support", + ErrorCodes::INCORRECT_QUERY); + + auto manual_create_query + = generateInnerTableCreateQuery(query, table_id_.database_name, generateInnerTableName(table_id_.table_name)); + InterpreterCreateQuery create_interpreter(manual_create_query, local_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); + inner_storage = global_context.getTable(manual_create_query->database, manual_create_query->table); + inner_table_id = inner_storage->getStorageID(); + } + fetch_column_query = generateFetchColumnsQuery(inner_table_id); + } + toTableTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncToTable(); }); + innerTableClearTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncClearInnerTable(); }); toTableTask->deactivate(); + innerTableClearTask->deactivate(); } @@ -359,9 +558,8 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parser window function ASTFunction & window_function = typeid_cast(*stageMergeableOneData.window_function); - ASTExpressionList &window_function_args = typeid_cast(*window_function.arguments); - const auto & children = window_function_args.children; - const auto & interval_p1 = std::static_pointer_cast(children.at(1)); + const auto & arguments = window_function.arguments->children; + const auto & interval_p1 = std::static_pointer_cast(arguments.at(1)); if (!interval_p1 || !startsWith(interval_p1->name, "toInterval")) throw Exception("Illegal type of last argument of function " + interval_p1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); @@ -399,24 +597,35 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con auto data_mergeable_stream = std::make_shared(select_block.execute().in); - BlocksListPtr new_mergeable_blocks = std::make_shared(); - while (Block block_ = data_mergeable_stream->read()) - { - const ColumnTuple * column_tuple = checkAndGetColumn(block_.getByName(window_view.window_column_name).column.get()); - block_.insert( - {ColumnUInt8::create(block_.rows(), WINDOW_VIEW_FIRE_STATUS::WAITING), std::make_shared(), "____fire_status"}); - block_.insert({column_tuple->getColumnPtr(1), std::make_shared(), "____w_end"}); - new_mergeable_blocks->push_back(block_); - } - - if (new_mergeable_blocks->empty()) - return; + if (window_view.has_inner_table) { - std::unique_lock lock(window_view.mutex); - window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); + auto stream = window_view.getInnerStorage()->write(window_view.getInnerQuery(), context); + while (Block block_ = data_mergeable_stream->read()) + { + const ColumnTuple * column_tuple + = checkAndGetColumn(block_.getByName(window_view.window_column_name).column.get()); + block_.insert({column_tuple->getColumnPtr(1), std::make_shared(), "____w_end"}); + stream->write(block_); + } } - window_view.condition.notify_all(); + else + { + BlocksListPtr new_mergeable_blocks = std::make_shared(); + while (Block block_ = data_mergeable_stream->read()) + { + const ColumnTuple * column_tuple + = checkAndGetColumn(block_.getByName(window_view.window_column_name).column.get()); + block_.insert({column_tuple->getColumnPtr(1), std::make_shared(), "____w_end"}); + new_mergeable_blocks->push_back(block_); + } + if (!new_mergeable_blocks->empty()) + { + std::unique_lock lock(window_view.mutex); + window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); + } + } + // window_view.condition.notify_all(); } StoragePtr StorageWindowView::getTargetTable() const @@ -434,7 +643,7 @@ void StorageWindowView::startup() // Start the working thread if (!target_table_id.empty()) toTableTask->activateAndSchedule(); - startNoUsersThread(temporary_window_view_timeout); + innerTableClearTask->activateAndSchedule(); } void StorageWindowView::shutdown() @@ -443,202 +652,71 @@ void StorageWindowView::shutdown() if (!shutdown_called.compare_exchange_strong(expected, true)) return; toTableTask->deactivate(); - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - { - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - } - } + innerTableClearTask->deactivate(); } StorageWindowView::~StorageWindowView() { shutdown(); - - { - std::lock_guard lock(no_users_thread_mutex); - if (no_users_thread.joinable()) - no_users_thread.detach(); - } -} - -bool StorageWindowView::refreshBlockStatus() -{ - UInt32 timestamp_now = std::time(nullptr); - for (BlocksListPtr mergeable_block : *mergeable_blocks) - { - for (Block & block : *mergeable_block) - { - auto & col_wend = block.getByName("____w_end").column; - const auto & wend_data = static_cast(*col_wend).getData(); - auto & col_status = block.getByName("____fire_status").column; - const auto & col_status_data = static_cast(*col_status).getData(); - - for (size_t i = 0; i < col_wend->size(); ++i) - { - if (wend_data[i] < timestamp_now && col_status_data[i] == WINDOW_VIEW_FIRE_STATUS::WAITING) - return true; - } - } - } - return false; } BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() { + if (has_inner_table) + return getNewBlocksInputStreamPtrInnerTable(); + if (mergeable_blocks->empty()) return {std::make_shared(getHeader())}; - { - std::lock_guard lock(mutex); - //delete fired blocks - for (BlocksListPtr mergeable_block : *mergeable_blocks) - { - mergeable_block->remove_if([](Block & block_) - { - auto & column_ = block_.getByName("____fire_status").column; - const auto & data = static_cast(*column_).getData(); - for (size_t i = 0; i < column_->size(); ++i) - { - if (data[i] != WINDOW_VIEW_FIRE_STATUS::RETIRED) - return false; - } - return true; - }); - } - mergeable_blocks->remove_if([](BlocksListPtr & ptr) { return ptr->size() == 0; }); - if (mergeable_blocks->empty()) - return {std::make_shared(getHeader())}; - - // mark blocks can be fired - UInt32 timestamp_now = std::time(nullptr); - for (BlocksListPtr mergeable_block : *mergeable_blocks) - { - for (Block & block : *mergeable_block) - { - auto & col_wend = block.getByName("____w_end").column; - const auto & wend_data = static_cast(*col_wend).getData(); - auto & col_status = block.getByName("____fire_status").column; - auto col_status_mutable = col_status->assumeMutable(); - auto & col_status_data = static_cast(*col_status_mutable).getData(); - - for (size_t i = 0; i < col_wend->size(); ++i) - { - if (wend_data[i] < timestamp_now && col_status_data[i] == WINDOW_VIEW_FIRE_STATUS::WAITING) - col_status_data[i] = WINDOW_VIEW_FIRE_STATUS::READY; - } - col_status = std::move(col_status_mutable); - } - } - } + UInt32 timestamp_now = std::time(nullptr); + UInt32 w_upper_bound = getWindowUpperBound(timestamp_now); BlockInputStreams from; auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, mutex); + BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, mutex,w_upper_bound); from.push_back(std::move(stream)); auto proxy_storage = std::make_shared( StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); return data; } -BlocksPtr StorageWindowView::getNewBlocks() +BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtrInnerTable() { - auto res = getNewBlocksInputStreamPtr(); - BlocksPtr blocks = std::make_shared(); - while (Block this_block = res->read()) - blocks->push_back(this_block); - return blocks; + // Fetch and filter data can be fired + UInt32 timestamp_now = std::time(nullptr); + UInt32 w_end = getWindowUpperBound(timestamp_now); + + auto & storage = getInnerStorage(); + InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); + + ColumnsWithTypeAndName columns_; + columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); + + ExpressionActionsPtr add_column_actions = std::make_shared(columns_, global_context); + add_column_actions->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(w_end)), std::make_shared(), "____w_end_now"})); + auto add_column_instream = std::make_shared(fetch.execute().in, add_column_actions); + + columns_.emplace_back(nullptr, std::make_shared(), "____w_end_now"); + const auto & function_equals = FunctionFactory::instance().get("equals", global_context); + ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); + apply_function_actions->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____w_end_now"}, "____filter")); + + auto stream = std::make_shared(add_column_instream, apply_function_actions, "____filter"); + + BlockInputStreams from; + from.push_back(std::move(stream)); + auto proxy_storage = std::make_shared( + StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); + + InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); + + BlockInputStreamPtr data = std::make_shared(select.execute().in); + return data; } -void StorageWindowView::noUsersThread(std::shared_ptr storage, const UInt64 & timeout) -{ - bool drop_table = false; - - if (storage->shutdown_called) - return; - - auto table_id = storage->getStorageID(); - { - while (1) - { - std::unique_lock lock(storage->no_users_thread_wakeup_mutex); - if (!storage->no_users_thread_condition.wait_for(lock, std::chrono::seconds(timeout), [&] { return storage->no_users_thread_wakeup; })) - { - storage->no_users_thread_wakeup = false; - if (storage->shutdown_called) - return; - if (!storage->global_context.getDependencies(table_id).empty()) - continue; - drop_table = true; - } - break; - } - } - - if (drop_table) - { - if (storage->global_context.tryGetTable(table_id)) - { - try - { - /// We create and execute `drop` query for this table - auto drop_query = std::make_shared(); - drop_query->database = table_id.database_name; - drop_query->table = table_id.table_name; - drop_query->kind = ASTDropQuery::Kind::Drop; - ASTPtr ast_drop_query = drop_query; - InterpreterDropQuery drop_interpreter(ast_drop_query, storage->global_context); - drop_interpreter.execute(); - } - catch (...) - { - } - } - } -} - -void StorageWindowView::startNoUsersThread(const UInt64 & timeout) -{ - bool expected = false; - if (!start_no_users_thread_called.compare_exchange_strong(expected, true)) - return; - - if (is_temporary) - { - std::lock_guard no_users_thread_lock(no_users_thread_mutex); - - if (shutdown_called) - return; - - if (no_users_thread.joinable()) - { - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = true; - no_users_thread_condition.notify_one(); - } - no_users_thread.join(); - } - { - std::lock_guard lock(no_users_thread_wakeup_mutex); - no_users_thread_wakeup = false; - } - if (!is_dropped) - no_users_thread = std::thread(&StorageWindowView::noUsersThread, - std::static_pointer_cast(shared_from_this()), timeout); - } - - start_no_users_thread_called = false; -} - - void registerStorageWindowView(StorageFactory & factory) { factory.registerStorage("WindowView", [](const StorageFactory::Arguments & args) @@ -648,7 +726,8 @@ void registerStorageWindowView(StorageFactory & factory) "Experimental WINDOW VIEW feature is not enabled (the setting 'allow_experimental_window_view')", ErrorCodes::SUPPORT_IS_DISABLED); - return StorageWindowView::create(args.table_id, args.local_context, args.query, args.columns); + return StorageWindowView::create(args.table_id, args.local_context, args.query, args.columns, args.attach); }); } + } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index c7da45c2c26..265386178bd 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -34,11 +34,6 @@ public: bool hasActiveUsers() { return active_ptr.use_count() > 1; } - void startNoUsersThread(const UInt64 & timeout); - std::mutex no_users_thread_wakeup_mutex; - bool no_users_thread_wakeup{false}; - std::condition_variable no_users_thread_condition; - void checkTableCanBeDropped() const override; StoragePtr getTargetTable() const; @@ -63,25 +58,45 @@ public: /// Read new data blocks that store query result BlockInputStreamPtr getNewBlocksInputStreamPtr(); + BlockInputStreamPtr getNewBlocksInputStreamPtrInnerTable(); + BlocksPtr getNewBlocks(); Block getHeader() const; - StoragePtr & getParentStorage() { return parent_storage; } + StoragePtr & getParentStorage() + { + if (parent_storage == nullptr) + parent_storage = global_context.getTable(select_table_id); + return parent_storage; + } + + StoragePtr& getInnerStorage() + { + if (inner_storage == nullptr && !inner_table_id.empty()) + inner_storage = global_context.getTable(inner_table_id); + return inner_storage; + } static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); + static void writeIntoWindowViewInnerTable(StorageWindowView & window_view, const Block & block, const Context & context); + ASTPtr innerQueryParser(ASTSelectQuery & inner_query); - inline UInt32 getWindowUpperBound(UInt32 time_sec); + std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); + + inline UInt32 getWindowLowerBound(UInt32 time_sec, int window_id_skew = 0); + inline UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); private: StorageID select_table_id = StorageID::createEmpty(); ASTPtr inner_query; + ASTPtr fetch_column_query; String window_column_name; - String window_end_column_alias; Context & global_context; StoragePtr parent_storage; + StoragePtr inner_storage; bool is_temporary{false}; mutable Block sample_block; @@ -101,26 +116,27 @@ private: const DateLUTImpl & time_zone; StorageID target_table_id = StorageID::createEmpty(); + StorageID inner_table_id = StorageID::createEmpty(); + bool has_inner_table; - static void noUsersThread(std::shared_ptr storage, const UInt64 & timeout); inline void flushToTable(); + inline void clearInnerTable(); void threadFuncToTable(); - bool refreshBlockStatus(); - std::mutex no_users_thread_mutex; - std::thread no_users_thread; + void threadFuncClearInnerTable(); std::atomic shutdown_called{false}; - std::atomic start_no_users_thread_called{false}; - UInt64 temporary_window_view_timeout; + // UInt64 temporary_window_view_timeout; + UInt64 inner_table_clear_interval; Poco::Timestamp timestamp; BackgroundSchedulePool::TaskHolder toTableTask; - BackgroundSchedulePool::TaskHolder toTableTask_preprocess; + BackgroundSchedulePool::TaskHolder innerTableClearTask; StorageWindowView( const StorageID & table_id_, Context & local_context, const ASTCreateQuery & query, - const ColumnsDescription & columns); + const ColumnsDescription & columns, + bool attach_); }; } diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 91f8fc0f685..4f4decc62db 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -12,31 +12,15 @@ namespace DB class WindowViewBlockInputStream : public IBlockInputStream { public: - ~WindowViewBlockInputStream() override - { - /// Start storage no users thread - /// if we are the last active user - if (!storage->is_dropped) - storage->startNoUsersThread(temporary_window_view_timeout_sec); - } - WindowViewBlockInputStream( std::shared_ptr storage_, std::shared_ptr active_ptr_, const bool has_limit_, - const UInt64 limit_, - // const UInt64 heartbeat_interval_sec_, - const UInt64 temporary_window_view_timeout_sec_) + const UInt64 limit_) : storage(std::move(storage_)) - // , active_ptr(std::move(active_ptr_)) , active(std::move(active_ptr_)) , has_limit(has_limit_) - , limit(limit_) - , temporary_window_view_timeout_sec(temporary_window_view_timeout_sec_) - { - /// grab active pointer - // active = active_ptr.lock(); - } + , limit(limit_) {} String getName() const override { return "WindowViewBlockInputStream"; } @@ -72,100 +56,65 @@ protected: return Block(); } /// If blocks were never assigned get blocks - if (!blocks) + if (!in_stream) { - if (!active) - return Block(); - // std::unique_lock lock(storage->mutex); - blocks = storage->getNewBlocks(); - it = blocks->begin(); - begin = blocks->begin(); - end = blocks->end(); + std::unique_lock lock(storage->mutex); + in_stream = storage->getNewBlocksInputStreamPtr(); } - if (isCancelled() || storage->is_dropped) { return Block(); } - if (it == end) + res = in_stream->read(); + if (!res) { if (!active) return Block(); - if (storage->refreshBlockStatus()) + + if (!end_of_blocks) { - // std::unique_lock lock(storage->mutex); - blocks = storage->getNewBlocks(); - it = blocks->begin(); - begin = blocks->begin(); - end = blocks->end(); + end_of_blocks = true; + num_updates += 1; + return getHeader(); + } + + std::unique_lock lock(storage->flushTableMutex); + UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); + UInt64 w_end = static_cast(storage->getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; + storage->condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); + + if (isCancelled() || storage->is_dropped) + { + return Block(); + } + { + std::unique_lock lock_(storage->mutex); + in_stream = storage->getNewBlocksInputStreamPtr(); + } + + res = in_stream->read(); + if (res) + { + end_of_blocks = false; + return res; } - /// No new blocks available wait for new ones else { - std::unique_lock lock(storage->flushTableMutex); - // std::unique_lock lock(storage->mutex); - if (!end_of_blocks) - { - end_of_blocks = true; - return getHeader(); - } - while (true) - { - UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - UInt64 w_end = static_cast(storage->getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; - storage->condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); - - if (isCancelled() || storage->is_dropped) - { - std::cout << "AAAAAAAAAAAAAAAA Cancelled:" << std::endl; - return Block(); - } - if (storage->refreshBlockStatus()) - { - break; - } - else - { - return getHeader(); - } - } + return getHeader(); } - return tryReadImpl(); } - res = *it; - - ++it; - - if (it == end) - { - end_of_blocks = false; - num_updates += 1; - } - - // last_event_timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - // last_event_timestamp_usec = timestamp_usec; return res; } private: std::shared_ptr storage; - // std::weak_ptr active_ptr; std::shared_ptr active; - BlocksPtr blocks; - - // std::mutex mutex; - Blocks::iterator it; - Blocks::iterator end; - Blocks::iterator begin; const bool has_limit; const UInt64 limit; - Int64 num_updates = 0; + Int64 num_updates = -1; bool end_of_blocks = false; - // UInt64 heartbeat_interval_usec; - UInt64 temporary_window_view_timeout_sec; - // UInt64 last_event_timestamp_usec = 0; - // UInt64 reschedule_us; + BlockInputStreamPtr in_stream; }; } diff --git a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h index 377b04c6713..b2529f861b7 100644 --- a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h +++ b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h @@ -12,21 +12,21 @@ namespace DB class WindowViewProxyStorage : public IStorage { public: - WindowViewProxyStorage(const StorageID & table_id_, StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) + WindowViewProxyStorage(const StorageID & table_id_, StoragePtr parent_storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) : IStorage(table_id_) - , storage(std::move(storage_)) + , parent_storage(std::move(parent_storage_)) , streams(std::move(streams_)) , to_stage(to_stage_) {} public: - std::string getName() const override { return "WindowViewProxyStorage(" + storage->getName() + ")"; } + std::string getName() const override { return "WindowViewProxyStorage(" + parent_storage->getName() + ")"; } - bool isRemote() const override { return storage->isRemote(); } - bool supportsSampling() const override { return storage->supportsSampling(); } - bool supportsFinal() const override { return storage->supportsFinal(); } - bool supportsPrewhere() const override { return storage->supportsPrewhere(); } - bool supportsReplication() const override { return storage->supportsReplication(); } - bool supportsDeduplication() const override { return storage->supportsDeduplication(); } + bool isRemote() const override { return parent_storage->isRemote(); } + bool supportsSampling() const override { return parent_storage->supportsSampling(); } + bool supportsFinal() const override { return parent_storage->supportsFinal(); } + bool supportsPrewhere() const override { return parent_storage->supportsPrewhere(); } + bool supportsReplication() const override { return parent_storage->supportsReplication(); } + bool supportsDeduplication() const override { return parent_storage->supportsDeduplication(); } QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; } @@ -41,30 +41,29 @@ public: return streams; } - bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); } - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override { return storage->mayBenefitFromIndexForIn(left_in_operand, query_context); } - ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); } - ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); } - ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); } - ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); } - Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); } - Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); } - Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); } - Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); } - Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); } + bool supportsIndexForIn() const override { return parent_storage->supportsIndexForIn(); } + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override { return parent_storage->mayBenefitFromIndexForIn(left_in_operand, query_context); } + ASTPtr getPartitionKeyAST() const override { return parent_storage->getPartitionKeyAST(); } + ASTPtr getSortingKeyAST() const override { return parent_storage->getSortingKeyAST(); } + ASTPtr getPrimaryKeyAST() const override { return parent_storage->getPrimaryKeyAST(); } + ASTPtr getSamplingKeyAST() const override { return parent_storage->getSamplingKeyAST(); } + Names getColumnsRequiredForPartitionKey() const override { return parent_storage->getColumnsRequiredForPartitionKey(); } + Names getColumnsRequiredForSortingKey() const override { return parent_storage->getColumnsRequiredForSortingKey(); } + Names getColumnsRequiredForPrimaryKey() const override { return parent_storage->getColumnsRequiredForPrimaryKey(); } + Names getColumnsRequiredForSampling() const override { return parent_storage->getColumnsRequiredForSampling(); } + Names getColumnsRequiredForFinal() const override { return parent_storage->getColumnsRequiredForFinal(); } - const ColumnsDescription & getColumns() const override { return storage->getColumns(); } + const ColumnsDescription & getColumns() const override { return parent_storage->getColumns(); } - void setColumns(ColumnsDescription columns_) override { return storage->setColumns(columns_); } + void setColumns(ColumnsDescription columns_) override { return parent_storage->setColumns(columns_); } - NameAndTypePair getColumn(const String & column_name) const override { return storage->getColumn(column_name); } + NameAndTypePair getColumn(const String & column_name) const override { return parent_storage->getColumn(column_name); } - bool hasColumn(const String & column_name) const override { return storage->hasColumn(column_name); } + bool hasColumn(const String & column_name) const override { return parent_storage->hasColumn(column_name); } private: - StoragePtr storage; + StoragePtr parent_storage; BlockInputStreams streams; QueryProcessingStage::Enum to_stage; - NameAndTypePair column_end; }; } diff --git a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference new file mode 100644 index 00000000000..65b71f1b41f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference @@ -0,0 +1,5 @@ +---With w_end--- +---WithOut w_end--- +---WITH--- +---WHERE--- +---ORDER_BY--- diff --git a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql new file mode 100644 index 00000000000..faa001d0346 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql @@ -0,0 +1,28 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +SELECT '---With w_end---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; + +SELECT '---WithOut w_end---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; + +SELECT '---WITH---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end, date_time FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; + +SELECT '---WHERE---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; + +SELECT '---ORDER_BY---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; + +DROP TABLE test.mt; +DROP TABLE test.wv; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference new file mode 100644 index 00000000000..65b71f1b41f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference @@ -0,0 +1,5 @@ +---With w_end--- +---WithOut w_end--- +---WITH--- +---WHERE--- +---ORDER_BY--- diff --git a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql new file mode 100644 index 00000000000..71cc2c20ecc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql @@ -0,0 +1,28 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +SELECT '---With w_end---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; + +SELECT '---WithOut w_end---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; + +SELECT '---WITH---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end, date_time FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; + +SELECT '---WHERE---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; + +SELECT '---ORDER_BY---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; + +DROP TABLE test.mt; +DROP TABLE test.wv; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference index 02d1dc8ad0f..6ed281c757a 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference @@ -1,3 +1,2 @@ -1 2020-01-09 12:00:00 2020-01-09 12:00:05 -1 2020-01-09 12:00:10 2020-01-09 12:00:15 -1 2020-01-09 12:00:20 2020-01-09 12:00:25 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql index 39240597856..dfa2ef9d1a6 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql @@ -3,16 +3,10 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS test.wv; DROP TABLE IF EXISTS test.mt; -CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); -CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) as w_start, TUMBLE_END(wid) as w_end FROM test.mt group by TUMBLE(timestamp, INTERVAL '5' SECOND) as wid; +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, toDateTime('2020-01-09 12:00:01')); -WATCH test.wv LIMIT 1; - -INSERT INTO test.mt VALUES (3, toDateTime('2020-01-09 12:00:10')); -WATCH test.wv LIMIT 1; - -INSERT INTO test.mt VALUES (5, toDateTime('2020-01-09 12:00:20')); +INSERT INTO test.mt VALUES (1, now()); WATCH test.wv LIMIT 1; DROP TABLE test.wv; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference index 62c2e82a1d4..6ed281c757a 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference @@ -1,10 +1,2 @@ -1 1990-01-01 11:59:57 1990-01-01 12:00:02 -1 1990-01-01 11:59:58 1990-01-01 12:00:03 -1 1990-01-01 11:59:59 1990-01-01 12:00:04 -1 1990-01-01 12:00:00 1990-01-01 12:00:05 -1 1990-01-01 12:00:01 1990-01-01 12:00:06 -1 1990-01-01 12:00:06 1990-01-01 12:00:11 -1 1990-01-01 12:00:07 1990-01-01 12:00:12 -1 1990-01-01 12:00:08 1990-01-01 12:00:13 -1 1990-01-01 12:00:09 1990-01-01 12:00:14 -1 1990-01-01 12:00:10 1990-01-01 12:00:15 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql index 7c5634c08f3..43ced360e38 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql @@ -3,13 +3,10 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS test.wv; DROP TABLE IF EXISTS test.mt; -CREATE TABLE test.mt(a Int32, timestamp DateTime) Engine=MergeTree order by tuple(); -CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) as w_start, HOP_END(wid) as w_end FROM test.mt group by HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '5' SECOND) as wid order by w_start; +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, toDateTime('1990-01-01 12:00:01')); -WATCH test.wv LIMIT 1; - -INSERT INTO test.mt VALUES (3, toDateTime('1990-01-01 12:00:10')); +INSERT INTO test.mt VALUES (1, now()); WATCH test.wv LIMIT 1; DROP TABLE test.wv; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql new file mode 100644 index 00000000000..0d7c94047f1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql @@ -0,0 +1,13 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +WATCH test.wv LIMIT 1; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql new file mode 100644 index 00000000000..94bc2739489 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql @@ -0,0 +1,13 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +WATCH test.wv LIMIT 1; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference b/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql b/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql new file mode 100644 index 00000000000..1169ae06eb4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(1); +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference b/dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference new file mode 100644 index 00000000000..986394f7c0f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference @@ -0,0 +1,3 @@ +0 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql b/dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql new file mode 100644 index 00000000000..a6227913d9d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test.dst(count UInt64) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(3); +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference b/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql b/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql new file mode 100644 index 00000000000..1f4a580b5b3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test.dst(count UInt64) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(1); +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference b/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference new file mode 100644 index 00000000000..986394f7c0f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference @@ -0,0 +1,3 @@ +0 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql b/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql new file mode 100644 index 00000000000..49e0f67f14a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; + +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test.dst(count UInt64) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(3); +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file From 4351a16103110096ef961a8e8ac271d9d99a88da Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 13 Feb 2020 02:08:52 +0800 Subject: [PATCH 016/609] multi 'WATCH' and 'TO' support --- dbms/src/Storages/WindowView/BlocksListInputStream.h | 8 ++++---- dbms/src/Storages/WindowView/StorageWindowView.cpp | 10 ++-------- dbms/src/Storages/WindowView/StorageWindowView.h | 6 +++--- .../Storages/WindowView/WindowViewBlockInputStream.h | 9 +++++---- 4 files changed, 14 insertions(+), 19 deletions(-) diff --git a/dbms/src/Storages/WindowView/BlocksListInputStream.h b/dbms/src/Storages/WindowView/BlocksListInputStream.h index f2fbb37f164..2280f06ac3a 100644 --- a/dbms/src/Storages/WindowView/BlocksListInputStream.h +++ b/dbms/src/Storages/WindowView/BlocksListInputStream.h @@ -16,8 +16,8 @@ class BlocksListInputStream : public IBlockInputStream { public: /// Acquires shared ownership of the blocks vector - BlocksListInputStream(BlocksListPtrs blocks_ptr_, Block header_, std::mutex &mutex_, UInt32 window_upper_bound_) - : blocks(blocks_ptr_), mutex(mutex_), window_upper_bound(window_upper_bound_), header(std::move(header_)) + BlocksListInputStream(BlocksListPtrs blocks_ptr_, Block header_, UInt32 window_upper_bound_) + : blocks(blocks_ptr_), window_upper_bound(window_upper_bound_), header(std::move(header_)) { it_blocks = blocks->begin(); end_blocks = blocks->end(); @@ -46,7 +46,7 @@ protected: IColumn::Filter filter(column_status->size(), 0); auto & data = static_cast(*column_status).getData(); { - std::unique_lock lock(mutex); + // std::unique_lock lock(mutex); for (size_t i = 0; i < column_status->size(); ++i) { if (data[i] == window_upper_bound) @@ -121,7 +121,7 @@ private: std::list::iterator end_blocks; BlocksList::iterator it; BlocksList::iterator end; - std::mutex & mutex; + // std::mutex & mutex; UInt32 window_upper_bound; Block header; }; diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 729d26c5c9c..895577d32d5 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -420,12 +420,6 @@ BlockInputStreams StorageWindowView::watch( size_t /*max_block_size*/, const unsigned /*num_streams*/) { - if (!target_table_id.empty()) - throw Exception("WATCH query is disabled for " + getName() + " when constructed with 'TO' clause.", ErrorCodes::INCORRECT_QUERY); - - if (active_ptr.use_count() > 1) - throw Exception("WATCH query is already attached, WINDOW VIEW only supports attaching one watch query.", ErrorCodes::INCORRECT_QUERY); - ASTWatchQuery & query = typeid_cast(*query_info.query); bool has_limit = false; @@ -519,7 +513,7 @@ StorageWindowView::StorageWindowView( } else { - if(query.storage->engine->name != "MergeTree") + if (query.storage->engine->name != "MergeTree") throw Exception( "The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support", ErrorCodes::INCORRECT_QUERY); @@ -673,7 +667,7 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() BlockInputStreams from; auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, mutex,w_upper_bound); + BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, w_upper_bound); from.push_back(std::move(stream)); auto proxy_storage = std::make_shared( StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 265386178bd..df42fee3434 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -64,11 +64,11 @@ public: Block getHeader() const; - StoragePtr & getParentStorage() + StoragePtr& getParentStorage() { - if (parent_storage == nullptr) + if (parent_storage == nullptr) parent_storage = global_context.getTable(select_table_id); - return parent_storage; + return parent_storage; } StoragePtr& getInnerStorage() diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 4f4decc62db..ef7cb3a2777 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -58,7 +58,7 @@ protected: /// If blocks were never assigned get blocks if (!in_stream) { - std::unique_lock lock(storage->mutex); + // std::unique_lock lock(storage->mutex); in_stream = storage->getNewBlocksInputStreamPtr(); } if (isCancelled() || storage->is_dropped) @@ -69,7 +69,7 @@ protected: res = in_stream->read(); if (!res) { - if (!active) + if (!(*active)) return Block(); if (!end_of_blocks) @@ -79,7 +79,7 @@ protected: return getHeader(); } - std::unique_lock lock(storage->flushTableMutex); + std::unique_lock lock(mutex); UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); UInt64 w_end = static_cast(storage->getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; storage->condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); @@ -89,7 +89,7 @@ protected: return Block(); } { - std::unique_lock lock_(storage->mutex); + // std::unique_lock lock_(storage->mutex); in_stream = storage->getNewBlocksInputStreamPtr(); } @@ -113,6 +113,7 @@ private: std::shared_ptr active; const bool has_limit; const UInt64 limit; + std::mutex mutex; Int64 num_updates = -1; bool end_of_blocks = false; BlockInputStreamPtr in_stream; From 817980a8784209f8a1143dc7686d0a1cca379e08 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 14 Feb 2020 16:07:03 +0800 Subject: [PATCH 017/609] watermark support --- dbms/src/Parsers/ASTCreateQuery.cpp | 6 + dbms/src/Parsers/ASTCreateQuery.h | 1 + dbms/src/Parsers/ParserCreateQuery.cpp | 15 +- dbms/src/Parsers/ParserCreateQuery.h | 2 +- .../Storages/WindowView/StorageWindowView.cpp | 271 +++++++++++++----- .../Storages/WindowView/StorageWindowView.h | 27 +- .../WindowView/WindowViewBlockInputStream.h | 58 ++-- ...50_window_view_sql_parser_tumble.reference | 1 + .../01050_window_view_sql_parser_tumble.sql | 4 + ...01051_window_view_sql_parser_hop.reference | 1 + .../01051_window_view_sql_parser_hop.sql | 4 + .../01052_window_view_watch_tumble.reference | 1 - .../01053_window_view_watch_hop.sql | 2 +- ...4_window_view_inner_watch_tumble.reference | 1 - .../01055_window_view_inner_watch_hop.sql | 2 +- 15 files changed, 293 insertions(+), 103 deletions(-) diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 5068fe7d5cf..3165d09692b 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -296,6 +296,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (is_populate) settings.ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : ""); + if (watermark_function) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK " << (settings.hilite ? hilite_none : ""); + watermark_function->formatImpl(settings, state, frame); + } + if (select) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 49f0c56e71d..529ee976e87 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -68,6 +68,7 @@ public: String to_database; /// For CREATE MATERIALIZED VIEW mv TO table. String to_table; ASTStorage * storage = nullptr; + ASTPtr watermark_function; String as_database; String as_table; ASTPtr as_table_function; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index d0938e7f0c0..0928098a103 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -625,11 +625,13 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserKeyword s_view("VIEW"); ParserKeyword s_window("WINDOW"); ParserToken s_dot(TokenType::Dot); + ParserToken s_eq(TokenType::Equals); ParserToken s_lparen(TokenType::OpeningRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket); ParserStorage storage_p; ParserIdentifier name_p; ParserTablePropertiesDeclarationList table_properties_p; + ParserIntervalOperatorExpression watermark_p; ParserSelectWithUnionQuery select_p; ASTPtr database; @@ -638,6 +640,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ASTPtr to_database; ASTPtr to_table; ASTPtr storage; + ASTPtr watermark; ASTPtr as_database; ASTPtr as_table; ASTPtr select; @@ -709,9 +712,18 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } - /// Internal ENGINE for WINDOW VIEW + /// Inner table ENGINE for WINDOW VIEW storage_p.parse(pos, storage, expected); + // WATERMARK + if (ParserKeyword{"WATERMARK"}.ignore(pos, expected)) + { + s_eq.ignore(pos, expected); + + if (!watermark_p.parse(pos, watermark, expected)) + return false; + } + /// AS SELECT ... if (!s_as.ignore(pos, expected)) return false; @@ -737,6 +749,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->set(query->columns_list, columns_list); query->set(query->storage, storage); + query->watermark_function = watermark; tryGetIdentifierNameInto(as_database, query->as_database); tryGetIdentifierNameInto(as_table, query->as_table); diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index c0fa8d7aa3e..d415385ed8c 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -320,7 +320,7 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; -/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [ENGINE [db.]name] [TO [db.]name] AS SELECT ... +/// CREATE|ATTACH WINDOW VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE [db.]name] [WATERMARK function] AS SELECT ... class ParserCreateWindowViewQuery : public IParserBase { protected: diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 895577d32d5..49add395563 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include @@ -53,6 +54,7 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW; extern const int SUPPORT_IS_DISABLED; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int CANNOT_PARSE_TEXT; } namespace @@ -117,6 +119,27 @@ namespace } } }; + + static inline IntervalKind strToIntervalKind(const String& interval_str) + { + if (interval_str == "Second") + return IntervalKind::Second; + else if (interval_str == "Minute") + return IntervalKind::Minute; + else if (interval_str == "Hour") + return IntervalKind::Hour; + else if (interval_str == "Day") + return IntervalKind::Day; + else if (interval_str == "Week") + return IntervalKind::Week; + else if (interval_str == "Month") + return IntervalKind::Month; + else if (interval_str == "Quarter") + return IntervalKind::Quarter; + else if (interval_str == "Year") + return IntervalKind::Year; + __builtin_unreachable(); + } static inline String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } @@ -231,7 +254,7 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &) auto table_id = getStorageID(); global_context.removeDependency(select_table_id, table_id); - if (has_inner_table) + if (!inner_table_id.empty()) executeDropQuery(ASTDropQuery::Kind::Drop, global_context, inner_table_id); std::lock_guard lock(mutex); @@ -244,7 +267,9 @@ inline void StorageWindowView::clearInnerTable() //delete fired blocks UInt32 timestamp_now = std::time(nullptr); UInt32 w_lower_bound = getWindowLowerBound(timestamp_now, -1); - if (has_inner_table) + if (has_watermark) + w_lower_bound = getWatermark(w_lower_bound); + if (!inner_table_id.empty()) { auto sql = generateDeleteRetiredQuery(inner_table_id, w_lower_bound); InterpreterAlterQuery alt_query(sql, global_context); @@ -271,11 +296,11 @@ inline void StorageWindowView::clearInnerTable() } } -inline void StorageWindowView::flushToTable() +inline void StorageWindowView::flushToTable(UInt32 timestamp_) { //write into dependent table StoragePtr target_table = getTargetTable(); - auto _blockInputStreamPtr = getNewBlocksInputStreamPtr(); + auto _blockInputStreamPtr = getNewBlocksInputStreamPtr(timestamp_); auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); auto stream = target_table->write(getInnerQuery(), global_context); copyData(*_blockInputStreamPtr, *stream); @@ -318,7 +343,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery return manual_create_query; } -UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec, int window_id_skew) +inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec, int window_id_skew) { switch (window_kind) { @@ -344,7 +369,7 @@ UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec, int window_id_ske __builtin_unreachable(); } -UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window_id_skew) +inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window_id_skew) { switch (window_kind) { @@ -371,6 +396,40 @@ UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window_id_ske __builtin_unreachable(); } +inline UInt32 StorageWindowView::getWatermark(UInt32 time_sec) +{ + switch (watermark_kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: \ + { \ + return AddTime::execute(time_sec, watermark_num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); +} + +inline void StorageWindowView::addFireSignal(UInt32 timestamp_) +{ + if (!target_table_id.empty()) + fire_signal.push_back(timestamp_); + for (auto watch_stream : watch_streams) + { + if (watch_stream) + watch_stream->addFireSignal(timestamp_); + } + condition.notify_all(); +} + void StorageWindowView::threadFuncClearInnerTable() { while (!shutdown_called) @@ -394,13 +453,20 @@ void StorageWindowView::threadFuncToTable() { while (!shutdown_called && !target_table_id.empty()) { - std::unique_lock lock(flushTableMutex); - UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - UInt64 w_end = static_cast(getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; - condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); + std::unique_lock lock(flush_table_mutex); + condition.wait_for(lock, std::chrono::seconds(5)); try { - flushToTable(); + while (!fire_signal.empty()) + { + UInt32 timestamp_; + { + std::unique_lock lock_(fire_signal_mutex); + timestamp_ = fire_signal.front(); + fire_signal.pop_front(); + } + flushToTable(timestamp_); + } } catch (...) { @@ -412,6 +478,20 @@ void StorageWindowView::threadFuncToTable() toTableTask->scheduleAfter(RESCHEDULE_MS); } +void StorageWindowView::threadFuncFire() +{ + while (!shutdown_called) + { + UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); + UInt64 w_end = static_cast(getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; + std::this_thread::sleep_for(std::chrono::microseconds(w_end - timestamp_usec)); + std::unique_lock lock(fire_signal_mutex); + addFireSignal(static_cast(w_end / 1000000)); + } + if (!shutdown_called) + toTableTask->scheduleAfter(RESCHEDULE_MS); +} + BlockInputStreams StorageWindowView::watch( const Names & /*column_names*/, const SelectQueryInfo & query_info, @@ -436,6 +516,11 @@ BlockInputStreams StorageWindowView::watch( active_ptr, has_limit, limit); + + { + std::lock_guard lock(fire_signal_mutex); + watch_streams.push_back(reader.get()); + } processed_stage = QueryProcessingStage::Complete; @@ -503,7 +588,31 @@ StorageWindowView::StorageWindowView( active_ptr = std::make_shared(true); - has_inner_table = query.storage; + if (query.watermark_function) + { + // parser watermark function + has_watermark = true; + const auto & watermark_function = std::static_pointer_cast(query.watermark_function); + if (!startsWith(watermark_function->name, "toInterval")) + throw Exception( + "Illegal type WATERMARK function " + watermark_function->name + ", should be Interval", ErrorCodes::ILLEGAL_COLUMN); + + String interval_str = watermark_function->name.substr(10); + const auto & interval_units_p1 = std::static_pointer_cast(watermark_function->children.front()->children.front()); + watermark_kind = strToIntervalKind(interval_str); + try + { + watermark_num_units = boost::lexical_cast(interval_units_p1->value.get()); + } + catch (const boost::bad_lexical_cast & exec) + { + throw Exception("Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); + } + if (watermark_num_units == 0) + has_watermark = false; + else if (watermark_num_units > 0) + watermark_num_units *= -1; + } if (query.storage) { @@ -531,8 +640,10 @@ StorageWindowView::StorageWindowView( toTableTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncToTable(); }); innerTableClearTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncClearInnerTable(); }); + fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFire(); }); toTableTask->deactivate(); innerTableClearTask->deactivate(); + fireTask->deactivate(); } @@ -553,29 +664,15 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parser window function ASTFunction & window_function = typeid_cast(*stageMergeableOneData.window_function); const auto & arguments = window_function.arguments->children; - const auto & interval_p1 = std::static_pointer_cast(arguments.at(1)); - if (!interval_p1 || !startsWith(interval_p1->name, "toInterval")) - throw Exception("Illegal type of last argument of function " + interval_p1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + const auto & arg0 = std::static_pointer_cast(arguments.at(0)); + timestamp_column_name = arg0->IAST::getAliasOrColumnName(); + const auto & arg1 = std::static_pointer_cast(arguments.at(1)); + if (!arg1 || !startsWith(arg1->name, "toInterval")) + throw Exception("Illegal type of last argument of function " + arg1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - String interval_str = interval_p1->name.substr(10); - if (interval_str == "Second") - window_kind = IntervalKind::Second; - else if (interval_str == "Minute") - window_kind = IntervalKind::Minute; - else if (interval_str == "Hour") - window_kind = IntervalKind::Hour; - else if (interval_str == "Day") - window_kind = IntervalKind::Day; - else if (interval_str == "Week") - window_kind = IntervalKind::Week; - else if (interval_str == "Month") - window_kind = IntervalKind::Month; - else if (interval_str == "Quarter") - window_kind = IntervalKind::Quarter; - else if (interval_str == "Year") - window_kind = IntervalKind::Year; - - const auto & interval_units_p1 = std::static_pointer_cast(interval_p1->children.front()->children.front()); + String interval_str = arg1->name.substr(10); + window_kind = strToIntervalKind(interval_str); + const auto & interval_units_p1 = std::static_pointer_cast(arg1->children.front()->children.front()); window_num_units = stoi(interval_units_p1->value.get()); return result; @@ -583,6 +680,8 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { + UInt32 timestamp_now = std::time(nullptr); + BlockInputStreams streams = {std::make_shared(block)}; auto window_proxy_storage = std::make_shared( StorageID("", "WindowViewProxyStorage"), window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); @@ -591,27 +690,76 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con auto data_mergeable_stream = std::make_shared(select_block.execute().in); + // extract ____w_end + ColumnsWithTypeAndName columns_; + columns_.emplace_back( + nullptr, + std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), + window_view.window_column_name); + const auto & function_tuple = FunctionFactory::instance().get("tupleElement", context); + ExpressionActionsPtr actions_ = std::make_shared(columns_, context); + actions_->add(ExpressionAction::addColumn( + {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); + actions_->add(ExpressionAction::applyFunction(function_tuple, Names{window_view.window_column_name, "____tuple_arg"}, "____w_end")); + actions_->add(ExpressionAction::removeColumn("____tuple_arg")); - if (window_view.has_inner_table) + BlockInputStreamPtr in_stream; + if (window_view.has_watermark) + { + UInt32 watermark = window_view.getWatermark(timestamp_now); + actions_->add( + ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), std::make_shared(), "____watermark"})); + const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); + actions_->add(ExpressionAction::applyFunction(function_greater, Names{"____w_end", "____watermark"}, "____filter")); + actions_->add(ExpressionAction::removeColumn("____watermark")); + in_stream = std::make_shared(data_mergeable_stream, actions_, "____filter", true); + } + else + in_stream = std::make_shared(data_mergeable_stream, actions_); + + if (!window_view.inner_table_id.empty()) { auto stream = window_view.getInnerStorage()->write(window_view.getInnerQuery(), context); - while (Block block_ = data_mergeable_stream->read()) + if (window_view.has_watermark) { - const ColumnTuple * column_tuple - = checkAndGetColumn(block_.getByName(window_view.window_column_name).column.get()); - block_.insert({column_tuple->getColumnPtr(1), std::make_shared(), "____w_end"}); - stream->write(block_); + while (Block block_ = in_stream->read()) + { + stream->write(std::move(block_)); + const ColumnUInt32::Container & wend_data + = static_cast(*block_.getByName("____w_end").column.get()).getData(); + for (size_t i = 0; i < wend_data.size(); ++i) + { + if (wend_data[i] < timestamp_now) + { + window_view.addFireSignal(wend_data[i]); + } + } + } } + else + copyData(*in_stream, *stream); } else { BlocksListPtr new_mergeable_blocks = std::make_shared(); - while (Block block_ = data_mergeable_stream->read()) + if (window_view.has_watermark) { - const ColumnTuple * column_tuple - = checkAndGetColumn(block_.getByName(window_view.window_column_name).column.get()); - block_.insert({column_tuple->getColumnPtr(1), std::make_shared(), "____w_end"}); - new_mergeable_blocks->push_back(block_); + while (Block block_ = in_stream->read()) + { + new_mergeable_blocks->push_back(std::move(block_)); + const ColumnUInt32::Container & wend_data + = static_cast(*block_.getByName("____w_end").column.get()).getData(); + for (size_t i = 0; i < wend_data.size(); ++i) + { + if (wend_data[i] < timestamp_now) + window_view.addFireSignal(wend_data[i]); + } + } + } + else + { + while (Block block_ = in_stream->read()) + new_mergeable_blocks->push_back(std::move(block_)); } if (!new_mergeable_blocks->empty()) { @@ -619,7 +767,6 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); } } - // window_view.condition.notify_all(); } StoragePtr StorageWindowView::getTargetTable() const @@ -638,6 +785,7 @@ void StorageWindowView::startup() if (!target_table_id.empty()) toTableTask->activateAndSchedule(); innerTableClearTask->activateAndSchedule(); + fireTask->activateAndSchedule(); } void StorageWindowView::shutdown() @@ -647,6 +795,7 @@ void StorageWindowView::shutdown() return; toTableTask->deactivate(); innerTableClearTask->deactivate(); + fireTask->deactivate(); } StorageWindowView::~StorageWindowView() @@ -654,20 +803,17 @@ StorageWindowView::~StorageWindowView() shutdown(); } -BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() +BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 timestamp_) { - if (has_inner_table) - return getNewBlocksInputStreamPtrInnerTable(); + if (!inner_table_id.empty()) + return getNewBlocksInputStreamPtrInnerTable(timestamp_); if (mergeable_blocks->empty()) return {std::make_shared(getHeader())}; - UInt32 timestamp_now = std::time(nullptr); - UInt32 w_upper_bound = getWindowUpperBound(timestamp_now); - BlockInputStreams from; auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, w_upper_bound); + BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, timestamp_); from.push_back(std::move(stream)); auto proxy_storage = std::make_shared( StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); @@ -677,28 +823,20 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr() return data; } -BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtrInnerTable() +BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtrInnerTable(UInt32 timestamp_) { - // Fetch and filter data can be fired - UInt32 timestamp_now = std::time(nullptr); - UInt32 w_end = getWindowUpperBound(timestamp_now); - auto & storage = getInnerStorage(); InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); ColumnsWithTypeAndName columns_; columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); - ExpressionActionsPtr add_column_actions = std::make_shared(columns_, global_context); - add_column_actions->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(w_end)), std::make_shared(), "____w_end_now"})); - auto add_column_instream = std::make_shared(fetch.execute().in, add_column_actions); - - columns_.emplace_back(nullptr, std::make_shared(), "____w_end_now"); + ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); + actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(timestamp_)), std::make_shared(), "____w_end_now"})); const auto & function_equals = FunctionFactory::instance().get("equals", global_context); ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); - apply_function_actions->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____w_end_now"}, "____filter")); - - auto stream = std::make_shared(add_column_instream, apply_function_actions, "____filter"); + actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____w_end_now"}, "____filter")); + auto stream = std::make_shared(fetch.execute().in, actions_, "____filter", true); BlockInputStreams from; from.push_back(std::move(stream)); @@ -706,7 +844,6 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtrInnerTable() StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); - BlockInputStreamPtr data = std::make_shared(select.execute().in); return data; } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index df42fee3434..8e1f7413c08 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -11,6 +11,7 @@ namespace DB { class IAST; +class WindowViewBlockInputStream; using ASTPtr = std::shared_ptr; using BlocksListPtr = std::shared_ptr; using BlocksListPtrs = std::shared_ptr>; @@ -56,15 +57,15 @@ public: std::shared_ptr getActivePtr() { return active_ptr; } /// Read new data blocks that store query result - BlockInputStreamPtr getNewBlocksInputStreamPtr(); + BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 timestamp_); - BlockInputStreamPtr getNewBlocksInputStreamPtrInnerTable(); + BlockInputStreamPtr getNewBlocksInputStreamPtrInnerTable(UInt32 timestamp_); BlocksPtr getNewBlocks(); Block getHeader() const; - StoragePtr& getParentStorage() + StoragePtr& getParentStorage() { if (parent_storage == nullptr) parent_storage = global_context.getTable(select_table_id); @@ -88,12 +89,14 @@ public: inline UInt32 getWindowLowerBound(UInt32 time_sec, int window_id_skew = 0); inline UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); + inline UInt32 getWatermark(UInt32 time_sec); private: StorageID select_table_id = StorageID::createEmpty(); ASTPtr inner_query; ASTPtr fetch_column_query; String window_column_name; + String timestamp_column_name; Context & global_context; StoragePtr parent_storage; StoragePtr inner_storage; @@ -102,7 +105,10 @@ private: /// Mutex for the blocks and ready condition std::mutex mutex; - std::mutex flushTableMutex; + std::mutex flush_table_mutex; + std::mutex fire_signal_mutex; + std::list fire_signal; + std::list watch_streams; /// New blocks ready condition to broadcast to readers /// that new blocks are available std::condition_variable condition; @@ -111,26 +117,31 @@ private: std::shared_ptr active_ptr; BlocksListPtrs mergeable_blocks; + bool has_watermark{false}; IntervalKind::Kind window_kind; Int64 window_num_units; + IntervalKind::Kind watermark_kind; + Int64 watermark_num_units; const DateLUTImpl & time_zone; StorageID target_table_id = StorageID::createEmpty(); StorageID inner_table_id = StorageID::createEmpty(); - bool has_inner_table; - inline void flushToTable(); - inline void clearInnerTable(); + void flushToTable(UInt32 timestamp_); + void clearInnerTable(); void threadFuncToTable(); void threadFuncClearInnerTable(); + void threadFuncFire(); + void addFireSignal(UInt32 timestamp_); + std::atomic shutdown_called{false}; - // UInt64 temporary_window_view_timeout; UInt64 inner_table_clear_interval; Poco::Timestamp timestamp; BackgroundSchedulePool::TaskHolder toTableTask; BackgroundSchedulePool::TaskHolder innerTableClearTask; + BackgroundSchedulePool::TaskHolder fireTask; StorageWindowView( const StorageID & table_id_, diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index ef7cb3a2777..89f15dad943 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -29,12 +30,25 @@ public: if (isCancelled() || storage->is_dropped) return; IBlockInputStream::cancel(kill); - std::lock_guard lock(storage->mutex); - storage->condition.notify_all(); + std::lock_guard lock(storage->fire_signal_mutex); + for (auto it = storage->watch_streams.begin() ; it != storage->watch_streams.end() ; ++it) + { + if (*it == this) + { + storage->watch_streams.erase(it); + break; + } + } } Block getHeader() const override { return storage->getHeader(); } + inline void addFireSignal(UInt32 timestamp) + { + std::lock_guard lock(fire_signal_mutex); + fire_signal.push_back(timestamp); + } + protected: Block readImpl() override { @@ -57,10 +71,8 @@ protected: } /// If blocks were never assigned get blocks if (!in_stream) - { - // std::unique_lock lock(storage->mutex); - in_stream = storage->getNewBlocksInputStreamPtr(); - } + in_stream = std::make_shared(getHeader()); + if (isCancelled() || storage->is_dropped) { return Block(); @@ -80,31 +92,31 @@ protected: } std::unique_lock lock(mutex); - UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - UInt64 w_end = static_cast(storage->getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; - storage->condition.wait_for(lock, std::chrono::microseconds(w_end - timestamp_usec)); + storage->condition.wait_for(lock, std::chrono::seconds(5)); if (isCancelled() || storage->is_dropped) { return Block(); } - { - // std::unique_lock lock_(storage->mutex); - in_stream = storage->getNewBlocksInputStreamPtr(); - } - res = in_stream->read(); - if (res) + while (!fire_signal.empty()) { - end_of_blocks = false; - return res; - } - else - { - return getHeader(); + UInt32 timestamp_; + { + std::unique_lock lock_(fire_signal_mutex); + timestamp_ = fire_signal.front(); + fire_signal.pop_front(); + } + in_stream = storage->getNewBlocksInputStreamPtr(timestamp_); + res = in_stream->read(); + if (res) + { + end_of_blocks = false; + return res; + } } + return getHeader(); } - return res; } @@ -117,5 +129,7 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; BlockInputStreamPtr in_stream; + std::mutex fire_signal_mutex; + std::list fire_signal; }; } diff --git a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference index 65b71f1b41f..75cd8e28af5 100644 --- a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference +++ b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference @@ -1,3 +1,4 @@ +---WATERMARK--- ---With w_end--- ---WithOut w_end--- ---WITH--- diff --git a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql index faa001d0346..55d92bbe37c 100644 --- a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql +++ b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql @@ -4,6 +4,10 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +SELECT '---WATERMARK---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; + SELECT '---With w_end---'; DROP TABLE IF EXISTS test.wv; CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; diff --git a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference index 65b71f1b41f..75cd8e28af5 100644 --- a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference +++ b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference @@ -1,3 +1,4 @@ +---WATERMARK--- ---With w_end--- ---WithOut w_end--- ---WITH--- diff --git a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql index 71cc2c20ecc..01d9ce9fe13 100644 --- a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql +++ b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql @@ -4,6 +4,10 @@ DROP TABLE IF EXISTS test.mt; CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +SELECT '---WATERMARK---'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; + SELECT '---With w_end---'; DROP TABLE IF EXISTS test.wv; CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference index 6ed281c757a..d00491fd7e5 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference +++ b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference @@ -1,2 +1 @@ 1 -1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql index 43ced360e38..1ddd273c103 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql +++ b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql @@ -7,7 +7,7 @@ CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tupl CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now()); -WATCH test.wv LIMIT 1; +WATCH test.wv LIMIT 2; DROP TABLE test.wv; DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference index 6ed281c757a..d00491fd7e5 100644 --- a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference +++ b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference @@ -1,2 +1 @@ 1 -1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql index 94bc2739489..35cccdaf0b8 100644 --- a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql +++ b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql @@ -7,7 +7,7 @@ CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tupl CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now()); -WATCH test.wv LIMIT 1; +WATCH test.wv LIMIT 2; DROP TABLE test.wv; DROP TABLE test.mt; From c296b76624a46a0ff6eef1582f4da2655ec99004 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 14 Feb 2020 16:55:56 +0800 Subject: [PATCH 018/609] fix code style --- .../Storages/WindowView/StorageWindowView.cpp | 17 +++++++---------- .../src/Storages/WindowView/StorageWindowView.h | 3 +-- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 49add395563..8d1c3913b45 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -119,7 +119,7 @@ namespace } } }; - + static inline IntervalKind strToIntervalKind(const String& interval_str) { if (interval_str == "Second") @@ -267,7 +267,7 @@ inline void StorageWindowView::clearInnerTable() //delete fired blocks UInt32 timestamp_now = std::time(nullptr); UInt32 w_lower_bound = getWindowLowerBound(timestamp_now, -1); - if (has_watermark) + if (watermark_num_units != 0) w_lower_bound = getWatermark(w_lower_bound); if (!inner_table_id.empty()) { @@ -516,7 +516,7 @@ BlockInputStreams StorageWindowView::watch( active_ptr, has_limit, limit); - + { std::lock_guard lock(fire_signal_mutex); watch_streams.push_back(reader.get()); @@ -591,7 +591,6 @@ StorageWindowView::StorageWindowView( if (query.watermark_function) { // parser watermark function - has_watermark = true; const auto & watermark_function = std::static_pointer_cast(query.watermark_function); if (!startsWith(watermark_function->name, "toInterval")) throw Exception( @@ -608,9 +607,7 @@ StorageWindowView::StorageWindowView( { throw Exception("Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); } - if (watermark_num_units == 0) - has_watermark = false; - else if (watermark_num_units > 0) + if (watermark_num_units > 0) watermark_num_units *= -1; } @@ -704,7 +701,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con actions_->add(ExpressionAction::removeColumn("____tuple_arg")); BlockInputStreamPtr in_stream; - if (window_view.has_watermark) + if (window_view.watermark_num_units != 0) { UInt32 watermark = window_view.getWatermark(timestamp_now); actions_->add( @@ -720,7 +717,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (!window_view.inner_table_id.empty()) { auto stream = window_view.getInnerStorage()->write(window_view.getInnerQuery(), context); - if (window_view.has_watermark) + if (window_view.watermark_num_units != 0) { while (Block block_ = in_stream->read()) { @@ -742,7 +739,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con else { BlocksListPtr new_mergeable_blocks = std::make_shared(); - if (window_view.has_watermark) + if (window_view.watermark_num_units != 0) { while (Block block_ = in_stream->read()) { diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 8e1f7413c08..b547982299a 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -117,11 +117,10 @@ private: std::shared_ptr active_ptr; BlocksListPtrs mergeable_blocks; - bool has_watermark{false}; IntervalKind::Kind window_kind; Int64 window_num_units; IntervalKind::Kind watermark_kind; - Int64 watermark_num_units; + Int64 watermark_num_units = 0; const DateLUTImpl & time_zone; StorageID target_table_id = StorageID::createEmpty(); From 00c3bfb72af6979698c04f96cdebfc90d71d1509 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 17 Feb 2020 13:06:03 +0800 Subject: [PATCH 019/609] processing time support --- .../AddingTimestampBlockInputStream.h | 39 ++++ .../Storages/WindowView/StorageWindowView.cpp | 196 +++++++++++------- .../Storages/WindowView/StorageWindowView.h | 140 ++++++------- .../WindowView/WindowViewProxyStorage.h | 35 +++- ...052_window_view_event_time_watch.reference | 10 + .../01052_window_view_event_time_watch.sql | 35 ++++ .../01052_window_view_watch_tumble.reference | 1 - .../01052_window_view_watch_tumble.sql | 13 -- .../01053_window_view_event_time_to.reference | 14 ++ .../01053_window_view_event_time_to.sql | 47 +++++ .../01053_window_view_watch_hop.reference | 2 - .../01053_window_view_watch_hop.sql | 13 -- ...4_window_view_inner_watch_tumble.reference | 1 - .../01054_window_view_inner_watch_tumble.sql | 13 -- ...1054_window_view_proc_time_watch.reference | 10 + .../01054_window_view_proc_time_watch.sql | 35 ++++ ...1055_window_view_inner_watch_hop.reference | 2 - .../01055_window_view_inner_watch_hop.sql | 13 -- .../01055_window_view_proc_time_to.reference | 14 ++ .../01055_window_view_proc_time_to.sql | 47 +++++ .../01056_window_view_to_tumble.reference | 2 - .../01056_window_view_to_tumble.sql | 18 -- .../01057_window_view_to_hop.reference | 3 - .../0_stateless/01057_window_view_to_hop.sql | 18 -- ...1058_window_view_inner_to_tumble.reference | 2 - .../01058_window_view_inner_to_tumble.sql | 18 -- .../01059_window_view_inner_to_hop.reference | 3 - .../01059_window_view_inner_to_hop.sql | 18 -- 28 files changed, 471 insertions(+), 291 deletions(-) create mode 100644 dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h create mode 100644 dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference create mode 100644 dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql delete mode 100644 dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference delete mode 100644 dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference delete mode 100644 dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference delete mode 100644 dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql delete mode 100644 dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference delete mode 100644 dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql delete mode 100644 dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference delete mode 100644 dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql delete mode 100644 dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference delete mode 100644 dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql diff --git a/dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h b/dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h new file mode 100644 index 00000000000..a307fccc3bc --- /dev/null +++ b/dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h @@ -0,0 +1,39 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/** Add timestamp column for processing time process in + * WINDOW VIEW + */ +class AddingTimestampBlockInputStream : public IBlockInputStream +{ +public: + AddingTimestampBlockInputStream(const BlockInputStreamPtr & input_, UInt32 timestamp_) : input(input_), timestamp(timestamp_) + { + cached_header = input->getHeader(); + cached_header.insert({ColumnUInt32::create(1, 1), std::make_shared(), "____timestamp"}); + } + + String getName() const override { return "AddingTimestamp"; } + + Block getHeader() const override { return cached_header.cloneEmpty(); } + +protected: + Block readImpl() override + { + Block res = input->read(); + if (res) + res.insert({ColumnUInt32::create(res.rows(), timestamp), std::make_shared(), "____timestamp"}); + return res; + } + +private: + BlockInputStreamPtr input; + Block cached_header; + UInt32 timestamp; +}; +} diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 8d1c3913b45..0505bf536c8 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -38,9 +38,10 @@ #include #include +#include +#include #include #include -#include #include @@ -111,8 +112,8 @@ namespace std::static_pointer_cast(ptr_)->setAlias(""); auto arrayJoin = makeASTFunction("arrayJoin", ptr_); arrayJoin->alias = node.alias; - data.window_column_name = arrayJoin->getColumnName(); node_ptr = arrayJoin; + data.window_column_name = arrayJoin->getColumnName(); } else if (serializeAST(node) != serializeAST(*data.window_function)) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); @@ -120,6 +121,43 @@ namespace } }; + class ParserProcTimeFinalMatcher + { + public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + bool is_proctime_tumble = false; + String window_column_name; + }; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) + { + return true; + } + + static void visit(ASTPtr & ast, Data & data) + { + if (const auto * t = ast->as()) + visit(*t, ast, data); + } + + private: + static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data & data) + { + if (node.name == "TUMBLE") + { + if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") + { + data.is_proctime_tumble = true; + node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); + data.window_column_name = node.getColumnName(); + } + } + } + }; + static inline IntervalKind strToIntervalKind(const String& interval_str) { if (interval_str == "Second") @@ -299,7 +337,7 @@ inline void StorageWindowView::clearInnerTable() inline void StorageWindowView::flushToTable(UInt32 timestamp_) { //write into dependent table - StoragePtr target_table = getTargetTable(); + StoragePtr target_table = getTargetStorage(); auto _blockInputStreamPtr = getNewBlocksInputStreamPtr(timestamp_); auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); auto stream = target_table->write(getInnerQuery(), global_context); @@ -315,12 +353,21 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto new_columns_list = std::make_shared(); - auto storage = getParentStorage(); auto sample_block_ - = InterpreterSelectQuery(getInnerQuery(), global_context, storage, SelectQueryOptions(QueryProcessingStage::WithMergeableState)) + = InterpreterSelectQuery(getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) .getSampleBlock(); auto columns_list = std::make_shared(); + + if (is_proctime_tumble) + { + auto column_window = std::make_shared(); + column_window->name = window_column_name; + column_window->type + = makeASTFunction("Tuple", std::make_shared("DateTime"), std::make_shared("DateTime")); + columns_list->children.push_back(column_window); + } + for (auto & column_ : sample_block_.getColumnsWithTypeAndName()) { ParserIdentifierWithOptionalParameters parser; @@ -331,10 +378,10 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery column_dec->type = ast; columns_list->children.push_back(column_dec); } - auto column_fire_status = std::make_shared(); - column_fire_status->name = "____w_end"; - column_fire_status->type = std::make_shared("DateTime"); - columns_list->children.push_back(column_fire_status); + auto column_wend = std::make_shared(); + column_wend->name = "____w_end"; + column_wend->type = std::make_shared("DateTime"); + columns_list->children.push_back(column_wend); new_columns_list->set(new_columns_list->columns, columns_list); manual_create_query->set(manual_create_query->columns_list, new_columns_list); @@ -569,6 +616,13 @@ StorageWindowView::StorageWindowView( select_table_id = StorageID(select_database_name, select_table_name); inner_query = innerQueryParser(select_query); + final_query = inner_query->clone(); + ParserProcTimeFinalMatcher::Data final_query_data; + ParserProcTimeFinalMatcher::Visitor(final_query_data).visit(final_query); + is_proctime_tumble = final_query_data.is_proctime_tumble; + if (is_proctime_tumble) + window_column_name = final_query_data.window_column_name; + /// If the table is not specified - use the table `system.one` if (select_table_name.empty()) { @@ -581,7 +635,6 @@ StorageWindowView::StorageWindowView( if (!query.to_table.empty()) target_table_id = StorageID(query.to_database, query.to_table); - is_temporary = query.temporary; inner_table_clear_interval = local_context.getSettingsRef().window_view_inner_table_clean_interval.totalSeconds(); mergeable_blocks = std::make_shared>(); @@ -590,6 +643,9 @@ StorageWindowView::StorageWindowView( if (query.watermark_function) { + if (is_proctime_tumble) + throw Exception("WATERMARK is not support for Processing time processing.", ErrorCodes::INCORRECT_QUERY); + // parser watermark function const auto & watermark_function = std::static_pointer_cast(query.watermark_function); if (!startsWith(watermark_function->name, "toInterval")) @@ -603,7 +659,7 @@ StorageWindowView::StorageWindowView( { watermark_num_units = boost::lexical_cast(interval_units_p1->value.get()); } - catch (const boost::bad_lexical_cast & exec) + catch (const boost::bad_lexical_cast &) { throw Exception("Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); } @@ -661,8 +717,6 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parser window function ASTFunction & window_function = typeid_cast(*stageMergeableOneData.window_function); const auto & arguments = window_function.arguments->children; - const auto & arg0 = std::static_pointer_cast(arguments.at(0)); - timestamp_column_name = arg0->IAST::getAliasOrColumnName(); const auto & arg1 = std::static_pointer_cast(arguments.at(1)); if (!arg1 || !startsWith(arg1->name, "toInterval")) throw Exception("Illegal type of last argument of function " + arg1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); @@ -678,12 +732,17 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { UInt32 timestamp_now = std::time(nullptr); + auto block_stream = std::make_shared(block); + BlockInputStreams streams; + if (window_view.is_proctime_tumble) + streams = {std::make_shared(block_stream, timestamp_now)}; + else + streams = {block_stream}; - BlockInputStreams streams = {std::make_shared(block)}; auto window_proxy_storage = std::make_shared( StorageID("", "WindowViewProxyStorage"), window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); InterpreterSelectQuery select_block( - window_view.getInnerQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); + window_view.getFinalQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); auto data_mergeable_stream = std::make_shared(select_block.execute().in); @@ -701,35 +760,35 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con actions_->add(ExpressionAction::removeColumn("____tuple_arg")); BlockInputStreamPtr in_stream; - if (window_view.watermark_num_units != 0) - { - UInt32 watermark = window_view.getWatermark(timestamp_now); - actions_->add( - ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), std::make_shared(), "____watermark"})); - const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); - actions_->add(ExpressionAction::applyFunction(function_greater, Names{"____w_end", "____watermark"}, "____filter")); - actions_->add(ExpressionAction::removeColumn("____watermark")); - in_stream = std::make_shared(data_mergeable_stream, actions_, "____filter", true); - } - else - in_stream = std::make_shared(data_mergeable_stream, actions_); + UInt32 watermark = window_view.getWatermark(timestamp_now); + actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), + std::make_shared(), + "____watermark"})); + const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); + actions_->add(ExpressionAction::applyFunction(function_greater, Names{"____w_end", "____watermark"}, "____filter")); + actions_->add(ExpressionAction::removeColumn("____watermark")); + in_stream = std::make_shared(data_mergeable_stream, actions_, "____filter", true); if (!window_view.inner_table_id.empty()) { auto stream = window_view.getInnerStorage()->write(window_view.getInnerQuery(), context); - if (window_view.watermark_num_units != 0) + if (window_view.is_proctime_tumble) + { + std::unique_lock lock(window_view.fire_signal_mutex); + copyData(*in_stream, *stream); + } + else if (window_view.watermark_num_units != 0) { while (Block block_ = in_stream->read()) { + auto column_wend = block_.getByName("____w_end").column; stream->write(std::move(block_)); const ColumnUInt32::Container & wend_data - = static_cast(*block_.getByName("____w_end").column.get()).getData(); + = static_cast(*column_wend).getData(); for (size_t i = 0; i < wend_data.size(); ++i) { if (wend_data[i] < timestamp_now) - { window_view.addFireSignal(wend_data[i]); - } } } } @@ -739,13 +798,20 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con else { BlocksListPtr new_mergeable_blocks = std::make_shared(); + if (window_view.is_proctime_tumble) + { + std::unique_lock lock(window_view.fire_signal_mutex); + while (Block block_ = in_stream->read()) + new_mergeable_blocks->push_back(std::move(block_)); + } if (window_view.watermark_num_units != 0) { while (Block block_ = in_stream->read()) { + auto column_wend = block_.getByName("____w_end").column; new_mergeable_blocks->push_back(std::move(block_)); const ColumnUInt32::Container & wend_data - = static_cast(*block_.getByName("____w_end").column.get()).getData(); + = static_cast(*column_wend).getData(); for (size_t i = 0; i < wend_data.size(); ++i) { if (wend_data[i] < timestamp_now) @@ -766,16 +832,6 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con } } -StoragePtr StorageWindowView::getTargetTable() const -{ - return global_context.getTable(target_table_id); -} - -StoragePtr StorageWindowView::tryGetTargetTable() const -{ - return global_context.tryGetTable(target_table_id); -} - void StorageWindowView::startup() { // Start the working thread @@ -802,45 +858,41 @@ StorageWindowView::~StorageWindowView() BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 timestamp_) { + BlockInputStreamPtr stream; + if (!inner_table_id.empty()) - return getNewBlocksInputStreamPtrInnerTable(timestamp_); + { + auto & storage = getInnerStorage(); + InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); - if (mergeable_blocks->empty()) - return {std::make_shared(getHeader())}; + ColumnsWithTypeAndName columns_; + columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); - BlockInputStreams from; - auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); - BlockInputStreamPtr stream = std::make_shared(mergeable_blocks, sample_block_, timestamp_); - from.push_back(std::move(stream)); - auto proxy_storage = std::make_shared( - StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); - - InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); - BlockInputStreamPtr data = std::make_shared(select.execute().in); - return data; -} - -BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtrInnerTable(UInt32 timestamp_) -{ - auto & storage = getInnerStorage(); - InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); - - ColumnsWithTypeAndName columns_; - columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); - - ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(timestamp_)), std::make_shared(), "____w_end_now"})); - const auto & function_equals = FunctionFactory::instance().get("equals", global_context); - ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____w_end_now"}, "____filter")); - auto stream = std::make_shared(fetch.execute().in, actions_, "____filter", true); + ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); + actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(timestamp_)), + std::make_shared(), + "____timestamp_now"})); + const auto & function_equals = FunctionFactory::instance().get("equals", global_context); + ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); + actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____timestamp_now"}, "____filter")); + actions_->add(ExpressionAction::removeColumn("____w_end")); + actions_->add(ExpressionAction::removeColumn("____timestamp_now")); + stream = std::make_shared(fetch.execute().in, actions_, "____filter", true); + } + else + { + if (mergeable_blocks->empty()) + return std::make_shared(getHeader()); + auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); + stream = std::make_shared(mergeable_blocks, sample_block_, timestamp_); + } BlockInputStreams from; from.push_back(std::move(stream)); auto proxy_storage = std::make_shared( StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); - InterpreterSelectQuery select(getInnerQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); + InterpreterSelectQuery select(getFinalQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); return data; } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index b547982299a..854e666710c 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -25,21 +25,11 @@ public: ~StorageWindowView() override; String getName() const override { return "WindowView"; } - ASTPtr getInnerQuery() const { return inner_query->clone(); } - - /// It is passed inside the query and solved at its level. bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } - bool isTemporary() { return is_temporary; } - - bool hasActiveUsers() { return active_ptr.use_count() > 1; } - void checkTableCanBeDropped() const override; - StoragePtr getTargetTable() const; - StoragePtr tryGetTargetTable() const; - void drop(TableStructureWriteLockHolder &) override; void startup() override; @@ -56,14 +46,70 @@ public: BlocksListPtrs getMergeableBlocksList() { return mergeable_blocks; } std::shared_ptr getActivePtr() { return active_ptr; } - /// Read new data blocks that store query result BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 timestamp_); - BlockInputStreamPtr getNewBlocksInputStreamPtrInnerTable(UInt32 timestamp_); + static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); - BlocksPtr getNewBlocks(); +private: + ASTPtr inner_query; + ASTPtr final_query; + ASTPtr fetch_column_query; + + Context & global_context; + bool is_proctime_tumble{false}; + std::atomic shutdown_called{false}; + mutable Block sample_block; + UInt64 inner_table_clear_interval; + const DateLUTImpl & time_zone; + std::list fire_signal; + std::list watch_streams; + std::condition_variable condition; + BlocksListPtrs mergeable_blocks; + + /// Mutex for the blocks and ready condition + std::mutex mutex; + std::mutex flush_table_mutex; + std::mutex fire_signal_mutex; + std::mutex proc_time_signal_mutex; + + /// Active users + std::shared_ptr active_ptr; + + IntervalKind::Kind window_kind; + IntervalKind::Kind watermark_kind; + Int64 window_num_units; + Int64 watermark_num_units = 0; + String window_column_name; + + StorageID select_table_id = StorageID::createEmpty(); + StorageID target_table_id = StorageID::createEmpty(); + StorageID inner_table_id = StorageID::createEmpty(); + StoragePtr parent_storage; + StoragePtr inner_storage; + StoragePtr target_storage; + + BackgroundSchedulePool::TaskHolder toTableTask; + BackgroundSchedulePool::TaskHolder innerTableClearTask; + BackgroundSchedulePool::TaskHolder fireTask; + + ASTPtr innerQueryParser(ASTSelectQuery & inner_query); + + std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); + + UInt32 getWindowLowerBound(UInt32 time_sec, int window_id_skew = 0); + UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); + UInt32 getWatermark(UInt32 time_sec); Block getHeader() const; + void flushToTable(UInt32 timestamp_); + void clearInnerTable(); + void threadFuncToTable(); + void threadFuncClearInnerTable(); + void threadFuncFire(); + void addFireSignal(UInt32 timestamp_); + + ASTPtr getInnerQuery() const { return inner_query->clone(); } + ASTPtr getFinalQuery() const { return final_query->clone(); } StoragePtr& getParentStorage() { @@ -79,68 +125,12 @@ public: return inner_storage; } - static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); - - static void writeIntoWindowViewInnerTable(StorageWindowView & window_view, const Block & block, const Context & context); - - ASTPtr innerQueryParser(ASTSelectQuery & inner_query); - - std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); - - inline UInt32 getWindowLowerBound(UInt32 time_sec, int window_id_skew = 0); - inline UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); - inline UInt32 getWatermark(UInt32 time_sec); - -private: - StorageID select_table_id = StorageID::createEmpty(); - ASTPtr inner_query; - ASTPtr fetch_column_query; - String window_column_name; - String timestamp_column_name; - Context & global_context; - StoragePtr parent_storage; - StoragePtr inner_storage; - bool is_temporary{false}; - mutable Block sample_block; - - /// Mutex for the blocks and ready condition - std::mutex mutex; - std::mutex flush_table_mutex; - std::mutex fire_signal_mutex; - std::list fire_signal; - std::list watch_streams; - /// New blocks ready condition to broadcast to readers - /// that new blocks are available - std::condition_variable condition; - - /// Active users - std::shared_ptr active_ptr; - BlocksListPtrs mergeable_blocks; - - IntervalKind::Kind window_kind; - Int64 window_num_units; - IntervalKind::Kind watermark_kind; - Int64 watermark_num_units = 0; - const DateLUTImpl & time_zone; - - StorageID target_table_id = StorageID::createEmpty(); - StorageID inner_table_id = StorageID::createEmpty(); - - void flushToTable(UInt32 timestamp_); - void clearInnerTable(); - void threadFuncToTable(); - void threadFuncClearInnerTable(); - void threadFuncFire(); - void addFireSignal(UInt32 timestamp_); - - std::atomic shutdown_called{false}; - UInt64 inner_table_clear_interval; - - Poco::Timestamp timestamp; - - BackgroundSchedulePool::TaskHolder toTableTask; - BackgroundSchedulePool::TaskHolder innerTableClearTask; - BackgroundSchedulePool::TaskHolder fireTask; + StoragePtr& getTargetStorage() + { + if (target_storage == nullptr && !target_table_id.empty()) + target_storage = global_context.getTable(target_table_id); + return target_storage; + } StorageWindowView( const StorageID & table_id_, diff --git a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h index b2529f861b7..e449b5a9f0c 100644 --- a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h +++ b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h @@ -4,6 +4,7 @@ #include #include #include +#include #include namespace DB @@ -12,11 +13,25 @@ namespace DB class WindowViewProxyStorage : public IStorage { public: + WindowViewProxyStorage(const StorageID & table_id_, StoragePtr parent_storage_, QueryProcessingStage::Enum to_stage_) + : IStorage(table_id_) + , parent_storage(std::move(parent_storage_)) + , streams({std::make_shared(Block())}) + , to_stage(to_stage_) + { + column_des = parent_storage->getColumns(); + column_des.add({"____timestamp", std::make_shared(), false}); + } + WindowViewProxyStorage(const StorageID & table_id_, StoragePtr parent_storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) : IStorage(table_id_) , parent_storage(std::move(parent_storage_)) , streams(std::move(streams_)) - , to_stage(to_stage_) {} + , to_stage(to_stage_) + { + column_des = parent_storage->getColumns(); + column_des.add({"____timestamp", std::make_shared(), false}); + } public: std::string getName() const override { return "WindowViewProxyStorage(" + parent_storage->getName() + ")"; } @@ -53,17 +68,29 @@ public: Names getColumnsRequiredForSampling() const override { return parent_storage->getColumnsRequiredForSampling(); } Names getColumnsRequiredForFinal() const override { return parent_storage->getColumnsRequiredForFinal(); } - const ColumnsDescription & getColumns() const override { return parent_storage->getColumns(); } + const ColumnsDescription & getColumns() const override { return column_des; } void setColumns(ColumnsDescription columns_) override { return parent_storage->setColumns(columns_); } - NameAndTypePair getColumn(const String & column_name) const override { return parent_storage->getColumn(column_name); } + NameAndTypePair getColumn(const String & column_name) const override + { + if (column_name == "____timestamp") + return {"____timestamp", std::shared_ptr()}; + return parent_storage->getColumn(column_name); + } - bool hasColumn(const String & column_name) const override { return parent_storage->hasColumn(column_name); } + bool hasColumn(const String & column_name) const override + { + if (column_name == "____timestamp") + return true; + return parent_storage->hasColumn(column_name); + } private: StoragePtr parent_storage; BlockInputStreams streams; + String window_column_name; + ColumnsDescription column_des; QueryProcessingStage::Enum to_stage; }; } diff --git a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference new file mode 100644 index 00000000000..048630d233f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference @@ -0,0 +1,10 @@ +--TUMBLE-- +1 +--HOP-- +1 +1 +--INNER_TUMBLE-- +1 +--INNER_HOP-- +1 +1 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql new file mode 100644 index 00000000000..ca29f1a36e6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql @@ -0,0 +1,35 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; + +SELECT '--TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); +WATCH test.wv LIMIT 1; + +SELECT '--HOP--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); +WATCH test.wv LIMIT 2; + +SELECT '--INNER_TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); +WATCH test.wv LIMIT 1; + +SELECT '--INNER_HOP--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); +WATCH test.wv LIMIT 2; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql b/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql deleted file mode 100644 index dfa2ef9d1a6..00000000000 --- a/dbms/tests/queries/0_stateless/01052_window_view_watch_tumble.sql +++ /dev/null @@ -1,13 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -WATCH test.wv LIMIT 1; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference b/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference new file mode 100644 index 00000000000..670c8eb78c8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference @@ -0,0 +1,14 @@ +--TUMBLE-- +0 +1 +--HOP-- +0 +1 +1 +--INNER_TUMBLE-- +0 +1 +--INNER_HOP-- +0 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql new file mode 100644 index 00000000000..fa93cffead1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql @@ -0,0 +1,47 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); + +SELECT '--TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count FROM test.dst; + +SELECT '--HOP--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count FROM test.dst; + +SELECT '--INNER_TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count FROM test.dst; + +SELECT '--INNER_HOP--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference deleted file mode 100644 index 6ed281c757a..00000000000 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql b/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql deleted file mode 100644 index 1ddd273c103..00000000000 --- a/dbms/tests/queries/0_stateless/01053_window_view_watch_hop.sql +++ /dev/null @@ -1,13 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -WATCH test.wv LIMIT 2; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql b/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql deleted file mode 100644 index 0d7c94047f1..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_inner_watch_tumble.sql +++ /dev/null @@ -1,13 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -WATCH test.wv LIMIT 1; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference b/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference new file mode 100644 index 00000000000..048630d233f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference @@ -0,0 +1,10 @@ +--TUMBLE-- +1 +--HOP-- +1 +1 +--INNER_TUMBLE-- +1 +--INNER_HOP-- +1 +1 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql new file mode 100644 index 00000000000..926ae776ce5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql @@ -0,0 +1,35 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); + +SELECT '--TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +WATCH test.wv LIMIT 1; + +SELECT '--HOP--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +WATCH test.wv LIMIT 2; + +SELECT '--INNER_TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +WATCH test.wv LIMIT 1; + +SELECT '--INNER_HOP--'; +DROP TABLE IF EXISTS test.wv; +CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +WATCH test.wv LIMIT 2; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference deleted file mode 100644 index 6ed281c757a..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql b/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql deleted file mode 100644 index 35cccdaf0b8..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_inner_watch_hop.sql +++ /dev/null @@ -1,13 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -WATCH test.wv LIMIT 2; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference new file mode 100644 index 00000000000..670c8eb78c8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference @@ -0,0 +1,14 @@ +--TUMBLE-- +0 +1 +--HOP-- +0 +1 +1 +--INNER_TUMBLE-- +0 +1 +--INNER_HOP-- +0 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql new file mode 100644 index 00000000000..38373e9af1c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql @@ -0,0 +1,47 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); + +SELECT '--TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(1); +SELECT count FROM test.dst; + +SELECT '--HOP--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(2); +SELECT count FROM test.dst; + +SELECT '--INNER_TUMBLE--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(1); +SELECT count FROM test.dst; + +SELECT '--INNER_HOP--'; +DROP TABLE IF EXISTS test.wv; +TRUNCATE TABLE test.dst; +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(2); +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference b/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql b/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql deleted file mode 100644 index 1169ae06eb4..00000000000 --- a/dbms/tests/queries/0_stateless/01056_window_view_to_tumble.sql +++ /dev/null @@ -1,18 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(1); -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference b/dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference deleted file mode 100644 index 986394f7c0f..00000000000 --- a/dbms/tests/queries/0_stateless/01057_window_view_to_hop.reference +++ /dev/null @@ -1,3 +0,0 @@ -0 -1 -1 diff --git a/dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql b/dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql deleted file mode 100644 index a6227913d9d..00000000000 --- a/dbms/tests/queries/0_stateless/01057_window_view_to_hop.sql +++ /dev/null @@ -1,18 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE TABLE test.dst(count UInt64) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(3); -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference b/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql b/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql deleted file mode 100644 index 1f4a580b5b3..00000000000 --- a/dbms/tests/queries/0_stateless/01058_window_view_inner_to_tumble.sql +++ /dev/null @@ -1,18 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE TABLE test.dst(count UInt64) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(1); -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference b/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference deleted file mode 100644 index 986394f7c0f..00000000000 --- a/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.reference +++ /dev/null @@ -1,3 +0,0 @@ -0 -1 -1 diff --git a/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql b/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql deleted file mode 100644 index 49e0f67f14a..00000000000 --- a/dbms/tests/queries/0_stateless/01059_window_view_inner_to_hop.sql +++ /dev/null @@ -1,18 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.wv; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; - -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE TABLE test.dst(count UInt64) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(3); -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file From 35600d6a9983ec3a4c72ec9f46668d4f05a10df9 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 17 Feb 2020 16:18:27 +0800 Subject: [PATCH 020/609] trauncate and optimize support --- dbms/src/Core/Defines.h | 2 +- dbms/src/Core/Settings.h | 2 +- .../Storages/WindowView/StorageWindowView.cpp | 43 +++++++++++++------ .../Storages/WindowView/StorageWindowView.h | 14 +++--- .../01052_window_view_event_time_watch.sql | 10 ++--- .../01055_window_view_proc_time_to.sql | 10 ++--- 6 files changed, 50 insertions(+), 31 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index c84ab12a6b4..fca458c0e90 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -34,7 +34,7 @@ #define DEFAULT_MERGE_BLOCK_SIZE 8192 #define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 -#define DEFAULT_WINDOW_VIEW_INNER_TABLE_CLEAN_INTERVAL_SEC 5 +#define DEFAULT_WINDOW_VIEW_CLEAN_INTERVAL_SEC 5 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) #define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index e27a9442a3e..0c0d5dd961e 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -387,7 +387,7 @@ struct Settings : public SettingsCollection M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ M(SettingBool, allow_experimental_window_view, false, "Enable WINDOW VIEW. Not mature enough.", 0) \ - M(SettingSeconds, window_view_inner_table_clean_interval, DEFAULT_WINDOW_VIEW_INNER_TABLE_CLEAN_INTERVAL_SEC, "The clean interval of window view inner table in seconds to free outdated data.", 0) \ + M(SettingSeconds, window_view_clean_interval, DEFAULT_WINDOW_VIEW_CLEAN_INTERVAL_SEC, "The clean interval of window view in seconds to free outdated data.", 0) \ M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 0505bf536c8..8b5c5b8deae 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -276,7 +276,6 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, { if (global_context.tryGetTable(target_table_id)) { - /// We create and execute `drop` query for internal table. auto drop_query = std::make_shared(); drop_query->database = target_table_id.database_name; drop_query->table = target_table_id.table_name; @@ -300,7 +299,26 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &) condition.notify_all(); } -inline void StorageWindowView::clearInnerTable() +void StorageWindowView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) +{ + if (!inner_table_id.empty()) + executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, inner_table_id); + else + { + std::lock_guard lock(mutex); + mergeable_blocks = std::make_shared>(); + } +} + +bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) +{ + if (inner_table_id.empty()) + throw Exception( + "OPTIMIZE only supported when creating WINDOW VIEW within INNER table.", ErrorCodes::INCORRECT_QUERY); + return getInnerStorage()->optimize(query, partition, final, deduplicate, context); +} + +inline void StorageWindowView::cleanCache() { //delete fired blocks UInt32 timestamp_now = std::time(nullptr); @@ -477,14 +495,14 @@ inline void StorageWindowView::addFireSignal(UInt32 timestamp_) condition.notify_all(); } -void StorageWindowView::threadFuncClearInnerTable() +void StorageWindowView::threadFuncCleanCache() { while (!shutdown_called) { try { - clearInnerTable(); - sleep(inner_table_clear_interval); + cleanCache(); + sleep(clean_interval); } catch (...) { @@ -493,7 +511,7 @@ void StorageWindowView::threadFuncClearInnerTable() } } if (!shutdown_called) - innerTableClearTask->scheduleAfter(RESCHEDULE_MS); + cleanCacheTask->scheduleAfter(RESCHEDULE_MS); } void StorageWindowView::threadFuncToTable() @@ -635,10 +653,8 @@ StorageWindowView::StorageWindowView( if (!query.to_table.empty()) target_table_id = StorageID(query.to_database, query.to_table); - inner_table_clear_interval = local_context.getSettingsRef().window_view_inner_table_clean_interval.totalSeconds(); - + clean_interval = local_context.getSettingsRef().window_view_clean_interval.totalSeconds(); mergeable_blocks = std::make_shared>(); - active_ptr = std::make_shared(true); if (query.watermark_function) @@ -692,10 +708,10 @@ StorageWindowView::StorageWindowView( } toTableTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncToTable(); }); - innerTableClearTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncClearInnerTable(); }); + cleanCacheTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFire(); }); toTableTask->deactivate(); - innerTableClearTask->deactivate(); + cleanCacheTask->deactivate(); fireTask->deactivate(); } @@ -743,7 +759,6 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con StorageID("", "WindowViewProxyStorage"), window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); InterpreterSelectQuery select_block( window_view.getFinalQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); - auto data_mergeable_stream = std::make_shared(select_block.execute().in); // extract ____w_end @@ -837,7 +852,7 @@ void StorageWindowView::startup() // Start the working thread if (!target_table_id.empty()) toTableTask->activateAndSchedule(); - innerTableClearTask->activateAndSchedule(); + // cleanCacheTask->activateAndSchedule(); fireTask->activateAndSchedule(); } @@ -847,7 +862,7 @@ void StorageWindowView::shutdown() if (!shutdown_called.compare_exchange_strong(expected, true)) return; toTableTask->deactivate(); - innerTableClearTask->deactivate(); + cleanCacheTask->deactivate(); fireTask->deactivate(); } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 854e666710c..cfb9ad129ca 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -32,6 +32,10 @@ public: void drop(TableStructureWriteLockHolder &) override; + void truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) override; + + bool optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) override; + void startup() override; void shutdown() override; @@ -54,12 +58,12 @@ private: ASTPtr inner_query; ASTPtr final_query; ASTPtr fetch_column_query; - + Context & global_context; bool is_proctime_tumble{false}; std::atomic shutdown_called{false}; mutable Block sample_block; - UInt64 inner_table_clear_interval; + UInt64 clean_interval; const DateLUTImpl & time_zone; std::list fire_signal; std::list watch_streams; @@ -89,7 +93,7 @@ private: StoragePtr target_storage; BackgroundSchedulePool::TaskHolder toTableTask; - BackgroundSchedulePool::TaskHolder innerTableClearTask; + BackgroundSchedulePool::TaskHolder cleanCacheTask; BackgroundSchedulePool::TaskHolder fireTask; ASTPtr innerQueryParser(ASTSelectQuery & inner_query); @@ -102,9 +106,9 @@ private: Block getHeader() const; void flushToTable(UInt32 timestamp_); - void clearInnerTable(); + void cleanCache(); void threadFuncToTable(); - void threadFuncClearInnerTable(); + void threadFuncCleanCache(); void threadFuncFire(); void addFireSignal(UInt32 timestamp_); diff --git a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql index ca29f1a36e6..58482549657 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql +++ b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql @@ -1,35 +1,35 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.wv; SELECT '--TUMBLE--'; -DROP TABLE IF EXISTS test.wv; CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); WATCH test.wv LIMIT 1; +DROP TABLE test.wv; SELECT '--HOP--'; -DROP TABLE IF EXISTS test.wv; CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); WATCH test.wv LIMIT 2; +DROP TABLE test.wv; SELECT '--INNER_TUMBLE--'; -DROP TABLE IF EXISTS test.wv; CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); WATCH test.wv LIMIT 1; +DROP TABLE test.wv; SELECT '--INNER_HOP--'; -DROP TABLE IF EXISTS test.wv; CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); WATCH test.wv LIMIT 2; - DROP TABLE test.wv; + DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql index 38373e9af1c..b660fcd4a5c 100644 --- a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql +++ b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql @@ -3,45 +3,45 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS test.mt; DROP TABLE IF EXISTS test.dst; DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); SELECT '--TUMBLE--'; -DROP TABLE IF EXISTS test.wv; TRUNCATE TABLE test.dst; CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1); SELECT sleep(1); SELECT count FROM test.dst; +DROP TABLE test.wv; SELECT '--HOP--'; -DROP TABLE IF EXISTS test.wv; TRUNCATE TABLE test.dst; CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; INSERT INTO test.mt VALUES (1); SELECT sleep(2); SELECT count FROM test.dst; +DROP TABLE test.wv; SELECT '--INNER_TUMBLE--'; -DROP TABLE IF EXISTS test.wv; TRUNCATE TABLE test.dst; CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1); SELECT sleep(1); SELECT count FROM test.dst; +DROP TABLE test.wv; SELECT '--INNER_HOP--'; -DROP TABLE IF EXISTS test.wv; TRUNCATE TABLE test.dst; CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; INSERT INTO test.mt VALUES (1); SELECT sleep(2); SELECT count FROM test.dst; - DROP TABLE test.wv; + DROP TABLE test.mt; From 328150797dc645bbd623846a7eae3f1507e99940 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 17 Feb 2020 23:03:39 +0800 Subject: [PATCH 021/609] try to fix stress test error --- dbms/src/Storages/WindowView/StorageWindowView.cpp | 2 ++ dbms/src/Storages/WindowView/WindowViewBlockInputStream.h | 7 +------ 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 8b5c5b8deae..9d2e9f4680e 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -800,6 +800,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con stream->write(std::move(block_)); const ColumnUInt32::Container & wend_data = static_cast(*column_wend).getData(); + std::unique_lock lock(window_view.fire_signal_mutex); for (size_t i = 0; i < wend_data.size(); ++i) { if (wend_data[i] < timestamp_now) @@ -827,6 +828,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con new_mergeable_blocks->push_back(std::move(block_)); const ColumnUInt32::Container & wend_data = static_cast(*column_wend).getData(); + std::unique_lock lock(window_view.fire_signal_mutex); for (size_t i = 0; i < wend_data.size(); ++i) { if (wend_data[i] < timestamp_now) diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 89f15dad943..d2bd151868a 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -43,7 +43,7 @@ public: Block getHeader() const override { return storage->getHeader(); } - inline void addFireSignal(UInt32 timestamp) + void addFireSignal(UInt32 timestamp) { std::lock_guard lock(fire_signal_mutex); fire_signal.push_back(timestamp); @@ -56,11 +56,6 @@ protected: return tryReadImpl(); } - /** tryRead method attempts to read a block in either blocking - * or non-blocking mode. If blocking is set to false - * then method return empty block with flag set to false - * to indicate that method would block to get the next block. - */ Block tryReadImpl() { Block res; From bc019649733ee01fa291dae9ab9f16a38fd066a0 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 21 Feb 2020 01:30:58 +0800 Subject: [PATCH 022/609] optimize write --- .../AddingTimestampBlockInputStream.h | 39 -------- .../Storages/WindowView/StorageWindowView.cpp | 88 +++++++++++-------- .../Storages/WindowView/StorageWindowView.h | 4 +- .../WindowView/WindowViewBlockInputStream.h | 11 +-- 4 files changed, 53 insertions(+), 89 deletions(-) delete mode 100644 dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h diff --git a/dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h b/dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h deleted file mode 100644 index a307fccc3bc..00000000000 --- a/dbms/src/Storages/WindowView/AddingTimestampBlockInputStream.h +++ /dev/null @@ -1,39 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ -/** Add timestamp column for processing time process in - * WINDOW VIEW - */ -class AddingTimestampBlockInputStream : public IBlockInputStream -{ -public: - AddingTimestampBlockInputStream(const BlockInputStreamPtr & input_, UInt32 timestamp_) : input(input_), timestamp(timestamp_) - { - cached_header = input->getHeader(); - cached_header.insert({ColumnUInt32::create(1, 1), std::make_shared(), "____timestamp"}); - } - - String getName() const override { return "AddingTimestamp"; } - - Block getHeader() const override { return cached_header.cloneEmpty(); } - -protected: - Block readImpl() override - { - Block res = input->read(); - if (res) - res.insert({ColumnUInt32::create(res.rows(), timestamp), std::make_shared(), "____timestamp"}); - return res; - } - -private: - BlockInputStreamPtr input; - Block cached_header; - UInt32 timestamp; -}; -} diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 9d2e9f4680e..1da8c620468 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -38,7 +39,6 @@ #include #include -#include #include #include #include @@ -350,6 +350,11 @@ inline void StorageWindowView::cleanCache() } mergeable_blocks->remove_if([](BlocksListPtr & ptr) { return ptr->size() == 0; }); } + + { + std::lock_guard lock(fire_signal_mutex); + watch_streams.remove_if([](std::weak_ptr & ptr) { return ptr.expired(); }); + } } inline void StorageWindowView::flushToTable(UInt32 timestamp_) @@ -487,10 +492,10 @@ inline void StorageWindowView::addFireSignal(UInt32 timestamp_) { if (!target_table_id.empty()) fire_signal.push_back(timestamp_); - for (auto watch_stream : watch_streams) + for (auto & watch_stream : watch_streams) { - if (watch_stream) - watch_stream->addFireSignal(timestamp_); + if (auto watch_stream_ = watch_stream.lock()) + watch_stream_->addFireSignal(timestamp_); } condition.notify_all(); } @@ -584,7 +589,7 @@ BlockInputStreams StorageWindowView::watch( { std::lock_guard lock(fire_signal_mutex); - watch_streams.push_back(reader.get()); + watch_streams.push_back(reader); } processed_stage = QueryProcessingStage::Complete; @@ -707,6 +712,27 @@ StorageWindowView::StorageWindowView( fetch_column_query = generateFetchColumnsQuery(inner_table_id); } + { + // generate write expressions + ColumnsWithTypeAndName columns__; + columns__.emplace_back( + nullptr, + std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), + window_column_name); + columns__.emplace_back(nullptr, std::make_shared(), "____timestamp"); + columns__.emplace_back(nullptr, std::make_shared(), "____watermark"); + const auto & function_tuple = FunctionFactory::instance().get("tupleElement", global_context); + writeExpressions = std::make_shared(columns__, global_context); + writeExpressions->add(ExpressionAction::addColumn( + {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); + writeExpressions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); + writeExpressions->add(ExpressionAction::removeColumn("____tuple_arg")); + + const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", global_context); + writeExpressions->add(ExpressionAction::applyFunction(function_greater, Names{"____w_end", "____watermark"}, "____filter")); + writeExpressions->add(ExpressionAction::removeColumn("____watermark")); + } + toTableTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncToTable(); }); cleanCacheTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFire(); }); @@ -748,41 +774,27 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { UInt32 timestamp_now = std::time(nullptr); + UInt32 watermark; auto block_stream = std::make_shared(block); - BlockInputStreams streams; + BlockInputStreamPtr source_stream; if (window_view.is_proctime_tumble) - streams = {std::make_shared(block_stream, timestamp_now)}; + { + source_stream = std::make_shared>(block_stream, std::make_shared(), timestamp_now, "____timestamp"); + watermark = window_view.getWindowLowerBound(timestamp_now); + } else - streams = {block_stream}; + { + source_stream = block_stream; + watermark = window_view.getWatermark(timestamp_now); + } - auto window_proxy_storage = std::make_shared( - StorageID("", "WindowViewProxyStorage"), window_view.getParentStorage(), std::move(streams), QueryProcessingStage::FetchColumns); InterpreterSelectQuery select_block( - window_view.getFinalQuery(), context, window_proxy_storage, QueryProcessingStage::WithMergeableState); - auto data_mergeable_stream = std::make_shared(select_block.execute().in); - - // extract ____w_end - ColumnsWithTypeAndName columns_; - columns_.emplace_back( - nullptr, - std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), - window_view.window_column_name); - const auto & function_tuple = FunctionFactory::instance().get("tupleElement", context); - ExpressionActionsPtr actions_ = std::make_shared(columns_, context); - actions_->add(ExpressionAction::addColumn( - {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); - actions_->add(ExpressionAction::applyFunction(function_tuple, Names{window_view.window_column_name, "____tuple_arg"}, "____w_end")); - actions_->add(ExpressionAction::removeColumn("____tuple_arg")); + window_view.getFinalQuery(), context, source_stream, QueryProcessingStage::WithMergeableState); + source_stream = std::make_shared(select_block.execute().in); + source_stream = std::make_shared>(source_stream, std::make_shared(), watermark, "____watermark"); BlockInputStreamPtr in_stream; - UInt32 watermark = window_view.getWatermark(timestamp_now); - actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), - std::make_shared(), - "____watermark"})); - const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); - actions_->add(ExpressionAction::applyFunction(function_greater, Names{"____w_end", "____watermark"}, "____filter")); - actions_->add(ExpressionAction::removeColumn("____watermark")); - in_stream = std::make_shared(data_mergeable_stream, actions_, "____filter", true); + in_stream = std::make_shared(source_stream, window_view.writeExpressions, "____filter", true); if (!window_view.inner_table_id.empty()) { @@ -841,11 +853,9 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con while (Block block_ = in_stream->read()) new_mergeable_blocks->push_back(std::move(block_)); } - if (!new_mergeable_blocks->empty()) - { - std::unique_lock lock(window_view.mutex); - window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); - } + + std::unique_lock lock(window_view.mutex); + window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); } } @@ -854,7 +864,7 @@ void StorageWindowView::startup() // Start the working thread if (!target_table_id.empty()) toTableTask->activateAndSchedule(); - // cleanCacheTask->activateAndSchedule(); + cleanCacheTask->activateAndSchedule(); fireTask->activateAndSchedule(); } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index cfb9ad129ca..f1680811c12 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -66,7 +66,7 @@ private: UInt64 clean_interval; const DateLUTImpl & time_zone; std::list fire_signal; - std::list watch_streams; + std::list> watch_streams; std::condition_variable condition; BlocksListPtrs mergeable_blocks; @@ -96,6 +96,8 @@ private: BackgroundSchedulePool::TaskHolder cleanCacheTask; BackgroundSchedulePool::TaskHolder fireTask; + ExpressionActionsPtr writeExpressions; + ASTPtr innerQueryParser(ASTSelectQuery & inner_query); std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index d2bd151868a..250dc5ef057 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -30,15 +30,6 @@ public: if (isCancelled() || storage->is_dropped) return; IBlockInputStream::cancel(kill); - std::lock_guard lock(storage->fire_signal_mutex); - for (auto it = storage->watch_streams.begin() ; it != storage->watch_streams.end() ; ++it) - { - if (*it == this) - { - storage->watch_streams.erase(it); - break; - } - } } Block getHeader() const override { return storage->getHeader(); } @@ -125,6 +116,6 @@ private: bool end_of_blocks = false; BlockInputStreamPtr in_stream; std::mutex fire_signal_mutex; - std::list fire_signal; + std::deque fire_signal; }; } From 6d35d8ccaf0f333a8eda3452e4cc6ec9d0645020 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 21 Feb 2020 20:35:26 +0800 Subject: [PATCH 023/609] optimize write --- .../Storages/WindowView/StorageWindowView.cpp | 78 +++++-------------- .../Storages/WindowView/StorageWindowView.h | 6 +- .../WindowView/WatermarkBlockInputStream.h | 62 +++++++++++++++ .../WindowView/WindowViewBlockInputStream.h | 4 +- 4 files changed, 89 insertions(+), 61 deletions(-) create mode 100644 dbms/src/Storages/WindowView/WatermarkBlockInputStream.h diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 1da8c620468..cdc06b93757 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -41,6 +41,7 @@ #include #include +#include #include #include @@ -490,6 +491,7 @@ inline UInt32 StorageWindowView::getWatermark(UInt32 time_sec) inline void StorageWindowView::addFireSignal(UInt32 timestamp_) { + std::unique_lock lock(fire_signal_mutex); if (!target_table_id.empty()) fire_signal.push_back(timestamp_); for (auto & watch_stream : watch_streams) @@ -527,11 +529,13 @@ void StorageWindowView::threadFuncToTable() condition.wait_for(lock, std::chrono::seconds(5)); try { - while (!fire_signal.empty()) + while (true) { UInt32 timestamp_; { std::unique_lock lock_(fire_signal_mutex); + if (fire_signal.empty()) + break; timestamp_ = fire_signal.front(); fire_signal.pop_front(); } @@ -555,7 +559,6 @@ void StorageWindowView::threadFuncFire() UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); UInt64 w_end = static_cast(getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; std::this_thread::sleep_for(std::chrono::microseconds(w_end - timestamp_usec)); - std::unique_lock lock(fire_signal_mutex); addFireSignal(static_cast(w_end / 1000000)); } if (!shutdown_called) @@ -773,86 +776,47 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { - UInt32 timestamp_now = std::time(nullptr); + UInt32 timestamp_now; UInt32 watermark; auto block_stream = std::make_shared(block); BlockInputStreamPtr source_stream; + + std::shared_lock fire_signal_lock; if (window_view.is_proctime_tumble) { + fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); + timestamp_now = std::time(nullptr); source_stream = std::make_shared>(block_stream, std::make_shared(), timestamp_now, "____timestamp"); watermark = window_view.getWindowLowerBound(timestamp_now); } else { source_stream = block_stream; + timestamp_now = std::time(nullptr); watermark = window_view.getWatermark(timestamp_now); } InterpreterSelectQuery select_block( window_view.getFinalQuery(), context, source_stream, QueryProcessingStage::WithMergeableState); - source_stream = std::make_shared(select_block.execute().in); - source_stream = std::make_shared>(source_stream, std::make_shared(), watermark, "____watermark"); + source_stream = std::make_shared>(select_block.execute().in, std::make_shared(), watermark, "____watermark"); + source_stream = std::make_shared(source_stream); - BlockInputStreamPtr in_stream; - in_stream = std::make_shared(source_stream, window_view.writeExpressions, "____filter", true); + source_stream = std::make_shared(source_stream, window_view.writeExpressions, "____filter", true); + if (window_view.watermark_num_units != 0) + source_stream = std::make_shared(source_stream, window_view, timestamp_now); if (!window_view.inner_table_id.empty()) { - auto stream = window_view.getInnerStorage()->write(window_view.getInnerQuery(), context); - if (window_view.is_proctime_tumble) - { - std::unique_lock lock(window_view.fire_signal_mutex); - copyData(*in_stream, *stream); - } - else if (window_view.watermark_num_units != 0) - { - while (Block block_ = in_stream->read()) - { - auto column_wend = block_.getByName("____w_end").column; - stream->write(std::move(block_)); - const ColumnUInt32::Container & wend_data - = static_cast(*column_wend).getData(); - std::unique_lock lock(window_view.fire_signal_mutex); - for (size_t i = 0; i < wend_data.size(); ++i) - { - if (wend_data[i] < timestamp_now) - window_view.addFireSignal(wend_data[i]); - } - } - } - else - copyData(*in_stream, *stream); + auto & inner_storage = window_view.getInnerStorage(); + auto lock_ = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); + auto stream = inner_storage->write(window_view.getInnerQuery(), context); + copyData(*source_stream, *stream); } else { BlocksListPtr new_mergeable_blocks = std::make_shared(); - if (window_view.is_proctime_tumble) - { - std::unique_lock lock(window_view.fire_signal_mutex); - while (Block block_ = in_stream->read()) + while (Block block_ = source_stream->read()) new_mergeable_blocks->push_back(std::move(block_)); - } - if (window_view.watermark_num_units != 0) - { - while (Block block_ = in_stream->read()) - { - auto column_wend = block_.getByName("____w_end").column; - new_mergeable_blocks->push_back(std::move(block_)); - const ColumnUInt32::Container & wend_data - = static_cast(*column_wend).getData(); - std::unique_lock lock(window_view.fire_signal_mutex); - for (size_t i = 0; i < wend_data.size(); ++i) - { - if (wend_data[i] < timestamp_now) - window_view.addFireSignal(wend_data[i]); - } - } - } - else - { - while (Block block_ = in_stream->read()) - new_mergeable_blocks->push_back(std::move(block_)); - } std::unique_lock lock(window_view.mutex); window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index f1680811c12..912d34bd42f 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -19,6 +19,7 @@ using BlocksListPtrs = std::shared_ptr>; class StorageWindowView : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; + friend class WatermarkBlockInputStream; friend class WindowViewBlockInputStream; public: @@ -65,7 +66,7 @@ private: mutable Block sample_block; UInt64 clean_interval; const DateLUTImpl & time_zone; - std::list fire_signal; + std::deque fire_signal; std::list> watch_streams; std::condition_variable condition; BlocksListPtrs mergeable_blocks; @@ -73,8 +74,7 @@ private: /// Mutex for the blocks and ready condition std::mutex mutex; std::mutex flush_table_mutex; - std::mutex fire_signal_mutex; - std::mutex proc_time_signal_mutex; + std::shared_mutex fire_signal_mutex; /// Active users std::shared_ptr active_ptr; diff --git a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h new file mode 100644 index 00000000000..444629ea209 --- /dev/null +++ b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +/** Adds a materialized const column to the block with a specified value. + */ +class WatermarkBlockInputStream : public IBlockInputStream +{ +public: + WatermarkBlockInputStream( + BlockInputStreamPtr input_, + StorageWindowView& storage_, + UInt32 timestamp_) + : storage(storage_), timestamp(timestamp_) + { + children.push_back(input_); + } + + String getName() const override { return "Watermark"; } + + Block getHeader() const override + { + return children.back()->getHeader(); + } + +protected: + Block readImpl() override + { + Block res = children.back()->read(); + if (!res) + return res; + + auto column_wend = res.getByName("____w_end").column; + const ColumnUInt32::Container & wend_data = static_cast(*column_wend).getData(); + for (size_t i = 0; i < wend_data.size(); ++i) + { + if (wend_data[i] < timestamp) + signals.push_back(wend_data[i]); + } + return res; + } + + void readSuffix() override + { + // while (!signal.empty()) + for (auto signal : signals) + storage.addFireSignal(signal); + signals.clear(); + } + +private: + StorageWindowView & storage; + UInt32 timestamp; + std::deque signals; +}; +} diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 250dc5ef057..5cae393807c 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -85,11 +85,13 @@ protected: return Block(); } - while (!fire_signal.empty()) + while (true) { UInt32 timestamp_; { std::unique_lock lock_(fire_signal_mutex); + if (fire_signal.empty()) + break; timestamp_ = fire_signal.front(); fire_signal.pop_front(); } From ea8d8a67337e2a2a894718f2e393cb2abba4a4b2 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 23 Feb 2020 01:06:10 +0800 Subject: [PATCH 024/609] use pipes --- .../WindowView/BlocksListInputStream.h | 128 ------------------ .../Storages/WindowView/BlocksListSource.h | 39 ++++++ .../Storages/WindowView/StorageWindowView.cpp | 114 ++++++++-------- .../Storages/WindowView/StorageWindowView.h | 30 +++- .../WindowView/WatermarkBlockInputStream.h | 3 - .../WindowView/WindowViewBlockInputStream.h | 4 +- .../WindowView/WindowViewBlocksMetadata.h | 13 -- .../WindowView/WindowViewProxyStorage.h | 71 ++-------- ...052_window_view_event_time_watch.reference | 10 -- .../01052_window_view_event_time_watch.sql | 35 ----- ...1052_window_view_proctime_tumble.reference | 5 + .../01052_window_view_proctime_tumble.sql | 25 ++++ .../01053_window_view_event_time_to.reference | 14 -- .../01053_window_view_event_time_to.sql | 47 ------- ...indow_view_proctime_tumble_inner.reference | 5 + ...1053_window_view_proctime_tumble_inner.sql | 25 ++++ ...1054_window_view_proc_time_watch.reference | 10 -- .../01054_window_view_proc_time_watch.sql | 35 ----- .../01054_window_view_proctime_hop.reference | 7 + .../01054_window_view_proctime_hop.sql | 25 ++++ .../01055_window_view_proc_time_to.reference | 14 -- .../01055_window_view_proc_time_to.sql | 47 ------- ...5_window_view_proctime_hop_inner.reference | 7 + .../01055_window_view_proctime_hop_inner.sql | 25 ++++ ...056_window_view_eventtime_tumble.reference | 5 + .../01056_window_view_eventtime_tumble.sql | 25 ++++ ...ndow_view_eventtime_tumble_inner.reference | 5 + ...057_window_view_eventtime_tumble_inner.sql | 25 ++++ .../01058_window_view_eventtime_hop.reference | 7 + .../01058_window_view_eventtime_hop.sql | 25 ++++ ..._window_view_eventtime_hop_inner.reference | 7 + .../01059_window_view_eventtime_hop_inner.sql | 25 ++++ 32 files changed, 386 insertions(+), 476 deletions(-) delete mode 100644 dbms/src/Storages/WindowView/BlocksListInputStream.h create mode 100644 dbms/src/Storages/WindowView/BlocksListSource.h delete mode 100644 dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h delete mode 100644 dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference delete mode 100644 dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql create mode 100644 dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql delete mode 100644 dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql create mode 100644 dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference create mode 100644 dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql create mode 100644 dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference create mode 100644 dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql create mode 100644 dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference create mode 100644 dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql create mode 100644 dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference create mode 100644 dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql diff --git a/dbms/src/Storages/WindowView/BlocksListInputStream.h b/dbms/src/Storages/WindowView/BlocksListInputStream.h deleted file mode 100644 index 2280f06ac3a..00000000000 --- a/dbms/src/Storages/WindowView/BlocksListInputStream.h +++ /dev/null @@ -1,128 +0,0 @@ -#pragma once - -#include -#include -#include - - -namespace DB -{ -/** A stream of blocks from a shared vector of blocks with metadata - */ -using BlocksListPtr = std::shared_ptr; -using BlocksListPtrs = std::shared_ptr>; - -class BlocksListInputStream : public IBlockInputStream -{ -public: - /// Acquires shared ownership of the blocks vector - BlocksListInputStream(BlocksListPtrs blocks_ptr_, Block header_, UInt32 window_upper_bound_) - : blocks(blocks_ptr_), window_upper_bound(window_upper_bound_), header(std::move(header_)) - { - it_blocks = blocks->begin(); - end_blocks = blocks->end(); - if (it_blocks != end_blocks) - { - it = (*it_blocks)->begin(); - end = (*it_blocks)->end(); - } - } - - String getName() const override { return "BlocksListInputStream"; } - - Block getHeader() const override { return header; } - -protected: - Block readImpl() override - { - while (it_blocks != end_blocks) - { - while (it != end) - { - Block &block = *it; - size_t columns = block.columns(); - - auto & column_status = block.getByName("____w_end").column; - IColumn::Filter filter(column_status->size(), 0); - auto & data = static_cast(*column_status).getData(); - { - // std::unique_lock lock(mutex); - for (size_t i = 0; i < column_status->size(); ++i) - { - if (data[i] == window_upper_bound) - filter[i] = 1; - } - } - - //filter block - size_t first_non_constant_column = 0; - for (size_t i = 0; i < columns; ++i) - { - if (!isColumnConst(*block.safeGetByPosition(i).column)) - { - first_non_constant_column = i; - break; - } - } - - Block res = block.cloneEmpty(); - size_t filtered_rows = 0; - { - ColumnWithTypeAndName & current_column = block.safeGetByPosition(first_non_constant_column); - ColumnWithTypeAndName & filtered_column = res.safeGetByPosition(first_non_constant_column); - filtered_column.column = current_column.column->filter(filter, -1); - filtered_rows = filtered_column.column->size(); - } - - /// If the current block is completely filtered out, let's move on to the next one. - if (filtered_rows == 0) - { - ++it; - continue; - } - - /// If all the rows pass through the filter. - if (filtered_rows == filter.size()) - { - ++it; - return block; - } - - /// Filter the rest of the columns. - for (size_t i = 0; i < columns; ++i) - { - ColumnWithTypeAndName & current_column = block.safeGetByPosition(i); - ColumnWithTypeAndName & filtered_column = res.safeGetByPosition(i); - - if (i == first_non_constant_column) - continue; - - if (isColumnConst(*current_column.column)) - filtered_column.column = current_column.column->cut(0, filtered_rows); - else - filtered_column.column = current_column.column->filter(filter, -1); - } - ++it; - return res; - } - ++it_blocks; - if (it_blocks != end_blocks) - { - it = (*it_blocks)->begin(); - end = (*it_blocks)->end(); - } - } - return Block(); - } - -private: - BlocksListPtrs blocks; - std::list::iterator it_blocks; - std::list::iterator end_blocks; - BlocksList::iterator it; - BlocksList::iterator end; - // std::mutex & mutex; - UInt32 window_upper_bound; - Block header; -}; -} diff --git a/dbms/src/Storages/WindowView/BlocksListSource.h b/dbms/src/Storages/WindowView/BlocksListSource.h new file mode 100644 index 00000000000..abcb280d493 --- /dev/null +++ b/dbms/src/Storages/WindowView/BlocksListSource.h @@ -0,0 +1,39 @@ +#pragma once + +#include + + +namespace DB +{ +using BlocksListPtr = std::shared_ptr; +using BlocksListPtrs = std::shared_ptr>; + +/** A stream of blocks from a shared list of blocks + */ +class BlocksListSource : public SourceWithProgress +{ +public: + BlocksListSource(const BlocksListPtr & blocks_ptr_, Block header) + : SourceWithProgress(std::move(header)) + , blocks(blocks_ptr_), it(blocks_ptr_->begin()), end(blocks_ptr_->end()) {} + + String getName() const override { return "BlocksList"; } + +protected: + Chunk generate() override + { + if (it == end) + return {}; + + Block res = *it; + ++it; + return Chunk(res.getColumns(), res.rows()); + } + +private: + BlocksListPtr blocks; + BlocksList::iterator it; + const BlocksList::iterator end; +}; + +} diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index cdc06b93757..08bbb480b79 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -1,30 +1,24 @@ -#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 @@ -35,11 +29,16 @@ #include #include #include +#include +#include +#include +#include +#include #include #include #include -#include +#include #include #include #include @@ -319,6 +318,15 @@ bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, return getInnerStorage()->optimize(query, partition, final, deduplicate, context); } +Pipes StorageWindowView::blocksToPipes(BlocksListPtrs & blocks, Block & sample_block) +{ + Pipes pipes; + for (auto & blocks_ : *blocks) + pipes.emplace_back(std::make_shared(blocks_, sample_block)); + + return pipes; +} + inline void StorageWindowView::cleanCache() { //delete fired blocks @@ -353,14 +361,13 @@ inline void StorageWindowView::cleanCache() } { - std::lock_guard lock(fire_signal_mutex); + std::unique_lock lock(fire_signal_mutex); watch_streams.remove_if([](std::weak_ptr & ptr) { return ptr.expired(); }); } } inline void StorageWindowView::flushToTable(UInt32 timestamp_) { - //write into dependent table StoragePtr target_table = getTargetStorage(); auto _blockInputStreamPtr = getNewBlocksInputStreamPtr(timestamp_); auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); @@ -591,7 +598,7 @@ BlockInputStreams StorageWindowView::watch( limit); { - std::lock_guard lock(fire_signal_mutex); + std::unique_lock lock(fire_signal_mutex); watch_streams.push_back(reader); } @@ -600,20 +607,6 @@ BlockInputStreams StorageWindowView::watch( return {reader}; } -Block StorageWindowView::getHeader() const -{ - if (!sample_block) - { - auto storage = global_context.getTable(select_table_id); - sample_block = InterpreterSelectQuery(getInnerQuery(), global_context, storage, SelectQueryOptions(QueryProcessingStage::Complete)) - .getSampleBlock(); - for (size_t i = 0; i < sample_block.columns(); ++i) - sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); - } - - return sample_block; -} - StorageWindowView::StorageWindowView( const StorageID & table_id_, Context & local_context, @@ -629,7 +622,6 @@ StorageWindowView::StorageWindowView( if (!query.select) throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); - /// Default value, if only table name exist in the query if (query.select->list_of_selects->children.size() != 1) throw Exception("UNION is not supported for Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); @@ -778,30 +770,32 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con { UInt32 timestamp_now; UInt32 watermark; - auto block_stream = std::make_shared(block); - BlockInputStreamPtr source_stream; + + Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); std::shared_lock fire_signal_lock; if (window_view.is_proctime_tumble) { fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); timestamp_now = std::time(nullptr); - source_stream = std::make_shared>(block_stream, std::make_shared(), timestamp_now, "____timestamp"); watermark = window_view.getWindowLowerBound(timestamp_now); + pipe.addSimpleTransform(std::make_shared>( + pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); } else { - source_stream = block_stream; timestamp_now = std::time(nullptr); watermark = window_view.getWatermark(timestamp_now); } - InterpreterSelectQuery select_block( - window_view.getFinalQuery(), context, source_stream, QueryProcessingStage::WithMergeableState); - source_stream = std::make_shared>(select_block.execute().in, std::make_shared(), watermark, "____watermark"); - source_stream = std::make_shared(source_stream); + InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); + + BlockInputStreamPtr source_stream = std::make_shared>(select_block.execute().in, std::make_shared(), watermark, "____watermark"); source_stream = std::make_shared(source_stream, window_view.writeExpressions, "____filter", true); + source_stream = std::make_shared( + source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + if (window_view.watermark_num_units != 0) source_stream = std::make_shared(source_stream, window_view, timestamp_now); @@ -849,42 +843,54 @@ StorageWindowView::~StorageWindowView() BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 timestamp_) { - BlockInputStreamPtr stream; + Pipes pipes; if (!inner_table_id.empty()) { auto & storage = getInnerStorage(); InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); - - ColumnsWithTypeAndName columns_; - columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); - - ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(timestamp_)), - std::make_shared(), - "____timestamp_now"})); - const auto & function_equals = FunctionFactory::instance().get("equals", global_context); - ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____timestamp_now"}, "____filter")); - actions_->add(ExpressionAction::removeColumn("____w_end")); - actions_->add(ExpressionAction::removeColumn("____timestamp_now")); - stream = std::make_shared(fetch.execute().in, actions_, "____filter", true); + QueryPipeline pipeline; + BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); + for (auto & stream : streams) + pipes.emplace_back(std::make_shared(std::move(stream))); } else { + std::unique_lock lock(mutex); if (mergeable_blocks->empty()) return std::make_shared(getHeader()); - auto sample_block_ = mergeable_blocks->front()->front().cloneEmpty(); - stream = std::make_shared(mergeable_blocks, sample_block_, timestamp_); + pipes = blocksToPipes(mergeable_blocks, getMergeableHeader()); } - BlockInputStreams from; - from.push_back(std::move(stream)); + ColumnsWithTypeAndName columns_; + columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); + + ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); + actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(timestamp_)), + std::make_shared(), + "____timestamp_now"})); + const auto & function_equals = FunctionFactory::instance().get("equals", global_context); + ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); + actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____timestamp_now"}, "____filter")); + actions_->add(ExpressionAction::removeColumn("____w_end")); + actions_->add(ExpressionAction::removeColumn("____timestamp_now")); + + for (auto & pipe : pipes) + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), actions_, + "____filter", true)); + auto proxy_storage = std::make_shared( - StorageID("", "WindowViewProxyStorage"), getParentStorage(), std::move(from), QueryProcessingStage::WithMergeableState); + StorageID("", "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(getFinalQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, global_context.getSettingsRef().min_insert_block_size_rows, + global_context.getSettingsRef().min_insert_block_size_bytes); return data; } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 912d34bd42f..45c220256f4 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -49,6 +50,7 @@ public: unsigned num_streams) override; BlocksListPtrs getMergeableBlocksList() { return mergeable_blocks; } + std::shared_ptr getActivePtr() { return active_ptr; } BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 timestamp_); @@ -64,6 +66,7 @@ private: bool is_proctime_tumble{false}; std::atomic shutdown_called{false}; mutable Block sample_block; + mutable Block mergeable_sample_block; UInt64 clean_interval; const DateLUTImpl & time_zone; std::deque fire_signal; @@ -106,7 +109,6 @@ private: UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); UInt32 getWatermark(UInt32 time_sec); - Block getHeader() const; void flushToTable(UInt32 timestamp_); void cleanCache(); void threadFuncToTable(); @@ -114,10 +116,12 @@ private: void threadFuncFire(); void addFireSignal(UInt32 timestamp_); + static Pipes blocksToPipes(BlocksListPtrs & blocks, Block & sample_block); + ASTPtr getInnerQuery() const { return inner_query->clone(); } ASTPtr getFinalQuery() const { return final_query->clone(); } - StoragePtr& getParentStorage() + StoragePtr getParentStorage() { if (parent_storage == nullptr) parent_storage = global_context.getTable(select_table_id); @@ -138,6 +142,28 @@ private: return target_storage; } + Block & getHeader() + { + if (!sample_block) + { + sample_block = InterpreterSelectQuery( + getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) + .getSampleBlock(); + for (size_t i = 0; i < sample_block.columns(); ++i) + sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); + } + return sample_block; + } + + Block & getMergeableHeader() + { + if (!mergeable_sample_block) + { + mergeable_sample_block = mergeable_blocks->front()->front().cloneEmpty(); + } + return mergeable_sample_block; + } + StorageWindowView( const StorageID & table_id_, Context & local_context, diff --git a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h index 444629ea209..263f892ae64 100644 --- a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -8,8 +8,6 @@ namespace DB { -/** Adds a materialized const column to the block with a specified value. - */ class WatermarkBlockInputStream : public IBlockInputStream { public: @@ -48,7 +46,6 @@ protected: void readSuffix() override { - // while (!signal.empty()) for (auto signal : signals) storage.addFireSignal(signal); signals.clear(); diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 5cae393807c..029e3079d44 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -8,7 +8,7 @@ namespace DB { /** Implements WINDOW VIEW table WATCH input stream. * Keeps stream alive by outputing blocks with no rows - * based on period specified by the heartbeat interval. + * based on window interval. */ class WindowViewBlockInputStream : public IBlockInputStream { @@ -23,7 +23,7 @@ public: , has_limit(has_limit_) , limit(limit_) {} - String getName() const override { return "WindowViewBlockInputStream"; } + String getName() const override { return "WindowViewBlock"; } void cancel(bool kill) override { diff --git a/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h b/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h deleted file mode 100644 index 40dab479518..00000000000 --- a/dbms/src/Storages/WindowView/WindowViewBlocksMetadata.h +++ /dev/null @@ -1,13 +0,0 @@ -#pragma once - -#include - -namespace DB -{ -enum WINDOW_VIEW_FIRE_STATUS -{ - WAITING, - READY, - RETIRED -}; -} diff --git a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h index e449b5a9f0c..6dcf0023bda 100644 --- a/dbms/src/Storages/WindowView/WindowViewProxyStorage.h +++ b/dbms/src/Storages/WindowView/WindowViewProxyStorage.h @@ -1,10 +1,7 @@ #pragma once -#include -#include -#include #include -#include +#include #include namespace DB @@ -13,39 +10,22 @@ namespace DB class WindowViewProxyStorage : public IStorage { public: - WindowViewProxyStorage(const StorageID & table_id_, StoragePtr parent_storage_, QueryProcessingStage::Enum to_stage_) + WindowViewProxyStorage(const StorageID & table_id_, ColumnsDescription columns_, Pipes pipes_, QueryProcessingStage::Enum to_stage_) : IStorage(table_id_) - , parent_storage(std::move(parent_storage_)) - , streams({std::make_shared(Block())}) + , pipes(std::move(pipes_)) , to_stage(to_stage_) { - column_des = parent_storage->getColumns(); - column_des.add({"____timestamp", std::make_shared(), false}); - } - - WindowViewProxyStorage(const StorageID & table_id_, StoragePtr parent_storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) - : IStorage(table_id_) - , parent_storage(std::move(parent_storage_)) - , streams(std::move(streams_)) - , to_stage(to_stage_) - { - column_des = parent_storage->getColumns(); - column_des.add({"____timestamp", std::make_shared(), false}); + columns_.add({"____timestamp", std::make_shared(), false}); + setColumns(std::move(columns_)); } public: - std::string getName() const override { return "WindowViewProxyStorage(" + parent_storage->getName() + ")"; } + std::string getName() const override { return "WindowViewProxy"; } - bool isRemote() const override { return parent_storage->isRemote(); } - bool supportsSampling() const override { return parent_storage->supportsSampling(); } - bool supportsFinal() const override { return parent_storage->supportsFinal(); } - bool supportsPrewhere() const override { return parent_storage->supportsPrewhere(); } - bool supportsReplication() const override { return parent_storage->supportsReplication(); } - bool supportsDeduplication() const override { return parent_storage->supportsDeduplication(); } QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; } - BlockInputStreams read( + Pipes read( const Names & /*column_names*/, const SelectQueryInfo & /*query_info*/, const Context & /*context*/, @@ -53,44 +33,11 @@ public: size_t /*max_block_size*/, unsigned /*num_streams*/) override { - return streams; - } - - bool supportsIndexForIn() const override { return parent_storage->supportsIndexForIn(); } - bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override { return parent_storage->mayBenefitFromIndexForIn(left_in_operand, query_context); } - ASTPtr getPartitionKeyAST() const override { return parent_storage->getPartitionKeyAST(); } - ASTPtr getSortingKeyAST() const override { return parent_storage->getSortingKeyAST(); } - ASTPtr getPrimaryKeyAST() const override { return parent_storage->getPrimaryKeyAST(); } - ASTPtr getSamplingKeyAST() const override { return parent_storage->getSamplingKeyAST(); } - Names getColumnsRequiredForPartitionKey() const override { return parent_storage->getColumnsRequiredForPartitionKey(); } - Names getColumnsRequiredForSortingKey() const override { return parent_storage->getColumnsRequiredForSortingKey(); } - Names getColumnsRequiredForPrimaryKey() const override { return parent_storage->getColumnsRequiredForPrimaryKey(); } - Names getColumnsRequiredForSampling() const override { return parent_storage->getColumnsRequiredForSampling(); } - Names getColumnsRequiredForFinal() const override { return parent_storage->getColumnsRequiredForFinal(); } - - const ColumnsDescription & getColumns() const override { return column_des; } - - void setColumns(ColumnsDescription columns_) override { return parent_storage->setColumns(columns_); } - - NameAndTypePair getColumn(const String & column_name) const override - { - if (column_name == "____timestamp") - return {"____timestamp", std::shared_ptr()}; - return parent_storage->getColumn(column_name); - } - - bool hasColumn(const String & column_name) const override - { - if (column_name == "____timestamp") - return true; - return parent_storage->hasColumn(column_name); + return std::move(pipes); } private: - StoragePtr parent_storage; - BlockInputStreams streams; - String window_column_name; - ColumnsDescription column_des; + Pipes pipes; QueryProcessingStage::Enum to_stage; }; } diff --git a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference deleted file mode 100644 index 048630d233f..00000000000 --- a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.reference +++ /dev/null @@ -1,10 +0,0 @@ ---TUMBLE-- -1 ---HOP-- -1 -1 ---INNER_TUMBLE-- -1 ---INNER_HOP-- -1 -1 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql b/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql deleted file mode 100644 index 58482549657..00000000000 --- a/dbms/tests/queries/0_stateless/01052_window_view_event_time_watch.sql +++ /dev/null @@ -1,35 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.wv; - -SELECT '--TUMBLE--'; -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); -WATCH test.wv LIMIT 1; -DROP TABLE test.wv; - -SELECT '--HOP--'; -CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); -WATCH test.wv LIMIT 2; -DROP TABLE test.wv; - -SELECT '--INNER_TUMBLE--'; -CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); -WATCH test.wv LIMIT 1; -DROP TABLE test.wv; - -SELECT '--INNER_HOP--'; -CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now() + INTERVAL '1' SECOND); -WATCH test.wv LIMIT 2; -DROP TABLE test.wv; - -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference new file mode 100644 index 00000000000..f6527043448 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference @@ -0,0 +1,5 @@ +--WATCH-- +1 +0 +--TO-- +1 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql new file mode 100644 index 00000000000..bc26760d3d7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 1; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference b/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference deleted file mode 100644 index 670c8eb78c8..00000000000 --- a/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.reference +++ /dev/null @@ -1,14 +0,0 @@ ---TUMBLE-- -0 -1 ---HOP-- -0 -1 -1 ---INNER_TUMBLE-- -0 -1 ---INNER_HOP-- -0 -1 -1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql deleted file mode 100644 index fa93cffead1..00000000000 --- a/dbms/tests/queries/0_stateless/01053_window_view_event_time_to.sql +++ /dev/null @@ -1,47 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); - -SELECT '--TUMBLE--'; -DROP TABLE IF EXISTS test.wv; -TRUNCATE TABLE test.dst; -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(2); -SELECT count FROM test.dst; - -SELECT '--HOP--'; -DROP TABLE IF EXISTS test.wv; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(2); -SELECT count FROM test.dst; - -SELECT '--INNER_TUMBLE--'; -DROP TABLE IF EXISTS test.wv; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(2); -SELECT count FROM test.dst; - -SELECT '--INNER_HOP--'; -DROP TABLE IF EXISTS test.wv; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); -SELECT sleep(2); -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference new file mode 100644 index 00000000000..f6527043448 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference @@ -0,0 +1,5 @@ +--WATCH-- +1 +0 +--TO-- +1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql new file mode 100644 index 00000000000..663669466c4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 1; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference b/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference deleted file mode 100644 index 048630d233f..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.reference +++ /dev/null @@ -1,10 +0,0 @@ ---TUMBLE-- -1 ---HOP-- -1 -1 ---INNER_TUMBLE-- -1 ---INNER_HOP-- -1 -1 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql deleted file mode 100644 index 926ae776ce5..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_proc_time_watch.sql +++ /dev/null @@ -1,35 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); - -SELECT '--TUMBLE--'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -WATCH test.wv LIMIT 1; - -SELECT '--HOP--'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a) FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -WATCH test.wv LIMIT 2; - -SELECT '--INNER_TUMBLE--'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -WATCH test.wv LIMIT 1; - -SELECT '--INNER_HOP--'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -WATCH test.wv LIMIT 2; - -DROP TABLE test.wv; -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference new file mode 100644 index 00000000000..12bcf8797bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference @@ -0,0 +1,7 @@ +--WATCH-- +1 +1 +0 +--TO-- +1 +1 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql new file mode 100644 index 00000000000..a231e9c2bed --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 2; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference deleted file mode 100644 index 670c8eb78c8..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.reference +++ /dev/null @@ -1,14 +0,0 @@ ---TUMBLE-- -0 -1 ---HOP-- -0 -1 -1 ---INNER_TUMBLE-- -0 -1 ---INNER_HOP-- -0 -1 -1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql deleted file mode 100644 index b660fcd4a5c..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_proc_time_to.sql +++ /dev/null @@ -1,47 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); - -SELECT '--TUMBLE--'; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -SELECT sleep(1); -SELECT count FROM test.dst; -DROP TABLE test.wv; - -SELECT '--HOP--'; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -SELECT sleep(2); -SELECT count FROM test.dst; -DROP TABLE test.wv; - -SELECT '--INNER_TUMBLE--'; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -SELECT sleep(1); -SELECT count FROM test.dst; -DROP TABLE test.wv; - -SELECT '--INNER_HOP--'; -TRUNCATE TABLE test.dst; -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); -SELECT sleep(2); -SELECT count FROM test.dst; -DROP TABLE test.wv; - -DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference new file mode 100644 index 00000000000..12bcf8797bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference @@ -0,0 +1,7 @@ +--WATCH-- +1 +1 +0 +--TO-- +1 +1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql new file mode 100644 index 00000000000..b76ba3147d7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 2; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference new file mode 100644 index 00000000000..f6527043448 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference @@ -0,0 +1,5 @@ +--WATCH-- +1 +0 +--TO-- +1 diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql new file mode 100644 index 00000000000..9cef55aeacb --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 1; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference new file mode 100644 index 00000000000..f6527043448 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference @@ -0,0 +1,5 @@ +--WATCH-- +1 +0 +--TO-- +1 diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql new file mode 100644 index 00000000000..89ff5892c55 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 1; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference new file mode 100644 index 00000000000..12bcf8797bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference @@ -0,0 +1,7 @@ +--WATCH-- +1 +1 +0 +--TO-- +1 +1 diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql new file mode 100644 index 00000000000..b36f1909c77 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 2; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference new file mode 100644 index 00000000000..12bcf8797bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference @@ -0,0 +1,7 @@ +--WATCH-- +1 +1 +0 +--TO-- +1 +1 diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql new file mode 100644 index 00000000000..96599c68552 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.`.inner.wv`; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); + +SELECT '--WATCH--'; +WATCH test.wv LIMIT 2; + +SELECT sleep(1); + +SELECT '--TO--'; +SELECT count FROM test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; \ No newline at end of file From dcf6ede525bd46c76cbda25a5305c90226c040f5 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 23 Feb 2020 23:32:34 +0800 Subject: [PATCH 025/609] eliminate dup calculations caused by WATCH/TO --- .../Storages/WindowView/StorageWindowView.cpp | 142 +++++++++++------- .../Storages/WindowView/StorageWindowView.h | 17 +-- .../WindowView/WatermarkBlockInputStream.h | 2 +- .../WindowView/WindowViewBlockInputStream.h | 64 +++----- .../01052_window_view_proctime_tumble.py | 49 ++++++ ...1052_window_view_proctime_tumble.reference | 5 - .../01052_window_view_proctime_tumble.sql | 25 --- ...01053_window_view_proctime_tumble_inner.py | 49 ++++++ ...indow_view_proctime_tumble_inner.reference | 5 - ...1053_window_view_proctime_tumble_inner.sql | 25 --- .../01054_window_view_proctime_hop.py | 50 ++++++ .../01054_window_view_proctime_hop.reference | 7 - .../01054_window_view_proctime_hop.sql | 25 --- .../01055_window_view_proctime_hop_inner.py | 50 ++++++ ...5_window_view_proctime_hop_inner.reference | 7 - .../01055_window_view_proctime_hop_inner.sql | 25 --- .../01056_window_view_eventtime_tumble.py | 49 ++++++ ...056_window_view_eventtime_tumble.reference | 5 - .../01056_window_view_eventtime_tumble.sql | 25 --- ...1057_window_view_eventtime_tumble_inner.py | 49 ++++++ ...ndow_view_eventtime_tumble_inner.reference | 5 - ...057_window_view_eventtime_tumble_inner.sql | 25 --- .../01058_window_view_eventtime_hop.py | 59 ++++++++ .../01058_window_view_eventtime_hop.reference | 7 - .../01058_window_view_eventtime_hop.sql | 25 --- .../01059_window_view_eventtime_hop_inner.py | 50 ++++++ ..._window_view_eventtime_hop_inner.reference | 7 - .../01059_window_view_eventtime_hop_inner.sql | 25 --- 28 files changed, 517 insertions(+), 361 deletions(-) create mode 100755 dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py delete mode 100644 dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql create mode 100755 dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py delete mode 100644 dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql create mode 100755 dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py delete mode 100644 dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql create mode 100755 dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py delete mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql create mode 100755 dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py delete mode 100644 dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql create mode 100755 dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py delete mode 100644 dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql create mode 100755 dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py delete mode 100644 dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql create mode 100755 dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py delete mode 100644 dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 08bbb480b79..a3a66cfdd3f 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -296,7 +296,7 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &) std::lock_guard lock(mutex); is_dropped = true; - condition.notify_all(); + fire_condition.notify_all(); } void StorageWindowView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) @@ -366,13 +366,40 @@ inline void StorageWindowView::cleanCache() } } -inline void StorageWindowView::flushToTable(UInt32 timestamp_) +inline void StorageWindowView::fire(UInt32 timestamp_) { - StoragePtr target_table = getTargetStorage(); - auto _blockInputStreamPtr = getNewBlocksInputStreamPtr(timestamp_); - auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); - auto stream = target_table->write(getInnerQuery(), global_context); - copyData(*_blockInputStreamPtr, *stream); + if (target_table_id.empty() || watch_streams.empty()) + return; + + auto in_stream = getNewBlocksInputStreamPtr(timestamp_); + if (target_table_id.empty()) + { + while (auto block_ = in_stream->read()) + { + for (auto & watch_stream : watch_streams) + { + if (auto watch_stream_ = watch_stream.lock()) + watch_stream_->addBlock(block_); + } + } + } + else + { + StoragePtr target_table = getTargetStorage(); + auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); + auto out_stream = target_table->write(getInnerQuery(), global_context); + + while (auto block_ = in_stream->read()) + { + for (auto & watch_stream : watch_streams) + { + if (auto watch_stream_ = watch_stream.lock()) + watch_stream_->addBlock(block_); + } + out_stream->write(std::move(block_)); + } + } + fire_condition.notify_all(); } std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name) @@ -474,6 +501,28 @@ inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window __builtin_unreachable(); } +static UInt32 addTime(UInt32 time_sec, IntervalKind::Kind window_kind, int window_num_units, const DateLUTImpl & time_zone) +{ + switch (window_kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: \ + { \ + return AddTime::execute(time_sec, window_num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); +} + inline UInt32 StorageWindowView::getWatermark(UInt32 time_sec) { switch (watermark_kind) @@ -496,17 +545,11 @@ inline UInt32 StorageWindowView::getWatermark(UInt32 time_sec) __builtin_unreachable(); } -inline void StorageWindowView::addFireSignal(UInt32 timestamp_) +inline void StorageWindowView::addLateFireSignal(UInt32 timestamp_) { std::unique_lock lock(fire_signal_mutex); - if (!target_table_id.empty()) - fire_signal.push_back(timestamp_); - for (auto & watch_stream : watch_streams) - { - if (auto watch_stream_ = watch_stream.lock()) - watch_stream_->addFireSignal(timestamp_); - } - condition.notify_all(); + late_fire_signal.push_back(timestamp_); + late_signal_condition.notify_all(); } void StorageWindowView::threadFuncCleanCache() @@ -528,48 +571,37 @@ void StorageWindowView::threadFuncCleanCache() cleanCacheTask->scheduleAfter(RESCHEDULE_MS); } -void StorageWindowView::threadFuncToTable() -{ - while (!shutdown_called && !target_table_id.empty()) - { - std::unique_lock lock(flush_table_mutex); - condition.wait_for(lock, std::chrono::seconds(5)); - try - { - while (true) - { - UInt32 timestamp_; - { - std::unique_lock lock_(fire_signal_mutex); - if (fire_signal.empty()) - break; - timestamp_ = fire_signal.front(); - fire_signal.pop_front(); - } - flushToTable(timestamp_); - } - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - break; - } - } - if (!shutdown_called) - toTableTask->scheduleAfter(RESCHEDULE_MS); -} - void StorageWindowView::threadFuncFire() { + std::unique_lock lock(fire_signal_mutex); while (!shutdown_called) { + UInt32 timestamp_now = std::time(nullptr); + while (next_fire_signal <= timestamp_now) + { + fire(next_fire_signal); + next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, time_zone); + } + + next_fire_signal = (getWindowUpperBound(timestamp_now)); UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - UInt64 w_end = static_cast(getWindowUpperBound(static_cast(timestamp_usec / 1000000))) * 1000000; - std::this_thread::sleep_for(std::chrono::microseconds(w_end - timestamp_usec)); - addFireSignal(static_cast(w_end / 1000000)); + late_signal_condition.wait_for(lock, std::chrono::microseconds(static_cast(next_fire_signal) * 1000000 - timestamp_usec)); + + // fire late events + while (true) + { + UInt32 timestamp_; + { + if (late_fire_signal.empty()) + break; + timestamp_ = late_fire_signal.front(); + late_fire_signal.pop_front(); + } + fire(timestamp_); + } } if (!shutdown_called) - toTableTask->scheduleAfter(RESCHEDULE_MS); + fireTask->scheduleAfter(RESCHEDULE_MS); } BlockInputStreams StorageWindowView::watch( @@ -593,7 +625,6 @@ BlockInputStreams StorageWindowView::watch( auto reader = std::make_shared( std::static_pointer_cast(shared_from_this()), - active_ptr, has_limit, limit); @@ -655,7 +686,7 @@ StorageWindowView::StorageWindowView( clean_interval = local_context.getSettingsRef().window_view_clean_interval.totalSeconds(); mergeable_blocks = std::make_shared>(); - active_ptr = std::make_shared(true); + next_fire_signal = getWindowUpperBound(std::time(nullptr)); if (query.watermark_function) { @@ -728,10 +759,8 @@ StorageWindowView::StorageWindowView( writeExpressions->add(ExpressionAction::removeColumn("____watermark")); } - toTableTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncToTable(); }); cleanCacheTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFire(); }); - toTableTask->deactivate(); cleanCacheTask->deactivate(); fireTask->deactivate(); } @@ -820,8 +849,6 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con void StorageWindowView::startup() { // Start the working thread - if (!target_table_id.empty()) - toTableTask->activateAndSchedule(); cleanCacheTask->activateAndSchedule(); fireTask->activateAndSchedule(); } @@ -831,7 +858,6 @@ void StorageWindowView::shutdown() bool expected = false; if (!shutdown_called.compare_exchange_strong(expected, true)) return; - toTableTask->deactivate(); cleanCacheTask->deactivate(); fireTask->deactivate(); } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 45c220256f4..48bd4fe4b09 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -51,8 +51,6 @@ public: BlocksListPtrs getMergeableBlocksList() { return mergeable_blocks; } - std::shared_ptr getActivePtr() { return active_ptr; } - BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 timestamp_); static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); @@ -69,9 +67,11 @@ private: mutable Block mergeable_sample_block; UInt64 clean_interval; const DateLUTImpl & time_zone; - std::deque fire_signal; + UInt32 next_fire_signal; + std::deque late_fire_signal; std::list> watch_streams; - std::condition_variable condition; + std::condition_variable_any late_signal_condition; + std::condition_variable fire_condition; BlocksListPtrs mergeable_blocks; /// Mutex for the blocks and ready condition @@ -79,9 +79,6 @@ private: std::mutex flush_table_mutex; std::shared_mutex fire_signal_mutex; - /// Active users - std::shared_ptr active_ptr; - IntervalKind::Kind window_kind; IntervalKind::Kind watermark_kind; Int64 window_num_units; @@ -95,7 +92,6 @@ private: StoragePtr inner_storage; StoragePtr target_storage; - BackgroundSchedulePool::TaskHolder toTableTask; BackgroundSchedulePool::TaskHolder cleanCacheTask; BackgroundSchedulePool::TaskHolder fireTask; @@ -109,12 +105,11 @@ private: UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); UInt32 getWatermark(UInt32 time_sec); - void flushToTable(UInt32 timestamp_); + void fire(UInt32 timestamp_); void cleanCache(); - void threadFuncToTable(); void threadFuncCleanCache(); void threadFuncFire(); - void addFireSignal(UInt32 timestamp_); + void addLateFireSignal(UInt32 timestamp_); static Pipes blocksToPipes(BlocksListPtrs & blocks, Block & sample_block); diff --git a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h index 263f892ae64..12cde5adcbe 100644 --- a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -47,7 +47,7 @@ protected: void readSuffix() override { for (auto signal : signals) - storage.addFireSignal(signal); + storage.addLateFireSignal(signal); signals.clear(); } diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index 029e3079d44..a67faebb68d 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -15,11 +15,9 @@ class WindowViewBlockInputStream : public IBlockInputStream public: WindowViewBlockInputStream( std::shared_ptr storage_, - std::shared_ptr active_ptr_, const bool has_limit_, const UInt64 limit_) : storage(std::move(storage_)) - , active(std::move(active_ptr_)) , has_limit(has_limit_) , limit(limit_) {} @@ -34,16 +32,15 @@ public: Block getHeader() const override { return storage->getHeader(); } - void addFireSignal(UInt32 timestamp) + void addBlock(Block block_) { - std::lock_guard lock(fire_signal_mutex); - fire_signal.push_back(timestamp); + std::lock_guard lock(blocks_mutex); + blocks.push_back(std::move(block_)); } protected: Block readImpl() override { - /// try reading return tryReadImpl(); } @@ -52,24 +49,14 @@ protected: Block res; if (has_limit && num_updates == static_cast(limit)) - { return Block(); - } - /// If blocks were never assigned get blocks - if (!in_stream) - in_stream = std::make_shared(getHeader()); if (isCancelled() || storage->is_dropped) - { return Block(); - } - res = in_stream->read(); - if (!res) + std::unique_lock lock_(blocks_mutex); + if (blocks.empty()) { - if (!(*active)) - return Block(); - if (!end_of_blocks) { end_of_blocks = true; @@ -77,47 +64,38 @@ protected: return getHeader(); } - std::unique_lock lock(mutex); - storage->condition.wait_for(lock, std::chrono::seconds(5)); + storage->fire_condition.wait_for(lock_, std::chrono::seconds(5)); if (isCancelled() || storage->is_dropped) { return Block(); } - while (true) + if (blocks.empty()) + return getHeader(); + else { - UInt32 timestamp_; - { - std::unique_lock lock_(fire_signal_mutex); - if (fire_signal.empty()) - break; - timestamp_ = fire_signal.front(); - fire_signal.pop_front(); - } - in_stream = storage->getNewBlocksInputStreamPtr(timestamp_); - res = in_stream->read(); - if (res) - { - end_of_blocks = false; - return res; - } + end_of_blocks = false; + res = blocks.front(); + blocks.pop_front(); + return res; } - return getHeader(); } - return res; + else + { + res = blocks.front(); + blocks.pop_front(); + return res; + } } private: std::shared_ptr storage; - std::shared_ptr active; + BlocksList blocks; const bool has_limit; const UInt64 limit; - std::mutex mutex; Int64 num_updates = -1; bool end_of_blocks = false; - BlockInputStreamPtr in_stream; - std::mutex fire_signal_mutex; - std::deque fire_signal; + std::mutex blocks_mutex; }; } diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py new file mode 100755 index 00000000000..8dc1cfb01a3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py @@ -0,0 +1,49 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 1') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference index f6527043448..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference @@ -1,5 +0,0 @@ ---WATCH-- -1 -0 ---TO-- -1 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql deleted file mode 100644 index bc26760d3d7..00000000000 --- a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 1; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py new file mode 100755 index 00000000000..c283407ae14 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py @@ -0,0 +1,49 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 1') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference index f6527043448..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference @@ -1,5 +0,0 @@ ---WATCH-- -1 -0 ---TO-- -1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql deleted file mode 100644 index 663669466c4..00000000000 --- a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 1; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py new file mode 100755 index 00000000000..502eaa3c4ec --- /dev/null +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py @@ -0,0 +1,50 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 2') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference index 12bcf8797bc..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference @@ -1,7 +0,0 @@ ---WATCH-- -1 -1 -0 ---TO-- -1 -1 diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql deleted file mode 100644 index a231e9c2bed..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 2; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py new file mode 100755 index 00000000000..feb972b3d77 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py @@ -0,0 +1,50 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 2') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference index 12bcf8797bc..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference @@ -1,7 +0,0 @@ ---WATCH-- -1 -1 -0 ---TO-- -1 -1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql deleted file mode 100644 index b76ba3147d7..00000000000 --- a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 2; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py new file mode 100755 index 00000000000..bcbb4ac98b7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py @@ -0,0 +1,49 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 1') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference index f6527043448..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference +++ b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference @@ -1,5 +0,0 @@ ---WATCH-- -1 -0 ---TO-- -1 diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql deleted file mode 100644 index 9cef55aeacb..00000000000 --- a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 1; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py new file mode 100755 index 00000000000..fc4a2bffeb2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py @@ -0,0 +1,49 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 1') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference index f6527043448..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference +++ b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference @@ -1,5 +0,0 @@ ---WATCH-- -1 -0 ---TO-- -1 diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql deleted file mode 100644 index 89ff5892c55..00000000000 --- a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 1; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py new file mode 100755 index 00000000000..f6205fb3a72 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py @@ -0,0 +1,59 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 2') + # client1.expect('1.*' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + client1.expect('1' + end_of_block) + # client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + # client1.expect('3.*' + end_of_block) + # send Ctrl-C + # client1.send('\x03', eol='') + # match = client1.expect('(%s)|([#\$] )' % prompt) + # # print(match) + # if match.groups()[1]: + # client1.send(client1.command) + # client1.expect(prompt) + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference index 12bcf8797bc..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference +++ b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference @@ -1,7 +0,0 @@ ---WATCH-- -1 -1 -0 ---TO-- -1 -1 diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql deleted file mode 100644 index b36f1909c77..00000000000 --- a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 2; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py new file mode 100755 index 00000000000..e631417bc82 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py @@ -0,0 +1,50 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +# log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.dst') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + + client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv LIMIT 2') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('1' + end_of_block) + client1.expect('1' + end_of_block) + + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) + client1.send('DROP TABLE test.dst') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference index 12bcf8797bc..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference +++ b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference @@ -1,7 +0,0 @@ ---WATCH-- -1 -1 -0 ---TO-- -1 -1 diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql deleted file mode 100644 index 96599c68552..00000000000 --- a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.`.inner.wv`; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); - -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now()); - -SELECT '--WATCH--'; -WATCH test.wv LIMIT 2; - -SELECT sleep(1); - -SELECT '--TO--'; -SELECT count FROM test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; \ No newline at end of file From 10788f5e13dd26e1fc9ac994ca40c528cb4bc422 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 25 Feb 2020 01:02:30 +0800 Subject: [PATCH 026/609] update tests --- .../01052_window_view_proctime_tumble.py | 18 +++++++++----- ...01053_window_view_proctime_tumble_inner.py | 16 +++++++++---- .../01054_window_view_proctime_hop.py | 19 ++++++++++----- .../01055_window_view_proctime_hop_inner.py | 17 +++++++++---- .../01056_window_view_eventtime_tumble.py | 18 +++++++++----- ...1057_window_view_eventtime_tumble_inner.py | 12 ++++++++-- .../01058_window_view_eventtime_hop.py | 24 +++++++++---------- .../01059_window_view_eventtime_hop_inner.py | 11 ++++++++- 8 files changed, 93 insertions(+), 42 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py index 8dc1cfb01a3..39c42f20db3 100755 --- a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -25,8 +25,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.wv') client1.expect(prompt) @@ -34,13 +32,21 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 1') - client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') + client2.send('INSERT INTO test.mt VALUES (1)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py index c283407ae14..9fd0ddd2036 100755 --- a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -34,13 +34,21 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 1') - client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') + client2.send('INSERT INTO test.mt VALUES (1)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py index 502eaa3c4ec..597c4932ed2 100755 --- a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -25,8 +25,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.wv') client1.expect(prompt) @@ -34,14 +32,23 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 2') - client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') + client2.send('INSERT INTO test.mt VALUES (1)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py index feb972b3d77..849bbdde237 100755 --- a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -34,14 +34,23 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 2') - client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') + client2.send('INSERT INTO test.mt VALUES (1)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py index bcbb4ac98b7..b113e0d1b23 100755 --- a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py +++ b/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -23,11 +23,9 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') @@ -37,10 +35,18 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 1') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') client2.send('INSERT INTO test.mt VALUES (1, now())') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py index fc4a2bffeb2..75bd5d0db39 100755 --- a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py +++ b/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -37,10 +37,18 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 1') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') client2.send('INSERT INTO test.mt VALUES (1, now())') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py index f6205fb3a72..2b1e07296ed 100755 --- a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py +++ b/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py @@ -10,7 +10,7 @@ from client import client, prompt, end_of_block log = None # uncomment the line below for debugging -#log=sys.stdout +# log=sys.stdout with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: client1.expect(prompt) @@ -25,8 +25,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.wv') client1.expect(prompt) @@ -37,20 +35,20 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 2') - # client1.expect('1.*' + end_of_block) + client1.send('WATCH test.wv') client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('Progress: 0.00 rows.*\)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') client1.expect('1' + end_of_block) - # client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') - # client1.expect('3.*' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C - # client1.send('\x03', eol='') - # match = client1.expect('(%s)|([#\$] )' % prompt) - # # print(match) - # if match.groups()[1]: - # client1.send(client1.command) - # client1.expect(prompt) + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py index e631417bc82..a2e6a84ec97 100755 --- a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py +++ b/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py @@ -37,11 +37,20 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") client1.expect(prompt) - client1.send('WATCH test.wv LIMIT 2') + client1.send('WATCH test.wv') + client1.expect('Progress: 0.00 rows.*\)') client2.send('INSERT INTO test.mt VALUES (1, now())') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') client1.expect('1' + end_of_block) + client1.expect('Progress: 1.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) client1.send('DROP TABLE test.wv') client1.expect(prompt) client1.send('DROP TABLE test.mt') From 17f7a8408a0f72be0c6647f6891823982207a0a7 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 28 Feb 2020 01:18:40 +0800 Subject: [PATCH 027/609] fixed watch when where is no target table --- .../Storages/WindowView/StorageWindowView.cpp | 4 ++-- ...01052_window_view_proctime_tumble_watch.py} | 2 +- ...indow_view_proctime_tumble_watch.reference} | 0 ...53_window_view_proctime_tumble_to.reference | 2 ++ .../01053_window_view_proctime_tumble_to.sql | 18 ++++++++++++++++++ ...window_view_proctime_tumble_inner_watch.py} | 2 +- ...view_proctime_tumble_inner_watch.reference} | 0 ...dow_view_proctime_tumble_inner_to.reference | 2 ++ ...55_window_view_proctime_tumble_inner_to.sql | 18 ++++++++++++++++++ ...=> 01056_window_view_proctime_hop_watch.py} | 4 +--- ...6_window_view_proctime_hop_watch.reference} | 0 ...01057_window_view_proctime_hop_to.reference | 2 ++ .../01057_window_view_proctime_hop_to.sql | 18 ++++++++++++++++++ ...58_window_view_proctime_hop_inner_watch.py} | 4 +--- ...ow_view_proctime_hop_inner_watch.reference} | 0 ...window_view_proctime_hop_inner_to.reference | 2 ++ ...01059_window_view_proctime_hop_inner_to.sql | 18 ++++++++++++++++++ ...1060_window_view_eventtime_tumble_watch.py} | 2 +- ...ndow_view_eventtime_tumble_watch.reference} | 0 ...1_window_view_eventtime_tumble_to.reference | 2 ++ .../01061_window_view_eventtime_tumble_to.sql | 18 ++++++++++++++++++ ...indow_view_eventtime_tumble_inner_watch.py} | 2 +- ...iew_eventtime_tumble_inner_watch.reference} | 0 ...ow_view_eventtime_tumble_inner_to.reference | 2 ++ ...3_window_view_eventtime_tumble_inner_to.sql | 18 ++++++++++++++++++ ...> 01064_window_view_eventtime_hop_watch.py} | 4 +--- ..._window_view_eventtime_hop_watch.reference} | 0 ...1065_window_view_eventtime_hop_to.reference | 2 ++ .../01065_window_view_eventtime_hop_to.sql | 18 ++++++++++++++++++ ...6_window_view_eventtime_hop_inner_watch.py} | 8 ++------ ...w_view_eventtime_hop_inner_watch.reference} | 0 ...indow_view_eventtime_hop_inner_to.reference | 2 ++ ...1067_window_view_eventtime_hop_inner_to.sql | 18 ++++++++++++++++++ 33 files changed, 171 insertions(+), 21 deletions(-) rename dbms/tests/queries/0_stateless/{01052_window_view_proctime_tumble.py => 01052_window_view_proctime_tumble_watch.py} (91%) rename dbms/tests/queries/0_stateless/{01052_window_view_proctime_tumble.reference => 01052_window_view_proctime_tumble_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.reference create mode 100755 dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.sql rename dbms/tests/queries/0_stateless/{01053_window_view_proctime_tumble_inner.py => 01054_window_view_proctime_tumble_inner_watch.py} (90%) rename dbms/tests/queries/0_stateless/{01053_window_view_proctime_tumble_inner.reference => 01054_window_view_proctime_tumble_inner_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.reference create mode 100755 dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.sql rename dbms/tests/queries/0_stateless/{01054_window_view_proctime_hop.py => 01056_window_view_proctime_hop_watch.py} (87%) rename dbms/tests/queries/0_stateless/{01054_window_view_proctime_hop.reference => 01056_window_view_proctime_hop_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.reference create mode 100755 dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.sql rename dbms/tests/queries/0_stateless/{01055_window_view_proctime_hop_inner.py => 01058_window_view_proctime_hop_inner_watch.py} (86%) rename dbms/tests/queries/0_stateless/{01055_window_view_proctime_hop_inner.reference => 01058_window_view_proctime_hop_inner_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.reference create mode 100755 dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01056_window_view_eventtime_tumble.py => 01060_window_view_eventtime_tumble_watch.py} (90%) rename dbms/tests/queries/0_stateless/{01056_window_view_eventtime_tumble.reference => 01060_window_view_eventtime_tumble_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.reference create mode 100755 dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql rename dbms/tests/queries/0_stateless/{01057_window_view_eventtime_tumble_inner.py => 01062_window_view_eventtime_tumble_inner_watch.py} (89%) rename dbms/tests/queries/0_stateless/{01057_window_view_eventtime_tumble_inner.reference => 01062_window_view_eventtime_tumble_inner_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.reference create mode 100755 dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql rename dbms/tests/queries/0_stateless/{01058_window_view_eventtime_hop.py => 01064_window_view_eventtime_hop_watch.py} (85%) rename dbms/tests/queries/0_stateless/{01058_window_view_eventtime_hop.reference => 01064_window_view_eventtime_hop_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.reference create mode 100755 dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql rename dbms/tests/queries/0_stateless/{01059_window_view_eventtime_hop_inner.py => 01066_window_view_eventtime_hop_inner_watch.py} (80%) rename dbms/tests/queries/0_stateless/{01059_window_view_eventtime_hop_inner.reference => 01066_window_view_eventtime_hop_inner_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.reference create mode 100755 dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index a3a66cfdd3f..42544975399 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -368,7 +368,7 @@ inline void StorageWindowView::cleanCache() inline void StorageWindowView::fire(UInt32 timestamp_) { - if (target_table_id.empty() || watch_streams.empty()) + if (target_table_id.empty() && watch_streams.empty()) return; auto in_stream = getNewBlocksInputStreamPtr(timestamp_); @@ -839,7 +839,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con { BlocksListPtr new_mergeable_blocks = std::make_shared(); while (Block block_ = source_stream->read()) - new_mergeable_blocks->push_back(std::move(block_)); + new_mergeable_blocks->push_back(std::move(block_)); std::unique_lock lock(window_view.mutex); window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py similarity index 91% rename from dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py rename to dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py index 39c42f20db3..60fb489ed97 100755 --- a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.py +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble.reference rename to dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.reference diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.reference b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.sql new file mode 100755 index 00000000000..5de210ad2b2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py b/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py similarity index 90% rename from dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py rename to dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py index 9fd0ddd2036..e18d73a5931 100755 --- a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.py +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py @@ -34,7 +34,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference b/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_inner.reference rename to dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.reference diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.sql new file mode 100755 index 00000000000..a2b65561590 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py b/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py similarity index 87% rename from dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py rename to dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py index 597c4932ed2..0a6a804e638 100755 --- a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.py +++ b/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') @@ -40,8 +40,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (1)') client1.expect('1' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference b/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01054_window_view_proctime_hop.reference rename to dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.reference b/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.sql b/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.sql new file mode 100755 index 00000000000..627516fcd46 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py b/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py similarity index 86% rename from dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py rename to dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py index 849bbdde237..c9e88b59075 100755 --- a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.py +++ b/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py @@ -34,7 +34,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') @@ -42,8 +42,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client2.send('INSERT INTO test.mt VALUES (1)') client1.expect('1' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference b/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01055_window_view_proctime_hop_inner.reference rename to dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.reference diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.sql new file mode 100755 index 00000000000..601b1aa3588 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv to test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py b/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py similarity index 90% rename from dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py rename to dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py index b113e0d1b23..14e73e9f0fa 100755 --- a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.py +++ b/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference b/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01056_window_view_eventtime_tumble.reference rename to dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.reference diff --git a/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.reference b/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql b/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql new file mode 100755 index 00000000000..8dd6b257e44 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py b/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py similarity index 89% rename from dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py rename to dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py index 75bd5d0db39..b6c16d06383 100755 --- a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.py +++ b/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py @@ -34,7 +34,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference b/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01057_window_view_eventtime_tumble_inner.reference rename to dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.reference diff --git a/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql new file mode 100755 index 00000000000..0033037667e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py b/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py similarity index 85% rename from dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py rename to dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py index 2b1e07296ed..916e38ad4e0 100755 --- a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.py +++ b/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') @@ -40,8 +40,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect('Progress: 0.00 rows.*\)') client1.expect('1' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') diff --git a/dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference b/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01058_window_view_eventtime_hop.reference rename to dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.reference b/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql new file mode 100755 index 00000000000..1f23c7271da --- /dev/null +++ b/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py b/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py similarity index 80% rename from dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py rename to dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py index a2e6a84ec97..2052ce06c02 100755 --- a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.py +++ b/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py @@ -25,8 +25,6 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) client1.send(' DROP TABLE IF EXISTS test.wv') client1.expect(prompt) @@ -34,14 +32,12 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '2' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') - client1.expect('Progress: 0.00 rows.*\)') client2.send('INSERT INTO test.mt VALUES (1, now())') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') + client1.expect('Progress: 0.00 rows.*\)') client1.expect('1' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') diff --git a/dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference b/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01059_window_view_eventtime_hop_inner.reference rename to dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.reference diff --git a/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql new file mode 100755 index 00000000000..5e4fde1c8bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now()); +SELECT sleep(2); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; From bd8eaf86883a51673e35723cc929efd821b8a3ce Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 28 Feb 2020 15:06:17 +0800 Subject: [PATCH 028/609] fix bug that late events will not trigger --- .../Storages/WindowView/StorageWindowView.cpp | 40 ++++++++++++------- .../Storages/WindowView/StorageWindowView.h | 3 +- ...01052_window_view_proctime_tumble_watch.py | 4 +- ...window_view_proctime_tumble_inner_watch.py | 6 +-- .../01056_window_view_proctime_hop_watch.py | 4 +- ...58_window_view_proctime_hop_inner_watch.py | 6 +-- ...1060_window_view_eventtime_tumble_watch.py | 2 +- .../01061_window_view_eventtime_tumble_to.sql | 2 +- ...indow_view_eventtime_tumble_inner_watch.py | 8 ++-- ..._window_view_eventtime_tumble_inner_to.sql | 2 +- .../01064_window_view_eventtime_hop_watch.py | 6 +-- .../01065_window_view_eventtime_hop_to.sql | 2 +- ...6_window_view_eventtime_hop_inner_watch.py | 6 +-- ...067_window_view_eventtime_hop_inner_to.sql | 2 +- ..._window_view_eventtime_watermark.reference | 2 + .../01068_window_view_eventtime_watermark.sql | 18 +++++++++ ...w_view_eventtime_inner_watermark.reference | 2 + ..._window_view_eventtime_inner_watermark.sql | 18 +++++++++ 18 files changed, 93 insertions(+), 40 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference create mode 100755 dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql create mode 100644 dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference create mode 100755 dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 42544975399..710d95828fc 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -49,13 +49,13 @@ namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; + extern const int CANNOT_PARSE_TEXT; + extern const int ILLEGAL_COLUMN; extern const int INCORRECT_QUERY; - extern const int TABLE_WAS_NOT_DROPPED; + extern const int LOGICAL_ERROR; extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW; extern const int SUPPORT_IS_DISABLED; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int CANNOT_PARSE_TEXT; + extern const int TABLE_WAS_NOT_DROPPED; } namespace @@ -128,7 +128,7 @@ namespace struct Data { - bool is_proctime_tumble = false; + bool is_proctime = false; String window_column_name; }; @@ -150,11 +150,16 @@ namespace { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") { - data.is_proctime_tumble = true; + data.is_proctime = true; node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); data.window_column_name = node.getColumnName(); } } + else if (node.name == "HOP") + { + if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") + data.is_proctime = true; + } } }; @@ -417,7 +422,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto columns_list = std::make_shared(); - if (is_proctime_tumble) + if (is_proctime && is_tumble) { auto column_window = std::make_shared(); column_window->name = window_column_name; @@ -668,8 +673,8 @@ StorageWindowView::StorageWindowView( final_query = inner_query->clone(); ParserProcTimeFinalMatcher::Data final_query_data; ParserProcTimeFinalMatcher::Visitor(final_query_data).visit(final_query); - is_proctime_tumble = final_query_data.is_proctime_tumble; - if (is_proctime_tumble) + is_proctime = final_query_data.is_proctime; + if (is_proctime && is_tumble) window_column_name = final_query_data.window_column_name; /// If the table is not specified - use the table `system.one` @@ -690,7 +695,7 @@ StorageWindowView::StorageWindowView( if (query.watermark_function) { - if (is_proctime_tumble) + if (is_proctime) throw Exception("WATERMARK is not support for Processing time processing.", ErrorCodes::INCORRECT_QUERY); // parser watermark function @@ -779,6 +784,7 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) if (!stageMergeableOneData.is_tumble && !stageMergeableOneData.is_hop) throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); window_column_name = stageMergeableOneData.window_column_name; + is_tumble = stageMergeableOneData.is_tumble; // parser window function ASTFunction & window_function = typeid_cast(*stageMergeableOneData.window_function); @@ -803,18 +809,22 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); std::shared_lock fire_signal_lock; - if (window_view.is_proctime_tumble) + if (window_view.is_proctime) { fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); timestamp_now = std::time(nullptr); watermark = window_view.getWindowLowerBound(timestamp_now); - pipe.addSimpleTransform(std::make_shared>( - pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); + if (window_view.is_tumble) + pipe.addSimpleTransform(std::make_shared>( + pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); } else { timestamp_now = std::time(nullptr); - watermark = window_view.getWatermark(timestamp_now); + if (window_view.watermark_num_units == 0) + watermark = window_view.getWindowLowerBound(timestamp_now); + else + watermark = window_view.getWatermark(timestamp_now); } InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); @@ -834,12 +844,14 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con auto lock_ = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); auto stream = inner_storage->write(window_view.getInnerQuery(), context); copyData(*source_stream, *stream); + source_stream->readSuffix(); } else { BlocksListPtr new_mergeable_blocks = std::make_shared(); while (Block block_ = source_stream->read()) new_mergeable_blocks->push_back(std::move(block_)); + source_stream->readSuffix(); std::unique_lock lock(window_view.mutex); window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 48bd4fe4b09..2dcf8a8da93 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -61,7 +61,8 @@ private: ASTPtr fetch_column_query; Context & global_context; - bool is_proctime_tumble{false}; + bool is_proctime{false}; + bool is_tumble; // false if is hop std::atomic shutdown_called{false}; mutable Block sample_block; mutable Block mergeable_sample_block; diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py index 60fb489ed97..29f7e947d41 100755 --- a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py +++ b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py @@ -23,9 +23,9 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py b/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py index e18d73a5931..df4fecd4d4f 100755 --- a/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py +++ b/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py @@ -23,11 +23,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.send('DROP TABLE IF EXISTS test.`.inner.wv`') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py b/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py index 0a6a804e638..3a630984fd7 100755 --- a/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py @@ -23,9 +23,9 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py b/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py index c9e88b59075..00867113e77 100755 --- a/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py +++ b/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py @@ -23,11 +23,11 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.send('DROP TABLE IF EXISTS test.`.inner.wv`') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') diff --git a/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py b/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py index 14e73e9f0fa..fd34f78b957 100755 --- a/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py +++ b/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql b/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql index 8dd6b257e44..333694c989c 100755 --- a/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS test.wv; CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now()); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py b/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py index b6c16d06383..7b3f36a35f1 100755 --- a/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py +++ b/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py @@ -23,18 +23,18 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.`.inner.wv`') + client1.send('DROP TABLE IF EXISTS test.`.inner.wv`') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql index 0033037667e..1b8afaa1541 100755 --- a/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS test.wv; CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now()); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py b/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py index 916e38ad4e0..4b057e9f565 100755 --- a/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py @@ -23,16 +23,16 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql index 1f23c7271da..722844741d4 100755 --- a/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS test.wv; CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now()); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py b/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py index 2052ce06c02..43b3086acbd 100755 --- a/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py +++ b/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py @@ -23,16 +23,16 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.dst') + client1.send('DROP TABLE IF EXISTS test.dst') client1.expect(prompt) - client1.send(' DROP TABLE IF EXISTS test.wv') + client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql index 5e4fde1c8bc..4e59b8602d6 100755 --- a/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS test.wv; CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK INTERVAL '1' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; INSERT INTO test.mt VALUES (1, now()); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference b/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql b/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql new file mode 100755 index 00000000000..29c3c884ee0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=INTERVAL '5' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now() - 2); +SELECT sleep(1); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference b/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference new file mode 100644 index 00000000000..0d66ea1aee9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference @@ -0,0 +1,2 @@ +0 +1 diff --git a/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql b/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql new file mode 100755 index 00000000000..02a9bbd919f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql @@ -0,0 +1,18 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree() order by tuple() WATERMARK=INTERVAL '5' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, now() - 2); +SELECT sleep(1); +SELECT count from test.dst; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; From 7b2ded1c9b589e3317f144dbb27168c688ab12a5 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 29 Feb 2020 02:21:36 +0800 Subject: [PATCH 029/609] remove duplicated call of readSuffix --- dbms/src/Storages/WindowView/StorageWindowView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 710d95828fc..20e3b3e892d 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -844,11 +844,11 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con auto lock_ = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); auto stream = inner_storage->write(window_view.getInnerQuery(), context); copyData(*source_stream, *stream); - source_stream->readSuffix(); } else { BlocksListPtr new_mergeable_blocks = std::make_shared(); + source_stream->readPrefix(); while (Block block_ = source_stream->read()) new_mergeable_blocks->push_back(std::move(block_)); source_stream->readSuffix(); From 2b6d15bad2fa1829c40029d04a0c0da87c118a16 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 2 Mar 2020 01:52:55 +0800 Subject: [PATCH 030/609] HOP_END calculate the first wid instead of last --- dbms/src/Functions/FunctionsWindow.h | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/dbms/src/Functions/FunctionsWindow.h b/dbms/src/Functions/FunctionsWindow.h index 740144fa67d..c5c3419136b 100644 --- a/dbms/src/Functions/FunctionsWindow.h +++ b/dbms/src/Functions/FunctionsWindow.h @@ -164,19 +164,11 @@ namespace auto res = ColumnUInt32::create(); ColumnUInt32::Container & res_data = res->getData(); res_data.reserve(column_offsets.size()); - if (index == 0) // lower bound of hop window + IColumn::Offset current_offset = 0; + for (size_t i = 0; i < column_offsets.size(); ++i) { - IColumn::Offset current_offset = 0; - for (size_t i = 0; i < column_offsets.size(); ++i) - { - res_data.push_back(bound_data[current_offset]); - current_offset = column_offsets[i]; - } - } - else // upper bound of hop window - { - for (size_t i = 0; i < column_offsets.size(); ++i) - res_data.push_back(bound_data[column_offsets[i] - 1]); + res_data.push_back(bound_data[current_offset]); + current_offset = column_offsets[i]; } return res; } From 33055fae752e512244a350a3ce21591237de2f38 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 2 Mar 2020 02:08:52 +0800 Subject: [PATCH 031/609] support multiple watermark strategies --- dbms/src/Parsers/ASTCreateQuery.cpp | 10 +- dbms/src/Parsers/ASTCreateQuery.h | 3 + dbms/src/Parsers/ParserCreateQuery.cpp | 14 +- .../Storages/WindowView/StorageWindowView.cpp | 408 ++++++++++-------- .../Storages/WindowView/StorageWindowView.h | 28 +- .../WindowView/WatermarkBlockInputStream.h | 40 +- .../01049_window_functions.reference | 8 +- ...01052_window_view_proctime_tumble_watch.py | 55 --- ..._window_view_proc_now_tumble_to.reference} | 0 ... 01053_window_view_proc_now_tumble_to.sql} | 0 ...w_view_proc_now_tumble_inner_to.reference} | 0 ..._window_view_proc_now_tumble_inner_to.sql} | 0 ...window_view_proctime_tumble_inner_watch.py | 57 --- ...055_window_view_proc_now_hop_to.reference} | 0 ... => 01055_window_view_proc_now_hop_to.sql} | 0 ...ndow_view_proc_now_hop_inner_to.reference} | 0 ...056_window_view_proc_now_hop_inner_to.sql} | 0 .../01056_window_view_proctime_hop_watch.py | 55 --- ...1057_window_view_proc_tumble_to.reference} | 0 ...l => 01057_window_view_proc_tumble_to.sql} | 0 ...indow_view_proc_tumble_inner_to.reference} | 0 ...1058_window_view_proc_tumble_inner_to.sql} | 0 ...58_window_view_proctime_hop_inner_watch.py | 57 --- ...> 01059_window_view_proc_hop_to.reference} | 0 ....sql => 01059_window_view_proc_hop_to.sql} | 0 ...ndow_view_eventtime_tumble_watch.reference | 0 ...0_window_view_proc_hop_inner_to.reference} | 0 ...> 01060_window_view_proc_hop_inner_to.sql} | 0 ...py => 01061_window_view_proc_hop_watch.py} | 6 - ...1061_window_view_proc_hop_watch.reference} | 0 ..._view_event_strict_asc_tumble_to.reference | 4 + ...window_view_event_strict_asc_tumble_to.sql | 25 ++ ...iew_eventtime_tumble_inner_watch.reference | 0 ...event_strict_asc_tumble_inner_to.reference | 4 + ..._view_event_strict_asc_tumble_inner_to.sql | 25 ++ ...dow_view_event_strict_asc_hop_to.reference | 7 + ...64_window_view_event_strict_asc_hop_to.sql | 25 ++ ..._window_view_eventtime_hop_watch.reference | 0 ...ew_event_strict_asc_hop_inner_to.reference | 7 + ...dow_view_event_strict_asc_hop_inner_to.sql | 25 ++ ...window_view_event_strict_asc_hop_watch.py} | 17 +- ...view_event_strict_asc_hop_watch.reference} | 0 ...w_view_eventtime_hop_inner_watch.reference | 0 ..._window_view_event_asc_tumble_to.reference | 4 + .../01067_window_view_event_asc_tumble_to.sql | 26 ++ ...w_view_event_asc_tumble_inner_to.reference | 4 + ..._window_view_event_asc_tumble_inner_to.sql | 26 ++ ..._window_view_eventtime_watermark.reference | 2 - .../01068_window_view_eventtime_watermark.sql | 18 - ...069_window_view_event_asc_hop_to.reference | 7 + .../01069_window_view_event_asc_hop_to.sql | 26 ++ ...w_view_eventtime_inner_watermark.reference | 2 - ..._window_view_eventtime_inner_watermark.sql | 18 - ...ndow_view_event_asc_hop_inner_to.reference | 7 + ...070_window_view_event_asc_hop_inner_to.sql | 26 ++ ... 01071_window_view_event_asc_hop_watch.py} | 21 +- ...window_view_event_asc_hop_watch.reference} | 0 ...dow_view_event_bounded_tumble_to.reference | 4 + ...72_window_view_event_bounded_tumble_to.sql | 26 ++ ...ew_event_bounded_tumble_inner_to.reference | 4 + ...dow_view_event_bounded_tumble_inner_to.sql | 26 ++ ...window_view_event_bounded_hop_to.reference | 7 + ...01074_window_view_event_bounded_hop_to.sql | 26 ++ ..._view_event_bounded_hop_inner_to.reference | 7 + ...window_view_event_bounded_hop_inner_to.sql | 26 ++ ...76_window_view_event_bounded_hop_watch.py} | 19 +- ...ow_view_event_bounded_hop_watch.reference} | 0 67 files changed, 717 insertions(+), 495 deletions(-) delete mode 100755 dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py rename dbms/tests/queries/0_stateless/{01053_window_view_proctime_tumble_to.reference => 01053_window_view_proc_now_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01053_window_view_proctime_tumble_to.sql => 01053_window_view_proc_now_tumble_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01055_window_view_proctime_tumble_inner_to.reference => 01054_window_view_proc_now_tumble_inner_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01055_window_view_proctime_tumble_inner_to.sql => 01054_window_view_proc_now_tumble_inner_to.sql} (100%) delete mode 100755 dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py rename dbms/tests/queries/0_stateless/{01057_window_view_proctime_hop_to.reference => 01055_window_view_proc_now_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01057_window_view_proctime_hop_to.sql => 01055_window_view_proc_now_hop_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01059_window_view_proctime_hop_inner_to.reference => 01056_window_view_proc_now_hop_inner_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01059_window_view_proctime_hop_inner_to.sql => 01056_window_view_proc_now_hop_inner_to.sql} (100%) delete mode 100755 dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py rename dbms/tests/queries/0_stateless/{01061_window_view_eventtime_tumble_to.reference => 01057_window_view_proc_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01061_window_view_eventtime_tumble_to.sql => 01057_window_view_proc_tumble_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01063_window_view_eventtime_tumble_inner_to.reference => 01058_window_view_proc_tumble_inner_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01063_window_view_eventtime_tumble_inner_to.sql => 01058_window_view_proc_tumble_inner_to.sql} (100%) delete mode 100755 dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py rename dbms/tests/queries/0_stateless/{01065_window_view_eventtime_hop_to.reference => 01059_window_view_proc_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01065_window_view_eventtime_hop_to.sql => 01059_window_view_proc_hop_to.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.reference rename dbms/tests/queries/0_stateless/{01067_window_view_eventtime_hop_inner_to.reference => 01060_window_view_proc_hop_inner_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01067_window_view_eventtime_hop_inner_to.sql => 01060_window_view_proc_hop_inner_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01064_window_view_eventtime_hop_watch.py => 01061_window_view_proc_hop_watch.py} (86%) rename dbms/tests/queries/0_stateless/{01052_window_view_proctime_tumble_watch.reference => 01061_window_view_proc_hop_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.reference create mode 100644 dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.reference create mode 100644 dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference create mode 100644 dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql create mode 100644 dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.reference create mode 100644 dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.reference create mode 100644 dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference create mode 100644 dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01060_window_view_eventtime_tumble_watch.py => 01066_window_view_event_strict_asc_hop_watch.py} (67%) rename dbms/tests/queries/0_stateless/{01054_window_view_proctime_tumble_inner_watch.reference => 01066_window_view_event_strict_asc_hop_watch.reference} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.reference create mode 100644 dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.reference create mode 100644 dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql create mode 100644 dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.reference create mode 100644 dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference delete mode 100755 dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql create mode 100644 dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.reference create mode 100644 dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference delete mode 100755 dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql create mode 100644 dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.reference create mode 100644 dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01062_window_view_eventtime_tumble_inner_watch.py => 01071_window_view_event_asc_hop_watch.py} (64%) rename dbms/tests/queries/0_stateless/{01056_window_view_proctime_hop_watch.reference => 01071_window_view_event_asc_hop_watch.reference} (100%) create mode 100644 dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference create mode 100644 dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql create mode 100644 dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference create mode 100644 dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql create mode 100644 dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference create mode 100644 dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql create mode 100644 dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference create mode 100644 dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01066_window_view_eventtime_hop_inner_watch.py => 01076_window_view_event_bounded_hop_watch.py} (63%) rename dbms/tests/queries/0_stateless/{01058_window_view_proctime_hop_inner_watch.reference => 01076_window_view_event_bounded_hop_watch.reference} (100%) diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 3165d09692b..81660cd8d74 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -296,7 +296,15 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat if (is_populate) settings.ostr << (settings.hilite ? hilite_keyword : "") << " POPULATE" << (settings.hilite ? hilite_none : ""); - if (watermark_function) + if (is_watermark_strictly_ascending) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK STRICTLY_ASCENDING" << (settings.hilite ? hilite_none : ""); + } + else if (is_watermark_ascending) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK ASCENDING" << (settings.hilite ? hilite_none : ""); + } + else if (is_watermark_bounded) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " WATERMARK " << (settings.hilite ? hilite_none : ""); watermark_function->formatImpl(settings, state, frame); diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 529ee976e87..7c0bdf825b7 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -62,6 +62,9 @@ public: bool is_populate{false}; bool is_dictionary{false}; /// CREATE DICTIONARY bool replace_view{false}; /// CREATE OR REPLACE VIEW + bool is_watermark_strictly_ascending{false}; /// STRICTLY ASCENDING WATERMARK STRATEGY + bool is_watermark_ascending{false}; /// ASCENDING WATERMARK STRATEGY + bool is_watermark_bounded{false}; /// BOUNDED OUT OF ORDERNESS WATERMARK STRATEGY ASTColumns * columns_list = nullptr; ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of dictionary ASTExpressionList * tables = nullptr; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 0928098a103..52820864e1b 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -647,6 +647,9 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String cluster_str; bool attach = false; + bool is_watermark_strictly_ascending = false; + bool is_watermark_ascending = false; + bool is_watermark_bounded = false; bool if_not_exists = false; bool is_temporary = false; @@ -720,7 +723,13 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & { s_eq.ignore(pos, expected); - if (!watermark_p.parse(pos, watermark, expected)) + if (ParserKeyword("STRICTLY_ASCENDING").ignore(pos,expected)) + is_watermark_strictly_ascending = true; + else if (ParserKeyword("ASCENDING").ignore(pos,expected)) + is_watermark_ascending = true; + else if (watermark_p.parse(pos, watermark, expected)) + is_watermark_bounded = true; + else return false; } @@ -749,6 +758,9 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->set(query->columns_list, columns_list); query->set(query->storage, storage); + query->is_watermark_strictly_ascending = is_watermark_strictly_ascending; + query->is_watermark_ascending = is_watermark_ascending; + query->is_watermark_bounded = is_watermark_bounded; query->watermark_function = watermark; tryGetIdentifierNameInto(as_database, query->as_database); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 20e3b3e892d..0cad71f3806 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -49,6 +49,7 @@ namespace DB { namespace ErrorCodes { + extern const int ARGUMENT_OUT_OF_BOUND; extern const int CANNOT_PARSE_TEXT; extern const int ILLEGAL_COLUMN; extern const int INCORRECT_QUERY; @@ -71,6 +72,7 @@ namespace { ASTPtr window_function; String window_column_name; + String timestamp_column_name; bool is_tumble = false; bool is_hop = false; }; @@ -98,6 +100,7 @@ namespace data.is_tumble = true; data.window_column_name = node.getColumnName(); data.window_function = node.clone(); + data.timestamp_column_name = node.arguments->children[0]->getColumnName(); } else if (serializeAST(node) != serializeAST(*data.window_function)) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); @@ -108,6 +111,7 @@ namespace { data.is_hop = true; data.window_function = node.clone(); + data.timestamp_column_name = node.arguments->children[0]->getColumnName(); auto ptr_ = node.clone(); std::static_pointer_cast(ptr_)->setAlias(""); auto arrayJoin = makeASTFunction("arrayJoin", ptr_); @@ -128,7 +132,7 @@ namespace struct Data { - bool is_proctime = false; + bool is_time_column_now = false; String window_column_name; }; @@ -150,7 +154,7 @@ namespace { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") { - data.is_proctime = true; + data.is_time_column_now = true; node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); data.window_column_name = node.getColumnName(); } @@ -158,7 +162,7 @@ namespace else if (node.name == "HOP") { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") - data.is_proctime = true; + data.is_time_column_now = true; } } }; @@ -184,6 +188,28 @@ namespace __builtin_unreachable(); } + static UInt32 addTime(UInt32 time_sec, IntervalKind::Kind window_kind, int window_num_units, const DateLUTImpl & time_zone) + { + switch (window_kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: \ + { \ + return AddTime::execute(time_sec, window_num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); + } + static inline String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } static ASTPtr generateDeleteRetiredQuery(StorageID inner_table_id, UInt32 timestamp) @@ -334,14 +360,27 @@ Pipes StorageWindowView::blocksToPipes(BlocksListPtrs & blocks, Block & sample_b inline void StorageWindowView::cleanCache() { - //delete fired blocks - UInt32 timestamp_now = std::time(nullptr); - UInt32 w_lower_bound = getWindowLowerBound(timestamp_now, -1); - if (watermark_num_units != 0) - w_lower_bound = getWatermark(w_lower_bound); + UInt32 w_bound; + if (is_proctime) + { + w_bound = getWindowUpperBound(std::time(nullptr)); + } + else + { + std::unique_lock lock(fire_signal_mutex); + if (max_watermark == 0) + return; + w_bound = addTime(max_watermark, window_kind, -1 * window_num_units, time_zone); + } + + if (is_tumble) + w_bound = addTime(w_bound, window_kind, -1 * window_num_units, time_zone); + else + w_bound = addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone); + if (!inner_table_id.empty()) { - auto sql = generateDeleteRetiredQuery(inner_table_id, w_lower_bound); + auto sql = generateDeleteRetiredQuery(inner_table_id, w_bound); InterpreterAlterQuery alt_query(sql, global_context); alt_query.execute(); } @@ -350,13 +389,13 @@ inline void StorageWindowView::cleanCache() std::lock_guard lock(mutex); for (BlocksListPtr mergeable_block : *mergeable_blocks) { - mergeable_block->remove_if([&w_lower_bound](Block & block_) + mergeable_block->remove_if([&w_bound](Block & block_) { auto & column_ = block_.getByName("____w_end").column; const auto & data = static_cast(*column_).getData(); for (size_t i = 0; i < column_->size(); ++i) { - if (data[i] >= w_lower_bound) + if (data[i] >= w_bound) return false; } return true; @@ -365,20 +404,19 @@ inline void StorageWindowView::cleanCache() mergeable_blocks->remove_if([](BlocksListPtr & ptr) { return ptr->size() == 0; }); } - { - std::unique_lock lock(fire_signal_mutex); - watch_streams.remove_if([](std::weak_ptr & ptr) { return ptr.expired(); }); - } + std::unique_lock lock(fire_signal_mutex); + watch_streams.remove_if([](std::weak_ptr & ptr) { return ptr.expired(); }); } -inline void StorageWindowView::fire(UInt32 timestamp_) +inline void StorageWindowView::fire(UInt32 watermark) { if (target_table_id.empty() && watch_streams.empty()) return; - auto in_stream = getNewBlocksInputStreamPtr(timestamp_); + auto in_stream = getNewBlocksInputStreamPtr(watermark); if (target_table_id.empty()) { + in_stream->readPrefix(); while (auto block_ = in_stream->read()) { for (auto & watch_stream : watch_streams) @@ -387,21 +425,32 @@ inline void StorageWindowView::fire(UInt32 timestamp_) watch_stream_->addBlock(block_); } } + in_stream->readSuffix(); } else { - StoragePtr target_table = getTargetStorage(); - auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); - auto out_stream = target_table->write(getInnerQuery(), global_context); - - while (auto block_ = in_stream->read()) + try { - for (auto & watch_stream : watch_streams) + StoragePtr target_table = getTargetStorage(); + auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); + auto out_stream = target_table->write(getInnerQuery(), global_context); + in_stream->readPrefix(); + out_stream->writePrefix(); + while (auto block_ = in_stream->read()) { - if (auto watch_stream_ = watch_stream.lock()) - watch_stream_->addBlock(block_); + for (auto & watch_stream : watch_streams) + { + if (auto watch_stream_ = watch_stream.lock()) + watch_stream_->addBlock(block_); + } + out_stream->write(std::move(block_)); } - out_stream->write(std::move(block_)); + in_stream->readSuffix(); + out_stream->writeSuffix(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); } } fire_condition.notify_all(); @@ -422,7 +471,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto columns_list = std::make_shared(); - if (is_proctime && is_tumble) + if (is_time_column_now && is_tumble) { auto column_window = std::make_shared(); column_window->name = window_column_name; @@ -453,32 +502,6 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery return manual_create_query; } -inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec, int window_id_skew) -{ - switch (window_kind) - { -#define CASE_WINDOW_KIND(KIND) \ - case IntervalKind::KIND: \ - { \ - UInt32 res = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ - if (window_id_skew != 0) \ - return AddTime::execute(res, window_id_skew * window_num_units, time_zone); \ - else \ - return res; \ - } - CASE_WINDOW_KIND(Second) - CASE_WINDOW_KIND(Minute) - CASE_WINDOW_KIND(Hour) - CASE_WINDOW_KIND(Day) - CASE_WINDOW_KIND(Week) - CASE_WINDOW_KIND(Month) - CASE_WINDOW_KIND(Quarter) - CASE_WINDOW_KIND(Year) -#undef CASE_WINDOW_KIND - } - __builtin_unreachable(); -} - inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window_id_skew) { switch (window_kind) @@ -506,55 +529,49 @@ inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window __builtin_unreachable(); } -static UInt32 addTime(UInt32 time_sec, IntervalKind::Kind window_kind, int window_num_units, const DateLUTImpl & time_zone) -{ - switch (window_kind) - { -#define CASE_WINDOW_KIND(KIND) \ - case IntervalKind::KIND: \ - { \ - return AddTime::execute(time_sec, window_num_units, time_zone); \ - } - CASE_WINDOW_KIND(Second) - CASE_WINDOW_KIND(Minute) - CASE_WINDOW_KIND(Hour) - CASE_WINDOW_KIND(Day) - CASE_WINDOW_KIND(Week) - CASE_WINDOW_KIND(Month) - CASE_WINDOW_KIND(Quarter) - CASE_WINDOW_KIND(Year) -#undef CASE_WINDOW_KIND - } - __builtin_unreachable(); -} - -inline UInt32 StorageWindowView::getWatermark(UInt32 time_sec) -{ - switch (watermark_kind) - { -#define CASE_WINDOW_KIND(KIND) \ - case IntervalKind::KIND: \ - { \ - return AddTime::execute(time_sec, watermark_num_units, time_zone); \ - } - CASE_WINDOW_KIND(Second) - CASE_WINDOW_KIND(Minute) - CASE_WINDOW_KIND(Hour) - CASE_WINDOW_KIND(Day) - CASE_WINDOW_KIND(Week) - CASE_WINDOW_KIND(Month) - CASE_WINDOW_KIND(Quarter) - CASE_WINDOW_KIND(Year) -#undef CASE_WINDOW_KIND - } - __builtin_unreachable(); -} - -inline void StorageWindowView::addLateFireSignal(UInt32 timestamp_) +inline void StorageWindowView::addFireSignal(std::deque & signals) { std::unique_lock lock(fire_signal_mutex); - late_fire_signal.push_back(timestamp_); - late_signal_condition.notify_all(); + for (auto signal : signals) + fire_signal.push_back(signal); + fire_signal_condition.notify_all(); +} + +inline void StorageWindowView::updateMaxTimestamp(UInt32 timestamp) +{ + std::unique_lock lock(fire_signal_mutex); + if (timestamp > max_timestamp) + max_timestamp = timestamp; +} + +inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) +{ + std::unique_lock lock(fire_signal_mutex); + if (max_watermark == 0) + { + max_watermark = watermark; + return; + } + + if (is_watermark_strictly_ascending) + { + while (max_watermark < watermark) + { + fire_signal.push_back(max_watermark); + max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); + } + } + else // strictly || bounded + { + UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units, time_zone); + while (max_watermark_bias <= max_timestamp) + { + fire_signal.push_back(max_watermark); + max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); + max_watermark_bias = addTime(max_watermark, window_kind, window_num_units, time_zone); + } + } + fire_signal_condition.notify_all(); } void StorageWindowView::threadFuncCleanCache() @@ -563,8 +580,8 @@ void StorageWindowView::threadFuncCleanCache() { try { - cleanCache(); sleep(clean_interval); + cleanCache(); } catch (...) { @@ -576,7 +593,7 @@ void StorageWindowView::threadFuncCleanCache() cleanCacheTask->scheduleAfter(RESCHEDULE_MS); } -void StorageWindowView::threadFuncFire() +void StorageWindowView::threadFuncFireProc() { std::unique_lock lock(fire_signal_mutex); while (!shutdown_called) @@ -588,21 +605,28 @@ void StorageWindowView::threadFuncFire() next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, time_zone); } - next_fire_signal = (getWindowUpperBound(timestamp_now)); + next_fire_signal = getWindowUpperBound(timestamp_now); UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - late_signal_condition.wait_for(lock, std::chrono::microseconds(static_cast(next_fire_signal) * 1000000 - timestamp_usec)); + fire_signal_condition.wait_for(lock, std::chrono::microseconds(static_cast(next_fire_signal) * 1000000 - timestamp_usec)); + } + if (!shutdown_called) + fireTask->scheduleAfter(RESCHEDULE_MS); +} - // fire late events - while (true) +void StorageWindowView::threadFuncFireEvent() +{ + std::unique_lock lock(fire_signal_mutex); + while (!shutdown_called) + { + bool signaled = std::cv_status::no_timeout == fire_signal_condition.wait_for(lock, std::chrono::seconds(5)); + + if (!signaled) + continue; + + while (!fire_signal.empty()) { - UInt32 timestamp_; - { - if (late_fire_signal.empty()) - break; - timestamp_ = late_fire_signal.front(); - late_fire_signal.pop_front(); - } - fire(timestamp_); + fire(fire_signal.front()); + fire_signal.pop_front(); } } if (!shutdown_called) @@ -673,9 +697,12 @@ StorageWindowView::StorageWindowView( final_query = inner_query->clone(); ParserProcTimeFinalMatcher::Data final_query_data; ParserProcTimeFinalMatcher::Visitor(final_query_data).visit(final_query); - is_proctime = final_query_data.is_proctime; - if (is_proctime && is_tumble) + is_time_column_now = final_query_data.is_time_column_now; + if (is_time_column_now && is_tumble) window_column_name = final_query_data.window_column_name; + is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; + is_watermark_ascending = query.is_watermark_ascending; + is_watermark_bounded = query.is_watermark_bounded; /// If the table is not specified - use the table `system.one` if (select_table_name.empty()) @@ -693,30 +720,39 @@ StorageWindowView::StorageWindowView( mergeable_blocks = std::make_shared>(); next_fire_signal = getWindowUpperBound(std::time(nullptr)); - if (query.watermark_function) + if (query.is_watermark_strictly_ascending || query.is_watermark_ascending || query.is_watermark_bounded) { - if (is_proctime) - throw Exception("WATERMARK is not support for Processing time processing.", ErrorCodes::INCORRECT_QUERY); - - // parser watermark function - const auto & watermark_function = std::static_pointer_cast(query.watermark_function); - if (!startsWith(watermark_function->name, "toInterval")) - throw Exception( - "Illegal type WATERMARK function " + watermark_function->name + ", should be Interval", ErrorCodes::ILLEGAL_COLUMN); - - String interval_str = watermark_function->name.substr(10); - const auto & interval_units_p1 = std::static_pointer_cast(watermark_function->children.front()->children.front()); - watermark_kind = strToIntervalKind(interval_str); - try + is_proctime = false; + if (is_time_column_now) + throw Exception("now() is not support for Event time processing.", ErrorCodes::INCORRECT_QUERY); + if (query.is_watermark_ascending) { - watermark_num_units = boost::lexical_cast(interval_units_p1->value.get()); + is_watermark_bounded = true; + watermark_kind = IntervalKind::Second; + watermark_num_units = 1; } - catch (const boost::bad_lexical_cast &) + else if (query.is_watermark_bounded) { - throw Exception("Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); + // parser watermark function + const auto & watermark_function = std::static_pointer_cast(query.watermark_function); + if (!startsWith(watermark_function->name, "toInterval")) + throw Exception( + "Illegal type WATERMARK function " + watermark_function->name + ", should be Interval", ErrorCodes::ILLEGAL_COLUMN); + + String interval_str = watermark_function->name.substr(10); + const auto & interval_units_p1 = std::static_pointer_cast(watermark_function->children.front()->children.front()); + watermark_kind = strToIntervalKind(interval_str); + try + { + watermark_num_units = boost::lexical_cast(interval_units_p1->value.get()); + } + catch (const boost::bad_lexical_cast &) + { + throw Exception("Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); + } + if (watermark_num_units <= 0) + throw Exception("Value for WATERMARK function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } - if (watermark_num_units > 0) - watermark_num_units *= -1; } if (query.storage) @@ -744,28 +780,26 @@ StorageWindowView::StorageWindowView( } { - // generate write expressions + // write expressions ColumnsWithTypeAndName columns__; columns__.emplace_back( nullptr, std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), window_column_name); columns__.emplace_back(nullptr, std::make_shared(), "____timestamp"); - columns__.emplace_back(nullptr, std::make_shared(), "____watermark"); const auto & function_tuple = FunctionFactory::instance().get("tupleElement", global_context); writeExpressions = std::make_shared(columns__, global_context); writeExpressions->add(ExpressionAction::addColumn( {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); writeExpressions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); writeExpressions->add(ExpressionAction::removeColumn("____tuple_arg")); - - const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", global_context); - writeExpressions->add(ExpressionAction::applyFunction(function_greater, Names{"____w_end", "____watermark"}, "____filter")); - writeExpressions->add(ExpressionAction::removeColumn("____watermark")); } cleanCacheTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); - fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFire(); }); + if (is_proctime) + fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); + else + fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); }); cleanCacheTask->deactivate(); fireTask->deactivate(); } @@ -784,6 +818,7 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) if (!stageMergeableOneData.is_tumble && !stageMergeableOneData.is_hop) throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); window_column_name = stageMergeableOneData.window_column_name; + timestamp_column_name = stageMergeableOneData.timestamp_column_name; is_tumble = stageMergeableOneData.is_tumble; // parser window function @@ -791,52 +826,81 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) const auto & arguments = window_function.arguments->children; const auto & arg1 = std::static_pointer_cast(arguments.at(1)); if (!arg1 || !startsWith(arg1->name, "toInterval")) - throw Exception("Illegal type of last argument of function " + arg1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - - String interval_str = arg1->name.substr(10); - window_kind = strToIntervalKind(interval_str); + throw Exception("Illegal type of second argument of function " + arg1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + window_kind = strToIntervalKind(arg1->name.substr(10)); const auto & interval_units_p1 = std::static_pointer_cast(arg1->children.front()->children.front()); - window_num_units = stoi(interval_units_p1->value.get()); + if (window_num_units <= 0) + throw Exception("Interval value for WINDOW function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + if (!is_tumble) + { + const auto & arg2 = std::static_pointer_cast(arguments.at(2)); + if (!arg2 || !startsWith(arg2->name, "toInterval")) + throw Exception("Illegal type of last argument of function " + arg2->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + hop_kind = strToIntervalKind(arg2->name.substr(10)); + const auto & interval_units_p2 = std::static_pointer_cast(arg2->children.front()->children.front()); + hop_num_units = stoi(interval_units_p2->value.get()); + if (hop_num_units <= 0) + throw Exception("Interval value for WINDOW function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } return result; } void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { - UInt32 timestamp_now; - UInt32 watermark; - Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); + BlockInputStreamPtr source_stream; std::shared_lock fire_signal_lock; if (window_view.is_proctime) { fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); - timestamp_now = std::time(nullptr); - watermark = window_view.getWindowLowerBound(timestamp_now); if (window_view.is_tumble) + { + UInt32 timestamp_now = std::time(nullptr); pipe.addSimpleTransform(std::make_shared>( pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); + } + InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); + + source_stream = select_block.execute().in; + source_stream = std::make_shared(source_stream, window_view.writeExpressions); + source_stream = std::make_shared( + source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); } else { - timestamp_now = std::time(nullptr); - if (window_view.watermark_num_units == 0) - watermark = window_view.getWindowLowerBound(timestamp_now); + UInt32 max_timestamp_ = 0; + if (!window_view.is_tumble || window_view.is_watermark_bounded) + { + auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; + const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); + for (size_t i = 0; i < timestamp_data.size(); ++i) + { + if (timestamp_data[i] > max_timestamp_) + max_timestamp_ = timestamp_data[i]; + } + } + + if (window_view.is_watermark_bounded) + window_view.updateMaxTimestamp(max_timestamp_); + + InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); + + source_stream = select_block.execute().in; + source_stream = std::make_shared(source_stream, window_view.writeExpressions); + source_stream = std::make_shared( + source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + + if (!window_view.is_tumble) + source_stream + = std::make_shared(source_stream, window_view, window_view.getWindowUpperBound(max_timestamp_)); else - watermark = window_view.getWatermark(timestamp_now); + source_stream = std::make_shared(source_stream, window_view); } - InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); - BlockInputStreamPtr source_stream = std::make_shared>(select_block.execute().in, std::make_shared(), watermark, "____watermark"); - - source_stream = std::make_shared(source_stream, window_view.writeExpressions, "____filter", true); - source_stream = std::make_shared( - source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); - - if (window_view.watermark_num_units != 0) - source_stream = std::make_shared(source_stream, window_view, timestamp_now); if (!window_view.inner_table_id.empty()) { @@ -879,7 +943,7 @@ StorageWindowView::~StorageWindowView() shutdown(); } -BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 timestamp_) +BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 watermark) { Pipes pipes; @@ -904,14 +968,14 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 timesta columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(timestamp_)), - std::make_shared(), - "____timestamp_now"})); + actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), + std::make_shared(), + "____watermark"})); const auto & function_equals = FunctionFactory::instance().get("equals", global_context); ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____timestamp_now"}, "____filter")); + actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____watermark"}, "____filter")); actions_->add(ExpressionAction::removeColumn("____w_end")); - actions_->add(ExpressionAction::removeColumn("____timestamp_now")); + actions_->add(ExpressionAction::removeColumn("____watermark")); for (auto & pipe : pipes) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), actions_, diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 2dcf8a8da93..55e0fd8324f 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -20,6 +20,7 @@ using BlocksListPtrs = std::shared_ptr>; class StorageWindowView : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; + friend class TimestampTransformation; friend class WatermarkBlockInputStream; friend class WindowViewBlockInputStream; @@ -51,7 +52,7 @@ public: BlocksListPtrs getMergeableBlocksList() { return mergeable_blocks; } - BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 timestamp_); + BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 watermark); static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); @@ -61,17 +62,23 @@ private: ASTPtr fetch_column_query; Context & global_context; - bool is_proctime{false}; + bool is_proctime{true}; + bool is_time_column_now; bool is_tumble; // false if is hop std::atomic shutdown_called{false}; mutable Block sample_block; mutable Block mergeable_sample_block; UInt64 clean_interval; const DateLUTImpl & time_zone; + UInt32 max_timestamp = 0; + UInt32 max_watermark = 0; + bool is_watermark_strictly_ascending{false}; + bool is_watermark_ascending{false}; + bool is_watermark_bounded{false}; UInt32 next_fire_signal; - std::deque late_fire_signal; + std::deque fire_signal; std::list> watch_streams; - std::condition_variable_any late_signal_condition; + std::condition_variable_any fire_signal_condition; std::condition_variable fire_condition; BlocksListPtrs mergeable_blocks; @@ -81,10 +88,13 @@ private: std::shared_mutex fire_signal_mutex; IntervalKind::Kind window_kind; + IntervalKind::Kind hop_kind; IntervalKind::Kind watermark_kind; Int64 window_num_units; + Int64 hop_num_units; Int64 watermark_num_units = 0; String window_column_name; + String timestamp_column_name; StorageID select_table_id = StorageID::createEmpty(); StorageID target_table_id = StorageID::createEmpty(); @@ -104,13 +114,15 @@ private: UInt32 getWindowLowerBound(UInt32 time_sec, int window_id_skew = 0); UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); - UInt32 getWatermark(UInt32 time_sec); - void fire(UInt32 timestamp_); + void fire(UInt32 watermark); void cleanCache(); void threadFuncCleanCache(); - void threadFuncFire(); - void addLateFireSignal(UInt32 timestamp_); + void threadFuncFireProc(); + void threadFuncFireEvent(); + void addFireSignal(std::deque & signals); + void updateMaxWatermark(UInt32 watermark); + void updateMaxTimestamp(UInt32 timestamp); static Pipes blocksToPipes(BlocksListPtrs & blocks, Block & sample_block); diff --git a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h index 12cde5adcbe..b93ab94bc71 100644 --- a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -11,11 +11,25 @@ namespace DB class WatermarkBlockInputStream : public IBlockInputStream { public: + WatermarkBlockInputStream( + BlockInputStreamPtr input_, + StorageWindowView& storage_) + : need_late_signal(false) + , watermark_specified(false) + , storage(storage_) + , max_watermark(0) + { + children.push_back(input_); + } + WatermarkBlockInputStream( BlockInputStreamPtr input_, StorageWindowView& storage_, - UInt32 timestamp_) - : storage(storage_), timestamp(timestamp_) + UInt32 max_watermark_) + : need_late_signal(false) + , watermark_specified(true) + , storage(storage_) + , max_watermark(max_watermark_) { children.push_back(input_); } @@ -34,26 +48,32 @@ protected: if (!res) return res; - auto column_wend = res.getByName("____w_end").column; + auto & column_wend = res.getByName("____w_end").column; const ColumnUInt32::Container & wend_data = static_cast(*column_wend).getData(); for (size_t i = 0; i < wend_data.size(); ++i) { - if (wend_data[i] < timestamp) - signals.push_back(wend_data[i]); + if (!watermark_specified && wend_data[i] > max_watermark) + max_watermark = wend_data[i]; + if (need_late_signal && wend_data[i] < late_timestamp) + late_signals.push_back(wend_data[i]); } return res; } void readSuffix() override { - for (auto signal : signals) - storage.addLateFireSignal(signal); - signals.clear(); + if (need_late_signal) + storage.addFireSignal(late_signals); + if (max_watermark > 0) + storage.updateMaxWatermark(max_watermark); } private: + bool need_late_signal; + bool watermark_specified; + std::deque late_signals; StorageWindowView & storage; - UInt32 timestamp; - std::deque signals; + UInt32 late_timestamp; + UInt32 max_watermark; }; } diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.reference b/dbms/tests/queries/0_stateless/01049_window_functions.reference index e69e82162a5..544fa875dfe 100644 --- a/dbms/tests/queries/0_stateless/01049_window_functions.reference +++ b/dbms/tests/queries/0_stateless/01049_window_functions.reference @@ -14,7 +14,7 @@ 2020-01-06 00:00:00 2020-01-07 00:00:00 2020-01-07 00:00:00 -2020-01-12 00:00:00 -2020-01-11 00:00:00 -2020-01-12 00:00:00 -2020-01-12 00:00:00 +2020-01-10 00:00:00 +2020-01-09 00:00:00 +2020-01-10 00:00:00 +2020-01-10 00:00:00 diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py b/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py deleted file mode 100755 index 29f7e947d41..00000000000 --- a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.py +++ /dev/null @@ -1,55 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -# log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('SET allow_experimental_window_view = 1') - client1.expect(prompt) - client2.send('SET allow_experimental_window_view = 1') - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.wv') - client1.expect(prompt) - - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") - client1.expect(prompt) - - client1.send('WATCH test.wv') - client1.expect('Progress: 0.00 rows.*\)') - client2.send('INSERT INTO test.mt VALUES (1)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') - - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.wv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.reference b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.reference rename to dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01053_window_view_proctime_tumble_to.sql rename to dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.reference rename to dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.reference diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01055_window_view_proctime_tumble_inner_to.sql rename to dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py b/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py deleted file mode 100755 index df4fecd4d4f..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.py +++ /dev/null @@ -1,57 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -# log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('SET allow_experimental_window_view = 1') - client1.expect(prompt) - client2.send('SET allow_experimental_window_view = 1') - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.wv') - client1.expect(prompt) - - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid;") - client1.expect(prompt) - - client1.send('WATCH test.wv') - client1.expect('Progress: 0.00 rows.*\)') - client2.send('INSERT INTO test.mt VALUES (1)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') - - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.wv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.reference b/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.reference rename to dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01057_window_view_proctime_hop_to.sql rename to dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.reference rename to dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.reference diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01059_window_view_proctime_hop_inner_to.sql rename to dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py b/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py deleted file mode 100755 index 3a630984fd7..00000000000 --- a/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.py +++ /dev/null @@ -1,55 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -# log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('SET allow_experimental_window_view = 1') - client1.expect(prompt) - client2.send('SET allow_experimental_window_view = 1') - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.wv') - client1.expect(prompt) - - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") - client1.expect(prompt) - - client1.send('WATCH test.wv') - client1.expect('Progress: 0.00 rows.*\)') - client2.send('INSERT INTO test.mt VALUES (1)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') - - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.wv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.reference b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.reference rename to dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01061_window_view_eventtime_tumble_to.sql rename to dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.reference rename to dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.reference diff --git a/dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01063_window_view_eventtime_tumble_inner_to.sql rename to dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py b/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py deleted file mode 100755 index 00867113e77..00000000000 --- a/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.py +++ /dev/null @@ -1,57 +0,0 @@ -#!/usr/bin/env python -import os -import sys -import signal - -CURDIR = os.path.dirname(os.path.realpath(__file__)) -sys.path.insert(0, os.path.join(CURDIR, 'helpers')) - -from client import client, prompt, end_of_block - -log = None -# uncomment the line below for debugging -# log=sys.stdout - -with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: - client1.expect(prompt) - client2.expect(prompt) - - client1.send('SET allow_experimental_window_view = 1') - client1.expect(prompt) - client2.send('SET allow_experimental_window_view = 1') - client2.expect(prompt) - - client1.send('DROP TABLE IF EXISTS test.mt') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.wv') - client1.expect(prompt) - - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send('CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple()') - client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") - client1.expect(prompt) - - client1.send('WATCH test.wv') - client1.expect('Progress: 0.00 rows.*\)') - client2.send('INSERT INTO test.mt VALUES (1)') - client1.expect('1' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') - - # send Ctrl-C - client1.send('\x03', eol='') - match = client1.expect('(%s)|([#\$] )' % prompt) - if match.groups()[1]: - client1.send(client1.command) - client1.expect(prompt) - client1.send('DROP TABLE test.wv') - client1.expect(prompt) - client1.send('DROP TABLE test.mt') - client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.reference b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.reference rename to dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01065_window_view_eventtime_hop_to.sql rename to dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.reference b/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.reference rename to dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.reference diff --git a/dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01067_window_view_eventtime_hop_inner_to.sql rename to dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql diff --git a/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py b/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py similarity index 86% rename from dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py rename to dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py index 4b057e9f565..6032862b234 100755 --- a/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py @@ -23,13 +23,9 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") @@ -51,5 +47,3 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE test.mt') client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.reference b/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01052_window_view_proctime_tumble_watch.reference rename to dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.reference b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.reference new file mode 100644 index 00000000000..50ab245a54d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.reference @@ -0,0 +1,4 @@ +0 +3 1990-01-01 12:00:05 +1 1990-01-01 12:00:10 +2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql new file mode 100644 index 00000000000..4ca08756f7d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.reference b/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference new file mode 100644 index 00000000000..50ab245a54d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference @@ -0,0 +1,4 @@ +0 +3 1990-01-01 12:00:05 +1 1990-01-01 12:00:10 +2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql new file mode 100644 index 00000000000..ed9f833c029 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.reference b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.reference new file mode 100644 index 00000000000..166a57070b0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.reference @@ -0,0 +1,7 @@ +0 +2 1990-01-01 12:00:02 +2 1990-01-01 12:00:04 +1 1990-01-01 12:00:06 +1 1990-01-01 12:00:08 +2 1990-01-01 12:00:12 +1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql new file mode 100644 index 00000000000..b371010e177 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.reference b/dbms/tests/queries/0_stateless/01064_window_view_eventtime_hop_watch.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference new file mode 100644 index 00000000000..166a57070b0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference @@ -0,0 +1,7 @@ +0 +2 1990-01-01 12:00:02 +2 1990-01-01 12:00:04 +1 1990-01-01 12:00:06 +1 1990-01-01 12:00:08 +2 1990-01-01 12:00:12 +1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql new file mode 100644 index 00000000000..a549458388d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql @@ -0,0 +1,25 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree() order by tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py similarity index 67% rename from dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py rename to dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py index fd34f78b957..fe96e66b954 100755 --- a/dbms/tests/queries/0_stateless/01060_window_view_eventtime_tumble_watch.py +++ b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py @@ -23,23 +23,24 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid;") client1.expect(prompt) client1.send('WATCH test.wv') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00');") client1.expect('Progress: 0.00 rows.*\)') - client2.send('INSERT INTO test.mt VALUES (1, now())') - client1.expect('1' + end_of_block) + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05');") + client1.expect('1*1990-01-01 12:00:02' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10');") + client1.expect('1*1990-01-01 12:00:06' + end_of_block) + client1.expect('1*1990-01-01 12:00:08' + end_of_block) + client1.expect('Progress: 3.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') @@ -51,5 +52,3 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE test.mt') client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.reference b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01054_window_view_proctime_tumble_inner_watch.reference rename to dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.reference b/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.reference deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.reference b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.reference new file mode 100644 index 00000000000..1096602f00f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.reference @@ -0,0 +1,4 @@ +0 +3 1990-01-01 12:00:05 +2 1990-01-01 12:00:10 +2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql new file mode 100644 index 00000000000..41b994ab9aa --- /dev/null +++ b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.reference new file mode 100644 index 00000000000..1096602f00f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.reference @@ -0,0 +1,4 @@ +0 +3 1990-01-01 12:00:05 +2 1990-01-01 12:00:10 +2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql new file mode 100644 index 00000000000..9864c55577c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference b/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql b/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql deleted file mode 100755 index 29c3c884ee0..00000000000 --- a/dbms/tests/queries/0_stateless/01068_window_view_eventtime_watermark.sql +++ /dev/null @@ -1,18 +0,0 @@ -SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=INTERVAL '5' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now() - 2); -SELECT sleep(1); -SELECT count from test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.reference b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.reference new file mode 100644 index 00000000000..047b7bfbec9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.reference @@ -0,0 +1,7 @@ +0 +2 1990-01-01 12:00:02 +2 1990-01-01 12:00:04 +1 1990-01-01 12:00:06 +2 1990-01-01 12:00:08 +2 1990-01-01 12:00:12 +1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql new file mode 100644 index 00000000000..aa89f1e85e8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference b/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql b/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql deleted file mode 100755 index 02a9bbd919f..00000000000 --- a/dbms/tests/queries/0_stateless/01069_window_view_eventtime_inner_watermark.sql +++ /dev/null @@ -1,18 +0,0 @@ -SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; - -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree() order by tuple() WATERMARK=INTERVAL '5' SECOND AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO test.mt VALUES (1, now() - 2); -SELECT sleep(1); -SELECT count from test.dst; - -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.reference new file mode 100644 index 00000000000..047b7bfbec9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.reference @@ -0,0 +1,7 @@ +0 +2 1990-01-01 12:00:02 +2 1990-01-01 12:00:04 +1 1990-01-01 12:00:06 +2 1990-01-01 12:00:08 +2 1990-01-01 12:00:12 +1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql new file mode 100644 index 00000000000..9e3c5faf75e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py b/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py similarity index 64% rename from dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py rename to dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py index 7b3f36a35f1..a122f56bc0e 100755 --- a/dbms/tests/queries/0_stateless/01062_window_view_eventtime_tumble_inner_watch.py +++ b/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py @@ -23,25 +23,26 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.`.inner.wv`') - client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid") client1.expect(prompt) client1.send('WATCH test.wv') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00');") client1.expect('Progress: 0.00 rows.*\)') - client2.send('INSERT INTO test.mt VALUES (1, now())') - client1.expect('1' + end_of_block) + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05');") + client1.expect('1*1990-01-01 12:00:02' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06');") + client1.expect('Progress: 1.00 rows.*\)') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10');") + client1.expect('1*1990-01-01 12:00:06' + end_of_block) + client1.expect('2*1990-01-01 12:00:08' + end_of_block) + client1.expect('Progress: 3.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') @@ -53,5 +54,3 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE test.mt') client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.reference b/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01056_window_view_proctime_hop_watch.reference rename to dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference new file mode 100644 index 00000000000..1096602f00f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference @@ -0,0 +1,4 @@ +0 +3 1990-01-01 12:00:05 +2 1990-01-01 12:00:10 +2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql new file mode 100644 index 00000000000..7622568aa31 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK = INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference new file mode 100644 index 00000000000..1096602f00f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference @@ -0,0 +1,4 @@ +0 +3 1990-01-01 12:00:05 +2 1990-01-01 12:00:10 +2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql new file mode 100644 index 00000000000..455d90a344a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference new file mode 100644 index 00000000000..047b7bfbec9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference @@ -0,0 +1,7 @@ +0 +2 1990-01-01 12:00:02 +2 1990-01-01 12:00:04 +1 1990-01-01 12:00:06 +2 1990-01-01 12:00:08 +2 1990-01-01 12:00:12 +1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql new file mode 100644 index 00000000000..a15c3bcacba --- /dev/null +++ b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference new file mode 100644 index 00000000000..047b7bfbec9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference @@ -0,0 +1,7 @@ +0 +2 1990-01-01 12:00:02 +2 1990-01-01 12:00:04 +1 1990-01-01 12:00:06 +2 1990-01-01 12:00:08 +2 1990-01-01 12:00:12 +1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql new file mode 100644 index 00000000000..948bbc0816e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql @@ -0,0 +1,26 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py b/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py similarity index 63% rename from dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py rename to dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py index 43b3086acbd..f2ed70fe7d6 100755 --- a/dbms/tests/queries/0_stateless/01066_window_view_eventtime_hop_inner_watch.py +++ b/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py @@ -23,23 +23,26 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('DROP TABLE IF EXISTS test.mt') client1.expect(prompt) - client1.send('DROP TABLE IF EXISTS test.dst') - client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) - client1.send('CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple()') - client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid;") + client1.send("CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid") client1.expect(prompt) client1.send('WATCH test.wv') - client2.send('INSERT INTO test.mt VALUES (1, now())') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00');") client1.expect('Progress: 0.00 rows.*\)') - client1.expect('1' + end_of_block) + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05');") + client1.expect('1*1990-01-01 12:00:02' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06');") + client1.expect('Progress: 1.00 rows.*\)') + client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10');") + client1.expect('1*1990-01-01 12:00:06' + end_of_block) + client1.expect('2*1990-01-01 12:00:08' + end_of_block) + client1.expect('Progress: 3.00 rows.*\)') # send Ctrl-C client1.send('\x03', eol='') @@ -51,5 +54,3 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE test.mt') client1.expect(prompt) - client1.send('DROP TABLE test.dst') - client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.reference b/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01058_window_view_proctime_hop_inner_watch.reference rename to dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.reference From a759799ca8f96f17ba1bdec703e0cc10c51a5f40 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 3 Mar 2020 12:42:12 +0800 Subject: [PATCH 032/609] support lateness --- dbms/src/Parsers/ASTCreateQuery.cpp | 6 + dbms/src/Parsers/ASTCreateQuery.h | 8 +- dbms/src/Parsers/ParserCreateQuery.cpp | 15 + .../Storages/WindowView/BlocksListSource.h | 1 - .../Storages/WindowView/StorageWindowView.cpp | 285 +++++++++++++----- .../Storages/WindowView/StorageWindowView.h | 70 ++--- .../WindowView/WatermarkBlockInputStream.h | 40 ++- ...nt_strict_asc_tumble_to_lateness.reference | 4 + ...ew_event_strict_asc_tumble_to_lateness.sql | 29 ++ ...iew_event_asc_tumble_to_lateness.reference | 4 + ...ndow_view_event_asc_tumble_to_lateness.sql | 29 ++ ...event_bounded_tumble_to_lateness.reference | 3 + ..._view_event_bounded_tumble_to_lateness.sql | 30 ++ 13 files changed, 384 insertions(+), 140 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.reference create mode 100644 dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql create mode 100644 dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.reference create mode 100644 dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql create mode 100644 dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.reference create mode 100644 dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index 81660cd8d74..ee923803610 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -310,6 +310,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat watermark_function->formatImpl(settings, state, frame); } + if (allowed_lateness) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " ALLOWED_LATENESS " << (settings.hilite ? hilite_none : ""); + lateness_function->formatImpl(settings, state, frame); + } + if (select) { settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 7c0bdf825b7..09382c6902f 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -62,9 +62,10 @@ public: bool is_populate{false}; bool is_dictionary{false}; /// CREATE DICTIONARY bool replace_view{false}; /// CREATE OR REPLACE VIEW - bool is_watermark_strictly_ascending{false}; /// STRICTLY ASCENDING WATERMARK STRATEGY - bool is_watermark_ascending{false}; /// ASCENDING WATERMARK STRATEGY - bool is_watermark_bounded{false}; /// BOUNDED OUT OF ORDERNESS WATERMARK STRATEGY + bool is_watermark_strictly_ascending{false}; /// STRICTLY ASCENDING WATERMARK STRATEGY FOR WINDOW VIEW + bool is_watermark_ascending{false}; /// ASCENDING WATERMARK STRATEGY FOR WINDOW VIEW + bool is_watermark_bounded{false}; /// BOUNDED OUT OF ORDERNESS WATERMARK STRATEGY FOR WINDOW VIEW + bool allowed_lateness{false}; /// ALLOWED LATENESS FOR WINDOW VIEW ASTColumns * columns_list = nullptr; ASTExpressionList * dictionary_attributes_list = nullptr; /// attributes of dictionary ASTExpressionList * tables = nullptr; @@ -72,6 +73,7 @@ public: String to_table; ASTStorage * storage = nullptr; ASTPtr watermark_function; + ASTPtr lateness_function; String as_database; String as_table; ASTPtr as_table_function; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 52820864e1b..d6d6da494b6 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -632,6 +632,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserIdentifier name_p; ParserTablePropertiesDeclarationList table_properties_p; ParserIntervalOperatorExpression watermark_p; + ParserIntervalOperatorExpression lateness_p; ParserSelectWithUnionQuery select_p; ASTPtr database; @@ -641,6 +642,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ASTPtr to_table; ASTPtr storage; ASTPtr watermark; + ASTPtr lateness; ASTPtr as_database; ASTPtr as_table; ASTPtr select; @@ -650,6 +652,7 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & bool is_watermark_strictly_ascending = false; bool is_watermark_ascending = false; bool is_watermark_bounded = false; + bool allowed_lateness = false; bool if_not_exists = false; bool is_temporary = false; @@ -733,6 +736,16 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & return false; } + // ALLOWED LATENESS + if (ParserKeyword{"ALLOWED_LATENESS"}.ignore(pos, expected)) + { + s_eq.ignore(pos, expected); + allowed_lateness = true; + + if (!lateness_p.parse(pos, lateness, expected)) + return false; + } + /// AS SELECT ... if (!s_as.ignore(pos, expected)) return false; @@ -762,6 +775,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->is_watermark_ascending = is_watermark_ascending; query->is_watermark_bounded = is_watermark_bounded; query->watermark_function = watermark; + query->allowed_lateness = allowed_lateness; + query->lateness_function = lateness; tryGetIdentifierNameInto(as_database, query->as_database); tryGetIdentifierNameInto(as_table, query->as_table); diff --git a/dbms/src/Storages/WindowView/BlocksListSource.h b/dbms/src/Storages/WindowView/BlocksListSource.h index abcb280d493..66cb475bc9e 100644 --- a/dbms/src/Storages/WindowView/BlocksListSource.h +++ b/dbms/src/Storages/WindowView/BlocksListSource.h @@ -6,7 +6,6 @@ namespace DB { using BlocksListPtr = std::shared_ptr; -using BlocksListPtrs = std::shared_ptr>; /** A stream of blocks from a shared list of blocks */ diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 0cad71f3806..af4ed197377 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -337,7 +336,7 @@ void StorageWindowView::truncate(const ASTPtr &, const Context &, TableStructure else { std::lock_guard lock(mutex); - mergeable_blocks = std::make_shared>(); + mergeable_blocks.clear(); } } @@ -349,12 +348,11 @@ bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, return getInnerStorage()->optimize(query, partition, final, deduplicate, context); } -Pipes StorageWindowView::blocksToPipes(BlocksListPtrs & blocks, Block & sample_block) +Pipes StorageWindowView::blocksToPipes(BlocksList & blocks, Block & sample_block) { Pipes pipes; - for (auto & blocks_ : *blocks) - pipes.emplace_back(std::make_shared(blocks_, sample_block)); - + for (auto & block_ : blocks) + pipes.emplace_back(std::make_shared(sample_block, Chunk(block_.getColumns(), block_.rows()))); return pipes; } @@ -367,16 +365,24 @@ inline void StorageWindowView::cleanCache() } else { - std::unique_lock lock(fire_signal_mutex); + std::lock_guard lock(fire_signal_mutex); if (max_watermark == 0) return; - w_bound = addTime(max_watermark, window_kind, -1 * window_num_units, time_zone); + w_bound = max_fired_watermark; + if (w_bound == 0) + return; + + if (allowed_lateness) + { + UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, time_zone); + lateness_bound = getWindowLowerBound(lateness_bound); + if (lateness_bound < w_bound) + w_bound = lateness_bound; + } } - if (is_tumble) - w_bound = addTime(w_bound, window_kind, -1 * window_num_units, time_zone); - else - w_bound = addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone); + w_bound = is_tumble ? addTime(w_bound, window_kind, -1 * window_num_units, time_zone) + : addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone); if (!inner_table_id.empty()) { @@ -387,24 +393,19 @@ inline void StorageWindowView::cleanCache() else { std::lock_guard lock(mutex); - for (BlocksListPtr mergeable_block : *mergeable_blocks) - { - mergeable_block->remove_if([&w_bound](Block & block_) + mergeable_blocks.remove_if([w_bound](Block & block_) { + auto & column_ = block_.getByName("____w_end").column; + const auto & data = static_cast(*column_).getData(); + for (size_t i = 0; i < column_->size(); ++i) { - auto & column_ = block_.getByName("____w_end").column; - const auto & data = static_cast(*column_).getData(); - for (size_t i = 0; i < column_->size(); ++i) - { - if (data[i] >= w_bound) - return false; - } - return true; - }); - } - mergeable_blocks->remove_if([](BlocksListPtr & ptr) { return ptr->size() == 0; }); + if (data[i] >= w_bound) + return false; + } + return true; + }); } - std::unique_lock lock(fire_signal_mutex); + std::lock_guard lock(fire_signal_mutex); watch_streams.remove_if([](std::weak_ptr & ptr) { return ptr.expired(); }); } @@ -413,7 +414,12 @@ inline void StorageWindowView::fire(UInt32 watermark) if (target_table_id.empty() && watch_streams.empty()) return; - auto in_stream = getNewBlocksInputStreamPtr(watermark); + BlockInputStreamPtr in_stream; + { + std::lock_guard lock(mutex); + in_stream = getNewBlocksInputStreamPtr(watermark); + } + if (target_table_id.empty()) { in_stream->readPrefix(); @@ -502,19 +508,23 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery return manual_create_query; } -inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window_id_skew) +inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) { - switch (window_kind) + IntervalKind window_kind_; + if (is_tumble) + window_kind_ = window_kind; + else + window_kind_ = hop_kind; + + switch (window_kind_) { #define CASE_WINDOW_KIND(KIND) \ case IntervalKind::KIND: \ { \ - UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ - UInt32 res = AddTime::execute(start, window_num_units, time_zone); \ - if (window_id_skew != 0) \ - return AddTime::execute(res, window_id_skew * window_num_units, time_zone); \ + if (is_tumble) \ + return ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ else \ - return res; \ + return ToStartOfTransform::execute(time_sec, hop_num_units, time_zone); \ } CASE_WINDOW_KIND(Second) CASE_WINDOW_KIND(Minute) @@ -529,49 +539,78 @@ inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec, int window __builtin_unreachable(); } -inline void StorageWindowView::addFireSignal(std::deque & signals) +inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) { - std::unique_lock lock(fire_signal_mutex); - for (auto signal : signals) - fire_signal.push_back(signal); + switch (window_kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: \ + { \ + UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ + return AddTime::execute(start, window_num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); +} + +inline void StorageWindowView::addFireSignal(std::set & signals) +{ + std::lock_guard lock(fire_signal_mutex); + for (auto it = signals.begin(); it != signals.end(); ++it) + fire_signal.push_back(*it); fire_signal_condition.notify_all(); } inline void StorageWindowView::updateMaxTimestamp(UInt32 timestamp) { - std::unique_lock lock(fire_signal_mutex); + std::lock_guard lock(fire_signal_mutex); if (timestamp > max_timestamp) max_timestamp = timestamp; } inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) { - std::unique_lock lock(fire_signal_mutex); + std::lock_guard lock(fire_signal_mutex); if (max_watermark == 0) { max_watermark = watermark; return; } + bool updated; if (is_watermark_strictly_ascending) { + updated = max_watermark < watermark; while (max_watermark < watermark) { fire_signal.push_back(max_watermark); + max_fired_watermark = max_watermark; max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); } } else // strictly || bounded { UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units, time_zone); + updated = max_watermark_bias <= watermark; while (max_watermark_bias <= max_timestamp) { fire_signal.push_back(max_watermark); + max_fired_watermark = max_watermark; max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); max_watermark_bias = addTime(max_watermark, window_kind, window_num_units, time_zone); } } - fire_signal_condition.notify_all(); + if (updated) + fire_signal_condition.notify_all(); } void StorageWindowView::threadFuncCleanCache() @@ -619,7 +658,6 @@ void StorageWindowView::threadFuncFireEvent() while (!shutdown_called) { bool signaled = std::cv_status::no_timeout == fire_signal_condition.wait_for(lock, std::chrono::seconds(5)); - if (!signaled) continue; @@ -645,23 +683,17 @@ BlockInputStreams StorageWindowView::watch( bool has_limit = false; UInt64 limit = 0; - if (query.limit_length) { has_limit = true; limit = safeGet(typeid_cast(*query.limit_length).value); } - auto reader = std::make_shared( - std::static_pointer_cast(shared_from_this()), - has_limit, - limit); - - { - std::unique_lock lock(fire_signal_mutex); - watch_streams.push_back(reader); - } + auto reader + = std::make_shared(std::static_pointer_cast(shared_from_this()), has_limit, limit); + std::lock_guard lock(fire_signal_mutex); + watch_streams.push_back(reader); processed_stage = QueryProcessingStage::Complete; return {reader}; @@ -717,7 +749,6 @@ StorageWindowView::StorageWindowView( target_table_id = StorageID(query.to_database, query.to_table); clean_interval = local_context.getSettingsRef().window_view_clean_interval.totalSeconds(); - mergeable_blocks = std::make_shared>(); next_fire_signal = getWindowUpperBound(std::time(nullptr)); if (query.is_watermark_strictly_ascending || query.is_watermark_ascending || query.is_watermark_bounded) @@ -736,12 +767,10 @@ StorageWindowView::StorageWindowView( // parser watermark function const auto & watermark_function = std::static_pointer_cast(query.watermark_function); if (!startsWith(watermark_function->name, "toInterval")) - throw Exception( - "Illegal type WATERMARK function " + watermark_function->name + ", should be Interval", ErrorCodes::ILLEGAL_COLUMN); + throw Exception("Illegal type of WATERMARK function, should be Interval", ErrorCodes::ILLEGAL_COLUMN); - String interval_str = watermark_function->name.substr(10); const auto & interval_units_p1 = std::static_pointer_cast(watermark_function->children.front()->children.front()); - watermark_kind = strToIntervalKind(interval_str); + watermark_kind = strToIntervalKind(watermark_function->name.substr(10)); try { watermark_num_units = boost::lexical_cast(interval_units_p1->value.get()); @@ -755,6 +784,30 @@ StorageWindowView::StorageWindowView( } } + if (query.allowed_lateness) + { + allowed_lateness = true; + + // parser lateness function + const auto & lateness_function = std::static_pointer_cast(query.lateness_function); + if (!startsWith(lateness_function->name, "toInterval")) + throw Exception("Illegal type of ALLOWED_LATENESS function, should be Interval", ErrorCodes::ILLEGAL_COLUMN); + + const auto & interval_units_p1 = std::static_pointer_cast(lateness_function->children.front()->children.front()); + lateness_kind = strToIntervalKind(lateness_function->name.substr(10)); + try + { + lateness_num_units = boost::lexical_cast(interval_units_p1->value.get()); + } + catch (const boost::bad_lexical_cast &) + { + throw Exception( + "Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); + } + if (lateness_num_units <= 0) + throw Exception("Value for ALLOWED_LATENESS function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } + if (query.storage) { if (attach_) @@ -871,21 +924,57 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con } else { - UInt32 max_timestamp_ = 0; - if (!window_view.is_tumble || window_view.is_watermark_bounded) + UInt32 max_fired_watermark_ = 0; + if (window_view.allowed_lateness) + { + UInt32 max_timestamp_ = 0; + UInt32 max_watermark_ = 0; + { + std::lock_guard lock(window_view.fire_signal_mutex); + max_fired_watermark_ = window_view.max_fired_watermark; + max_watermark_ = window_view.max_watermark; + max_timestamp_ = window_view.max_timestamp; + } + + if (max_timestamp_!= 0) + { + UInt32 lateness_bound + = addTime(max_timestamp_, window_view.lateness_kind, -1 * window_view.lateness_num_units, window_view.time_zone); + if (window_view.is_watermark_bounded) + { + UInt32 watermark_lower_bound = window_view.is_tumble + ? addTime(max_watermark_, window_view.window_kind, -1 * window_view.window_num_units, window_view.time_zone) + : addTime(max_watermark_, window_view.hop_kind, -1 * window_view.hop_num_units, window_view.time_zone); + if (watermark_lower_bound < lateness_bound) + lateness_bound = watermark_lower_bound; + } + + ColumnsWithTypeAndName columns__; + columns__.emplace_back(nullptr, std::make_shared(), window_view.timestamp_column_name); + ExpressionActionsPtr filterExpressions = std::make_shared(columns__, context); + filterExpressions->add( + ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(lateness_bound)), + std::make_shared(), + "____lateness_bound"})); + const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); + filterExpressions->add(ExpressionAction::applyFunction( + function_greater, Names{window_view.timestamp_column_name, "____lateness_bound"}, "____filter")); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), filterExpressions, "____filter", true)); + } + } + + UInt32 max_timestamp__ = 0; + if (!window_view.is_tumble || window_view.is_watermark_bounded || window_view.allowed_lateness) { auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); for (size_t i = 0; i < timestamp_data.size(); ++i) { - if (timestamp_data[i] > max_timestamp_) - max_timestamp_ = timestamp_data[i]; + if (timestamp_data[i] > max_timestamp__) + max_timestamp__ = timestamp_data[i]; } } - if (window_view.is_watermark_bounded) - window_view.updateMaxTimestamp(max_timestamp_); - InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; @@ -895,13 +984,17 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (!window_view.is_tumble) source_stream - = std::make_shared(source_stream, window_view, window_view.getWindowUpperBound(max_timestamp_)); + = std::make_shared(source_stream, window_view, window_view.getWindowUpperBound(max_timestamp__)); else source_stream = std::make_shared(source_stream, window_view); + + if (window_view.is_watermark_bounded || window_view.allowed_lateness) + std::static_pointer_cast(source_stream)->setMaxTimestamp(max_timestamp__); + + if (window_view.allowed_lateness && max_fired_watermark_ != 0) + std::static_pointer_cast(source_stream)->setAllowedLateness(max_fired_watermark_); } - - if (!window_view.inner_table_id.empty()) { auto & inner_storage = window_view.getInnerStorage(); @@ -911,14 +1004,13 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con } else { - BlocksListPtr new_mergeable_blocks = std::make_shared(); source_stream->readPrefix(); - while (Block block_ = source_stream->read()) - new_mergeable_blocks->push_back(std::move(block_)); + { + std::lock_guard lock(window_view.mutex); + while (Block block_ = source_stream->read()) + window_view.mergeable_blocks.push_back(std::move(block_)); + } source_stream->readSuffix(); - - std::unique_lock lock(window_view.mutex); - window_view.getMergeableBlocksList()->push_back(new_mergeable_blocks); } } @@ -943,6 +1035,48 @@ StorageWindowView::~StorageWindowView() shutdown(); } +Block & StorageWindowView::getHeader() const +{ + std::lock_guard lock(sample_block_lock); + if (!sample_block) + { + sample_block = InterpreterSelectQuery( + getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) + .getSampleBlock(); + for (size_t i = 0; i < sample_block.columns(); ++i) + sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); + } + return sample_block; +} + +StoragePtr StorageWindowView::getParentStorage() const +{ + if (parent_storage == nullptr) + parent_storage = global_context.getTable(select_table_id); + return parent_storage; +} + +Block & StorageWindowView::getMergeableHeader() const +{ + if (!mergeable_sample_block) + mergeable_sample_block = mergeable_blocks.front().cloneEmpty(); + return mergeable_sample_block; +} + +StoragePtr & StorageWindowView::getInnerStorage() const +{ + if (inner_storage == nullptr && !inner_table_id.empty()) + inner_storage = global_context.getTable(inner_table_id); + return inner_storage; +} + +StoragePtr & StorageWindowView::getTargetStorage() const +{ + if (target_storage == nullptr && !target_table_id.empty()) + target_storage = global_context.getTable(target_table_id); + return target_storage; +} + BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 watermark) { Pipes pipes; @@ -958,8 +1092,7 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma } else { - std::unique_lock lock(mutex); - if (mergeable_blocks->empty()) + if (mergeable_blocks.empty()) return std::make_shared(getHeader()); pipes = blocksToPipes(mergeable_blocks, getMergeableHeader()); } diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 55e0fd8324f..8c814d16c14 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -14,8 +14,6 @@ namespace DB class IAST; class WindowViewBlockInputStream; using ASTPtr = std::shared_ptr; -using BlocksListPtr = std::shared_ptr; -using BlocksListPtrs = std::shared_ptr>; class StorageWindowView : public ext::shared_ptr_helper, public IStorage { @@ -50,8 +48,6 @@ public: size_t max_block_size, unsigned num_streams) override; - BlocksListPtrs getMergeableBlocksList() { return mergeable_blocks; } - BlockInputStreamPtr getNewBlocksInputStreamPtr(UInt32 watermark); static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); @@ -71,37 +67,42 @@ private: UInt64 clean_interval; const DateLUTImpl & time_zone; UInt32 max_timestamp = 0; - UInt32 max_watermark = 0; + UInt32 max_watermark = 0; // next watermark to fire + UInt32 max_fired_watermark = 0; bool is_watermark_strictly_ascending{false}; bool is_watermark_ascending{false}; bool is_watermark_bounded{false}; + bool allowed_lateness{false}; UInt32 next_fire_signal; std::deque fire_signal; std::list> watch_streams; std::condition_variable_any fire_signal_condition; std::condition_variable fire_condition; - BlocksListPtrs mergeable_blocks; + BlocksList mergeable_blocks; /// Mutex for the blocks and ready condition std::mutex mutex; std::mutex flush_table_mutex; std::shared_mutex fire_signal_mutex; + mutable std::mutex sample_block_lock; /// Mutex to protect access to sample block and inner_blocks_query IntervalKind::Kind window_kind; IntervalKind::Kind hop_kind; IntervalKind::Kind watermark_kind; + IntervalKind::Kind lateness_kind; Int64 window_num_units; Int64 hop_num_units; Int64 watermark_num_units = 0; + Int64 lateness_num_units = 0; String window_column_name; String timestamp_column_name; StorageID select_table_id = StorageID::createEmpty(); StorageID target_table_id = StorageID::createEmpty(); StorageID inner_table_id = StorageID::createEmpty(); - StoragePtr parent_storage; - StoragePtr inner_storage; - StoragePtr target_storage; + mutable StoragePtr parent_storage; + mutable StoragePtr inner_storage; + mutable StoragePtr target_storage; BackgroundSchedulePool::TaskHolder cleanCacheTask; BackgroundSchedulePool::TaskHolder fireTask; @@ -112,65 +113,32 @@ private: std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); - UInt32 getWindowLowerBound(UInt32 time_sec, int window_id_skew = 0); - UInt32 getWindowUpperBound(UInt32 time_sec, int window_id_skew = 0); + UInt32 getWindowLowerBound(UInt32 time_sec); + UInt32 getWindowUpperBound(UInt32 time_sec); void fire(UInt32 watermark); void cleanCache(); void threadFuncCleanCache(); void threadFuncFireProc(); void threadFuncFireEvent(); - void addFireSignal(std::deque & signals); + void addFireSignal(std::set & signals); void updateMaxWatermark(UInt32 watermark); void updateMaxTimestamp(UInt32 timestamp); - static Pipes blocksToPipes(BlocksListPtrs & blocks, Block & sample_block); + static Pipes blocksToPipes(BlocksList & blocks, Block & sample_block); ASTPtr getInnerQuery() const { return inner_query->clone(); } ASTPtr getFinalQuery() const { return final_query->clone(); } - StoragePtr getParentStorage() - { - if (parent_storage == nullptr) - parent_storage = global_context.getTable(select_table_id); - return parent_storage; - } + StoragePtr getParentStorage() const; - StoragePtr& getInnerStorage() - { - if (inner_storage == nullptr && !inner_table_id.empty()) - inner_storage = global_context.getTable(inner_table_id); - return inner_storage; - } + StoragePtr & getInnerStorage() const; - StoragePtr& getTargetStorage() - { - if (target_storage == nullptr && !target_table_id.empty()) - target_storage = global_context.getTable(target_table_id); - return target_storage; - } + StoragePtr & getTargetStorage() const; - Block & getHeader() - { - if (!sample_block) - { - sample_block = InterpreterSelectQuery( - getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) - .getSampleBlock(); - for (size_t i = 0; i < sample_block.columns(); ++i) - sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); - } - return sample_block; - } + Block & getHeader() const; - Block & getMergeableHeader() - { - if (!mergeable_sample_block) - { - mergeable_sample_block = mergeable_blocks->front()->front().cloneEmpty(); - } - return mergeable_sample_block; - } + Block & getMergeableHeader() const; StorageWindowView( const StorageID & table_id_, diff --git a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h index b93ab94bc71..0b43582a532 100644 --- a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -14,9 +14,12 @@ public: WatermarkBlockInputStream( BlockInputStreamPtr input_, StorageWindowView& storage_) - : need_late_signal(false) + : allowed_lateness(false) + , update_timestamp(false) , watermark_specified(false) , storage(storage_) + , lateness_upper_bound(0) + , max_timestamp(0) , max_watermark(0) { children.push_back(input_); @@ -26,9 +29,12 @@ public: BlockInputStreamPtr input_, StorageWindowView& storage_, UInt32 max_watermark_) - : need_late_signal(false) + : allowed_lateness(false) + , update_timestamp(false) , watermark_specified(true) , storage(storage_) + , lateness_upper_bound(0) + , max_timestamp(0) , max_watermark(max_watermark_) { children.push_back(input_); @@ -41,6 +47,18 @@ public: return children.back()->getHeader(); } + void setAllowedLateness(UInt32 upper_bound) + { + allowed_lateness = true; + lateness_upper_bound = upper_bound; + } + + void setMaxTimestamp(UInt32 timestamp) + { + update_timestamp = true; + max_timestamp = timestamp; + } + protected: Block readImpl() override { @@ -54,26 +72,30 @@ protected: { if (!watermark_specified && wend_data[i] > max_watermark) max_watermark = wend_data[i]; - if (need_late_signal && wend_data[i] < late_timestamp) - late_signals.push_back(wend_data[i]); + if (allowed_lateness && wend_data[i] <= lateness_upper_bound) + late_signals.insert(wend_data[i]); } return res; } void readSuffix() override { - if (need_late_signal) - storage.addFireSignal(late_signals); + if (update_timestamp) + storage.updateMaxTimestamp(max_timestamp); if (max_watermark > 0) storage.updateMaxWatermark(max_watermark); + if (allowed_lateness) + storage.addFireSignal(late_signals); } private: - bool need_late_signal; + bool allowed_lateness; + bool update_timestamp; bool watermark_specified; - std::deque late_signals; + std::set late_signals; StorageWindowView & storage; - UInt32 late_timestamp; + UInt32 lateness_upper_bound; + UInt32 max_timestamp; UInt32 max_watermark; }; } diff --git a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.reference b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.reference new file mode 100644 index 00000000000..cee01df9c92 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.reference @@ -0,0 +1,4 @@ +0 +2 1990-01-01 12:00:05 +3 1990-01-01 12:00:05 +4 1990-01-01 12:00:10 diff --git a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql new file mode 100644 index 00000000000..30ed27385f4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql @@ -0,0 +1,29 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=STRICTLY_ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:04'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:03'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:07'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end, count; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.reference b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.reference new file mode 100644 index 00000000000..cee01df9c92 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.reference @@ -0,0 +1,4 @@ +0 +2 1990-01-01 12:00:05 +3 1990-01-01 12:00:05 +4 1990-01-01 12:00:10 diff --git a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql new file mode 100644 index 00000000000..54b34860e98 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql @@ -0,0 +1,29 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:04'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:03'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:07'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end, count; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; diff --git a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.reference b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.reference new file mode 100644 index 00000000000..8abc4a63899 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.reference @@ -0,0 +1,3 @@ +0 +4 1990-01-01 12:00:05 +4 1990-01-01 12:00:10 diff --git a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql new file mode 100644 index 00000000000..92b9d930b97 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql @@ -0,0 +1,30 @@ +SET allow_experimental_window_view = 1; +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS test.dst; +DROP TABLE IF EXISTS test.wv; + +CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=INTERVAL '2' SECOND ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:04'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:03'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:07'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:12'); + +SELECT sleep(1); +SELECT * from test.dst order by w_end, count; + +DROP TABLE test.wv; +DROP TABLE test.mt; +DROP TABLE test.dst; From 7e7f1cea9f5739fba877bbfa51316b637e1c8177 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 6 Mar 2020 22:58:16 +0800 Subject: [PATCH 033/609] add heartbeat interval setting --- dbms/src/Core/Defines.h | 1 + dbms/src/Core/Settings.h | 1 + dbms/src/Storages/WindowView/StorageWindowView.cpp | 10 +++++++--- .../Storages/WindowView/WindowViewBlockInputStream.h | 9 ++++++--- 4 files changed, 15 insertions(+), 6 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index fca458c0e90..47c42317896 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -35,6 +35,7 @@ #define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 #define DEFAULT_WINDOW_VIEW_CLEAN_INTERVAL_SEC 5 +#define DEFAULT_WINDOW_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) #define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 0c0d5dd961e..e1591f04852 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -388,6 +388,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.", 0) \ M(SettingBool, allow_experimental_window_view, false, "Enable WINDOW VIEW. Not mature enough.", 0) \ M(SettingSeconds, window_view_clean_interval, DEFAULT_WINDOW_VIEW_CLEAN_INTERVAL_SEC, "The clean interval of window view in seconds to free outdated data.", 0) \ + M(SettingSeconds, window_view_heartbeat_interval, DEFAULT_WINDOW_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate watch query is alive.", 0) \ M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index af4ed197377..b929f1d2b88 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -393,7 +393,8 @@ inline void StorageWindowView::cleanCache() else { std::lock_guard lock(mutex); - mergeable_blocks.remove_if([w_bound](Block & block_) { + mergeable_blocks.remove_if([w_bound](Block & block_) + { auto & column_ = block_.getByName("____w_end").column; const auto & data = static_cast(*column_).getData(); for (size_t i = 0; i < column_->size(); ++i) @@ -689,8 +690,11 @@ BlockInputStreams StorageWindowView::watch( limit = safeGet(typeid_cast(*query.limit_length).value); } - auto reader - = std::make_shared(std::static_pointer_cast(shared_from_this()), has_limit, limit); + auto reader = std::make_shared( + std::static_pointer_cast(shared_from_this()), + has_limit, + limit, + global_context.getSettingsRef().window_view_heartbeat_interval.totalSeconds()); std::lock_guard lock(fire_signal_mutex); watch_streams.push_back(reader); diff --git a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h index a67faebb68d..01bbfe54ab2 100644 --- a/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -16,10 +16,12 @@ public: WindowViewBlockInputStream( std::shared_ptr storage_, const bool has_limit_, - const UInt64 limit_) + const UInt64 limit_, + const UInt64 heartbeat_interval_sec_) : storage(std::move(storage_)) , has_limit(has_limit_) - , limit(limit_) {} + , limit(limit_) + , heartbeat_interval_sec(heartbeat_interval_sec_) {} String getName() const override { return "WindowViewBlock"; } @@ -64,7 +66,7 @@ protected: return getHeader(); } - storage->fire_condition.wait_for(lock_, std::chrono::seconds(5)); + storage->fire_condition.wait_for(lock_, std::chrono::seconds(heartbeat_interval_sec)); if (isCancelled() || storage->is_dropped) { @@ -97,5 +99,6 @@ private: Int64 num_updates = -1; bool end_of_blocks = false; std::mutex blocks_mutex; + UInt64 heartbeat_interval_sec; }; } From 42dbdcc205c70cffdbec3a92dd2205450164309b Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 11 Mar 2020 12:45:29 +0800 Subject: [PATCH 034/609] revert registerStorages --- dbms/src/Storages/registerStorages.cpp | 1 - dbms/src/Storages/registerStorages.h | 1 - 2 files changed, 2 deletions(-) diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index 3badb8cdf03..6874ce3384e 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -30,7 +30,6 @@ void registerStorages() registerStorageMaterializedView(factory); registerStorageLiveView(factory); registerStorageGenerateRandom(factory); - registerStorageGenerate(factory); registerStorageWindowView(factory); #if USE_AWS_S3 diff --git a/dbms/src/Storages/registerStorages.h b/dbms/src/Storages/registerStorages.h index 55ead6bbfcb..581a92eff96 100644 --- a/dbms/src/Storages/registerStorages.h +++ b/dbms/src/Storages/registerStorages.h @@ -24,7 +24,6 @@ void registerStorageView(StorageFactory & factory); void registerStorageMaterializedView(StorageFactory & factory); void registerStorageLiveView(StorageFactory & factory); void registerStorageGenerateRandom(StorageFactory & factory); -void registerStorageGenerate(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); #if USE_AWS_S3 From 8486f24168a5a16240ec527e3abef1646c371104 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 11 Mar 2020 12:57:04 +0800 Subject: [PATCH 035/609] remove hard code database in tests --- .../01050_window_view_sql_parser_tumble.sql | 32 +++++++------- .../01051_window_view_sql_parser_hop.sql | 32 +++++++------- .../01053_window_view_proc_now_tumble_to.sql | 22 +++++----- ...4_window_view_proc_now_tumble_inner_to.sql | 22 +++++----- .../01055_window_view_proc_now_hop_to.sql | 22 +++++----- ...1056_window_view_proc_now_hop_inner_to.sql | 22 +++++----- .../01057_window_view_proc_tumble_to.sql | 22 +++++----- ...01058_window_view_proc_tumble_inner_to.sql | 22 +++++----- .../01059_window_view_proc_hop_to.sql | 22 +++++----- .../01060_window_view_proc_hop_inner_to.sql | 22 +++++----- ...window_view_event_strict_asc_tumble_to.sql | 34 +++++++-------- ..._view_event_strict_asc_tumble_inner_to.sql | 34 +++++++-------- ...64_window_view_event_strict_asc_hop_to.sql | 34 +++++++-------- ...dow_view_event_strict_asc_hop_inner_to.sql | 34 +++++++-------- .../01067_window_view_event_asc_tumble_to.sql | 36 ++++++++-------- ..._window_view_event_asc_tumble_inner_to.sql | 36 ++++++++-------- .../01069_window_view_event_asc_hop_to.sql | 36 ++++++++-------- ...070_window_view_event_asc_hop_inner_to.sql | 36 ++++++++-------- ...72_window_view_event_bounded_tumble_to.sql | 36 ++++++++-------- ...dow_view_event_bounded_tumble_inner_to.sql | 36 ++++++++-------- ...01074_window_view_event_bounded_hop_to.sql | 36 ++++++++-------- ...window_view_event_bounded_hop_inner_to.sql | 36 ++++++++-------- ...ew_event_strict_asc_tumble_to_lateness.sql | 40 +++++++++--------- ...ndow_view_event_asc_tumble_to_lateness.sql | 40 +++++++++--------- ..._view_event_bounded_tumble_to_lateness.sql | 42 +++++++++---------- 25 files changed, 393 insertions(+), 393 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql index 55d92bbe37c..37b86314cfa 100644 --- a/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql +++ b/dbms/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql @@ -1,32 +1,32 @@ SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---WATERMARK---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---With w_end---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WithOut w_end---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WITH---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end, date_time FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end, date_time FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WHERE---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---ORDER_BY---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; -DROP TABLE test.mt; -DROP TABLE test.wv; \ No newline at end of file +DROP TABLE mt; +DROP TABLE wv; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql index 01d9ce9fe13..eae0de10c4b 100644 --- a/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql +++ b/dbms/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql @@ -1,32 +1,32 @@ SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; +DROP TABLE IF EXISTS mt; -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---WATERMARK---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---With w_end---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WithOut w_end---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WITH---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end, date_time FROM test.mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end, date_time FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WHERE---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---ORDER_BY---'; -DROP TABLE IF EXISTS test.wv; -CREATE WINDOW VIEW test.wv AS SELECT count(a), HOP_START(wid) AS w_start FROM test.mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; -DROP TABLE test.mt; -DROP TABLE test.wv; \ No newline at end of file +DROP TABLE mt; +DROP TABLE wv; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql index 5de210ad2b2..4ebda16f5cf 100755 --- a/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1); +INSERT INTO mt VALUES (1); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql index a2b65561590..477b8ed4fae 100755 --- a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1); +INSERT INTO mt VALUES (1); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql index 627516fcd46..1c4b0cdbc26 100755 --- a/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1); +INSERT INTO mt VALUES (1); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql index 601b1aa3588..a41fabdccf9 100755 --- a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv to test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv to dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1); +INSERT INTO mt VALUES (1); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql index 333694c989c..c64e5c49c86 100755 --- a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, now()); +INSERT INTO mt VALUES (1, now()); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql index 1b8afaa1541..5af4267033d 100755 --- a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, now()); +INSERT INTO mt VALUES (1, now()); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql index 722844741d4..51c819dac29 100755 --- a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, now()); +INSERT INTO mt VALUES (1, now()); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql index 4e59b8602d6..125d98c6b25 100755 --- a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql @@ -1,18 +1,18 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM test.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, now()); +INSERT INTO mt VALUES (1, now()); SELECT sleep(2); -SELECT count from test.dst; +SELECT count from dst; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql index 4ca08756f7d..6a675d26df9 100644 --- a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql @@ -1,25 +1,25 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql index ed9f833c029..5025f2a075f 100644 --- a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql @@ -1,25 +1,25 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql index b371010e177..eedc24d19fc 100644 --- a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql @@ -1,25 +1,25 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql index a549458388d..872f39649ef 100644 --- a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql @@ -1,25 +1,25 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree() order by tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree() order by tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql index 41b994ab9aa..16c9b9d6e10 100644 --- a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql index 9864c55577c..13aae098fcb 100644 --- a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql index aa89f1e85e8..f6c0e036868 100644 --- a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql index 9e3c5faf75e..34aae857a29 100644 --- a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql index 7622568aa31..8cddd87d8f4 100644 --- a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK = INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK = INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql index 455d90a344a..ef84d472b1e 100644 --- a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql index a15c3bcacba..b1f90a66765 100644 --- a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql index 948bbc0816e..ca4969123a0 100644 --- a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql @@ -1,26 +1,26 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:30'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); SELECT sleep(1); -SELECT * from test.dst order by w_end; +SELECT * from dst order by w_end; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql index 30ed27385f4..2882be59ef4 100644 --- a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql +++ b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql @@ -1,29 +1,29 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=STRICTLY_ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:04'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:03'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:07'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:04'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:03'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:07'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); SELECT sleep(1); -SELECT * from test.dst order by w_end, count; +SELECT * from dst order by w_end, count; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql index 54b34860e98..31eaf397e47 100644 --- a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql +++ b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql @@ -1,29 +1,29 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:04'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:03'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:07'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:04'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:03'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:07'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); SELECT sleep(1); -SELECT * from test.dst order by w_end, count; +SELECT * from dst order by w_end, count; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql index 92b9d930b97..96c1d2d0099 100644 --- a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql +++ b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql @@ -1,30 +1,30 @@ SET allow_experimental_window_view = 1; SET allow_experimental_window_view = 1; -DROP TABLE IF EXISTS test.mt; -DROP TABLE IF EXISTS test.dst; -DROP TABLE IF EXISTS test.wv; +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS dst; +DROP TABLE IF EXISTS wv; -CREATE TABLE test.dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); +CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=INTERVAL '2' SECOND ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=INTERVAL '2' SECOND ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:04'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:03'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:07'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:12'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:04'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:03'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:07'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); +INSERT INTO mt VALUES (1, '1990/01/01 12:00:12'); SELECT sleep(1); -SELECT * from test.dst order by w_end, count; +SELECT * from dst order by w_end, count; -DROP TABLE test.wv; -DROP TABLE test.mt; -DROP TABLE test.dst; +DROP TABLE wv; +DROP TABLE mt; +DROP TABLE dst; From 05f5062a09f66ab47d46df2be127336b7f14a11b Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 22 Mar 2020 23:03:16 +0800 Subject: [PATCH 036/609] support inner table params --- .../Storages/WindowView/StorageWindowView.cpp | 202 +++++++++++------- .../Storages/WindowView/StorageWindowView.h | 3 +- ...4_window_view_proc_now_tumble_inner_to.sql | 2 +- ...1056_window_view_proc_now_hop_inner_to.sql | 2 +- ...01058_window_view_proc_tumble_inner_to.sql | 2 +- .../01060_window_view_proc_hop_inner_to.sql | 2 +- ..._view_event_strict_asc_tumble_inner_to.sql | 2 +- ...dow_view_event_strict_asc_hop_inner_to.sql | 2 +- ..._window_view_event_asc_tumble_inner_to.sql | 2 +- ...070_window_view_event_asc_hop_inner_to.sql | 2 +- ...window_view_event_bounded_hop_inner_to.sql | 2 +- 11 files changed, 141 insertions(+), 82 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index b929f1d2b88..e0e358a2544 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -24,10 +24,12 @@ #include #include #include +#include #include #include #include #include +#include #include #include #include @@ -62,108 +64,115 @@ namespace { const auto RESCHEDULE_MS = 500; - class ParserStageMergeableOneMatcher + struct StageMergeableVisitorData { - public: - using Visitor = InDepthNodeVisitor; + using TypeToVisit = ASTFunction; - struct Data - { - ASTPtr window_function; - String window_column_name; - String timestamp_column_name; - bool is_tumble = false; - bool is_hop = false; - }; + ASTPtr window_function; + String window_column_name; + String window_column_alias; + String timestamp_column_name; + bool is_tumble = false; + bool is_hop = false; - static bool needChildVisit(ASTPtr & node, const ASTPtr &) - { - if (node->as()) - return false; - return true; - } - - static void visit(ASTPtr & ast, Data & data) - { - if (const auto * t = ast->as()) - visit(*t, ast, data); - } - - private: - static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data & data) + void visit(const ASTFunction & node, ASTPtr & node_ptr) { if (node.name == "TUMBLE") { - if (!data.window_function) + if (!window_function) { - data.is_tumble = true; - data.window_column_name = node.getColumnName(); - data.window_function = node.clone(); - data.timestamp_column_name = node.arguments->children[0]->getColumnName(); + is_tumble = true; + window_column_name = node.getColumnName(); + window_column_alias = node.alias; + window_function = node.clone(); + timestamp_column_name = node.arguments->children[0]->getColumnName(); } - else if (serializeAST(node) != serializeAST(*data.window_function)) + else if (serializeAST(node) != serializeAST(*window_function)) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); } else if (node.name == "HOP") { - if (!data.window_function) + if (!window_function) { - data.is_hop = true; - data.window_function = node.clone(); - data.timestamp_column_name = node.arguments->children[0]->getColumnName(); + is_hop = true; + window_function = node.clone(); + timestamp_column_name = node.arguments->children[0]->getColumnName(); auto ptr_ = node.clone(); std::static_pointer_cast(ptr_)->setAlias(""); auto arrayJoin = makeASTFunction("arrayJoin", ptr_); arrayJoin->alias = node.alias; node_ptr = arrayJoin; - data.window_column_name = arrayJoin->getColumnName(); + window_column_name = arrayJoin->getColumnName(); + window_column_alias = arrayJoin->alias; } - else if (serializeAST(node) != serializeAST(*data.window_function)) + else if (serializeAST(node) != serializeAST(*window_function)) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); } } }; - class ParserProcTimeFinalMatcher + struct ReplaceFuncNowVisitorData + { + using TypeToVisit = ASTFunction; + + bool is_time_column_func_now = false; + String window_column_name; + + void visit(ASTFunction & node, ASTPtr & node_ptr) + { + if (node.name == "TUMBLE") + { + if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") + { + is_time_column_func_now = true; + node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); + window_column_name = node.getColumnName(); + } + } + else if (node.name == "HOP") + { + if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") + is_time_column_func_now = true; + } + } + }; + + class ReplaceFunctionWindowMatcher { public: - using Visitor = InDepthNodeVisitor; + using Visitor = InDepthNodeVisitor; struct Data { - bool is_time_column_now = false; String window_column_name; + String window_column_alias; }; - static bool needChildVisit(ASTPtr &, const ASTPtr &) - { - return true; - } + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } static void visit(ASTPtr & ast, Data & data) { if (const auto * t = ast->as()) visit(*t, ast, data); + if (const auto * t = ast->as()) + visit(*t, ast, data); } private: static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data & data) { - if (node.name == "TUMBLE") + if (node.name == "TUMBLE" || node.name == "HOP") { - if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") - { - data.is_time_column_now = true; - node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); - data.window_column_name = node.getColumnName(); - } - } - else if (node.name == "HOP") - { - if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") - data.is_time_column_now = true; + if (queryToString(node) == data.window_column_name) + node_ptr = std::make_shared(data.window_column_name); } } + + static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data) + { + if (node.name == data.window_column_alias) + node_ptr = std::make_shared(data.window_column_name); + } }; static inline IntervalKind strToIntervalKind(const String& interval_str) @@ -478,7 +487,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto columns_list = std::make_shared(); - if (is_time_column_now && is_tumble) + if (is_time_column_func_now && is_tumble) { auto column_window = std::make_shared(); column_window->name = window_column_name; @@ -502,9 +511,57 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery column_wend->type = std::make_shared("DateTime"); columns_list->children.push_back(column_wend); + if (inner_create_query.storage->ttl_table) + throw Exception("TTL is not supported for inner table in Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + + ReplaceFunctionWindowMatcher::Data query_data; + query_data.window_column_name = window_column_name; + query_data.window_column_alias = window_column_alias; + ReplaceFunctionWindowMatcher::Visitor visitor(query_data); + + ReplaceFuncNowVisitorData parser_proc_time_data; + InDepthNodeVisitor, true> time_now_visitor(parser_proc_time_data); + + auto storage = std::make_shared(); + storage->set(storage->engine, inner_create_query.storage->engine->clone()); + if (inner_create_query.storage->partition_by) + { + auto partition_by = inner_create_query.storage->partition_by->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(partition_by); + visitor.visit(partition_by); + storage->set(storage->partition_by, partition_by); + } + if (inner_create_query.storage->primary_key) + { + auto primary_key = inner_create_query.storage->primary_key->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(primary_key); + visitor.visit(primary_key); + storage->set(storage->primary_key, primary_key); + } + if (inner_create_query.storage->order_by) + { + auto order_by = inner_create_query.storage->order_by->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(order_by); + visitor.visit(order_by); + storage->set(storage->order_by, order_by); + } + if (inner_create_query.storage->sample_by) + { + auto sample_by = inner_create_query.storage->sample_by->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(sample_by); + visitor.visit(sample_by); + storage->set(storage->sample_by, sample_by); + } + if (inner_create_query.storage->settings) + storage->set(storage->settings, inner_create_query.storage->settings->clone()); + new_columns_list->set(new_columns_list->columns, columns_list); manual_create_query->set(manual_create_query->columns_list, new_columns_list); - manual_create_query->set(manual_create_query->storage, inner_create_query.storage->ptr()); + manual_create_query->set(manual_create_query->storage, storage); return manual_create_query; } @@ -731,10 +788,10 @@ StorageWindowView::StorageWindowView( inner_query = innerQueryParser(select_query); final_query = inner_query->clone(); - ParserProcTimeFinalMatcher::Data final_query_data; - ParserProcTimeFinalMatcher::Visitor(final_query_data).visit(final_query); - is_time_column_now = final_query_data.is_time_column_now; - if (is_time_column_now && is_tumble) + ReplaceFuncNowVisitorData final_query_data; + InDepthNodeVisitor, true>(final_query_data).visit(final_query); + is_time_column_func_now = final_query_data.is_time_column_func_now; + if (is_time_column_func_now && is_tumble) window_column_name = final_query_data.window_column_name; is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; is_watermark_ascending = query.is_watermark_ascending; @@ -758,7 +815,7 @@ StorageWindowView::StorageWindowView( if (query.is_watermark_strictly_ascending || query.is_watermark_ascending || query.is_watermark_bounded) { is_proctime = false; - if (is_time_column_now) + if (is_time_column_func_now) throw Exception("now() is not support for Event time processing.", ErrorCodes::INCORRECT_QUERY); if (query.is_watermark_ascending) { @@ -820,7 +877,7 @@ StorageWindowView::StorageWindowView( } else { - if (query.storage->engine->name != "MergeTree") + if (!endsWith(query.storage->engine->name, "MergeTree")) throw Exception( "The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support", ErrorCodes::INCORRECT_QUERY); @@ -870,16 +927,17 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parse stage mergeable ASTPtr result = query.clone(); ASTPtr expr_list = result; - ParserStageMergeableOneMatcher::Data stageMergeableOneData; - ParserStageMergeableOneMatcher::Visitor(stageMergeableOneData).visit(expr_list); - if (!stageMergeableOneData.is_tumble && !stageMergeableOneData.is_hop) + StageMergeableVisitorData stageMergeableData; + InDepthNodeVisitor, true>(stageMergeableData).visit(expr_list); + if (!stageMergeableData.is_tumble && !stageMergeableData.is_hop) throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); - window_column_name = stageMergeableOneData.window_column_name; - timestamp_column_name = stageMergeableOneData.timestamp_column_name; - is_tumble = stageMergeableOneData.is_tumble; + window_column_name = stageMergeableData.window_column_name; + window_column_alias = stageMergeableData.window_column_alias; + timestamp_column_name = stageMergeableData.timestamp_column_name; + is_tumble = stageMergeableData.is_tumble; // parser window function - ASTFunction & window_function = typeid_cast(*stageMergeableOneData.window_function); + ASTFunction & window_function = typeid_cast(*stageMergeableData.window_function); const auto & arguments = window_function.arguments->children; const auto & arg1 = std::static_pointer_cast(arguments.at(1)); if (!arg1 || !startsWith(arg1->name, "toInterval")) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 8c814d16c14..87460d5280d 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -59,7 +59,7 @@ private: Context & global_context; bool is_proctime{true}; - bool is_time_column_now; + bool is_time_column_func_now; bool is_tumble; // false if is hop std::atomic shutdown_called{false}; mutable Block sample_block; @@ -95,6 +95,7 @@ private: Int64 watermark_num_units = 0; Int64 lateness_num_units = 0; String window_column_name; + String window_column_alias; String timestamp_column_name; StorageID select_table_id = StorageID::createEmpty(); diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql index 477b8ed4fae..a184d43ebbc 100755 --- a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree PARTITION BY wid ORDER BY tuple(TUMBLE(now(), INTERVAL '1' SECOND)) AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; INSERT INTO mt VALUES (1); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql index a41fabdccf9..89c78784ab3 100755 --- a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv to dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW wv to dst ENGINE=AggregatingMergeTree ORDER BY tuple(wid) AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; INSERT INTO mt VALUES (1); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql index 5af4267033d..f4226f24cec 100755 --- a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; INSERT INTO mt VALUES (1, now()); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql index 125d98c6b25..0a6a4fff2cb 100755 --- a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; INSERT INTO mt VALUES (1, now()); SELECT sleep(2); diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql index 5025f2a075f..f4ad0b6dcd6 100644 --- a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql index 872f39649ef..34502703f14 100644 --- a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree() order by tuple() WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree() order by wid WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql index 13aae098fcb..e50b94dd269 100644 --- a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql index 34aae857a29..e889761f3dd 100644 --- a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql index ca4969123a0..1f2179182e1 100644 --- a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); From 2c0bfbe4bbe2b5d9ed3956e2afe9dc1f9a98e849 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 23 Mar 2020 11:21:01 +0800 Subject: [PATCH 037/609] fix build for clang-8 --- .../Storages/WindowView/StorageWindowView.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 6840e9445e1..18f13a9627c 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -175,7 +175,7 @@ namespace } }; - static inline IntervalKind strToIntervalKind(const String& interval_str) + IntervalKind strToIntervalKind(const String& interval_str) { if (interval_str == "Second") return IntervalKind::Second; @@ -196,7 +196,7 @@ namespace __builtin_unreachable(); } - static UInt32 addTime(UInt32 time_sec, IntervalKind::Kind window_kind, int window_num_units, const DateLUTImpl & time_zone) + UInt32 addTime(UInt32 time_sec, IntervalKind::Kind window_kind, int window_num_units, const DateLUTImpl & time_zone) { switch (window_kind) { @@ -218,9 +218,9 @@ namespace __builtin_unreachable(); } - static inline String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } + String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } - static ASTPtr generateDeleteRetiredQuery(StorageID inner_table_id, UInt32 timestamp) + ASTPtr generateDeleteRetiredQuery(StorageID inner_table_id, UInt32 timestamp) { auto function_equal = makeASTFunction( "less", std::make_shared("____w_end"), std::make_shared(timestamp)); @@ -240,7 +240,7 @@ namespace return alterQuery; } - static std::shared_ptr generateFetchColumnsQuery(const StorageID & inner_storage) + std::shared_ptr generateFetchColumnsQuery(const StorageID & inner_storage) { auto res_query = std::make_shared(); auto select = std::make_shared(); @@ -623,8 +623,8 @@ inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) inline void StorageWindowView::addFireSignal(std::set & signals) { std::lock_guard lock(fire_signal_mutex); - for (auto it = signals.begin(); it != signals.end(); ++it) - fire_signal.push_back(*it); + for (auto & signal : signals) + fire_signal.push_back(signal); fire_signal_condition.notify_all(); } @@ -1030,10 +1030,10 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con { auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); - for (size_t i = 0; i < timestamp_data.size(); ++i) + for (auto& timestamp_ : timestamp_data) { - if (timestamp_data[i] > max_timestamp__) - max_timestamp__ = timestamp_data[i]; + if (timestamp_ > max_timestamp__) + max_timestamp__ = timestamp_; } } From 2430521fbae7a40094757640e884699f6888d72f Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 24 Mar 2020 10:40:34 +0800 Subject: [PATCH 038/609] reduce waiting time in tests --- dbms/src/Storages/WindowView/StorageWindowView.cpp | 4 ++-- .../queries/0_stateless/01061_window_view_proc_hop_watch.py | 2 ++ .../01066_window_view_event_strict_asc_hop_watch.py | 2 ++ .../0_stateless/01071_window_view_event_asc_hop_watch.py | 2 ++ .../0_stateless/01076_window_view_event_bounded_hop_watch.py | 2 ++ 5 files changed, 10 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 18f13a9627c..b31144bc329 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -732,7 +732,7 @@ void StorageWindowView::threadFuncFireEvent() BlockInputStreams StorageWindowView::watch( const Names & /*column_names*/, const SelectQueryInfo & query_info, - const Context & /*context*/, + const Context & context, QueryProcessingStage::Enum & processed_stage, size_t /*max_block_size*/, const unsigned /*num_streams*/) @@ -751,7 +751,7 @@ BlockInputStreams StorageWindowView::watch( std::static_pointer_cast(shared_from_this()), has_limit, limit, - global_context.getSettingsRef().window_view_heartbeat_interval.totalSeconds()); + context.getSettingsRef().window_view_heartbeat_interval.totalSeconds()); std::lock_guard lock(fire_signal_mutex); watch_streams.push_back(reader); diff --git a/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py b/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py index 6032862b234..4b7d7055ac4 100755 --- a/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py @@ -18,6 +18,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('SET allow_experimental_window_view = 1') client1.expect(prompt) + client1.send('SET window_view_heartbeat_interval = 1') + client1.expect(prompt) client2.send('SET allow_experimental_window_view = 1') client2.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py index fe96e66b954..a9ee0557736 100755 --- a/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py @@ -18,6 +18,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('SET allow_experimental_window_view = 1') client1.expect(prompt) + client1.send('SET window_view_heartbeat_interval = 1') + client1.expect(prompt) client2.send('SET allow_experimental_window_view = 1') client2.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py b/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py index a122f56bc0e..028cc62e10f 100755 --- a/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py @@ -18,6 +18,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('SET allow_experimental_window_view = 1') client1.expect(prompt) + client1.send('SET window_view_heartbeat_interval = 1') + client1.expect(prompt) client2.send('SET allow_experimental_window_view = 1') client2.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py b/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py index f2ed70fe7d6..d9d1603b922 100755 --- a/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py +++ b/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py @@ -18,6 +18,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('SET allow_experimental_window_view = 1') client1.expect(prompt) + client1.send('SET window_view_heartbeat_interval = 1') + client1.expect(prompt) client2.send('SET allow_experimental_window_view = 1') client2.expect(prompt) From 8802f81bc9b404df464a742b1c6e21fe2c1749a1 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 24 Mar 2020 10:46:56 +0800 Subject: [PATCH 039/609] update code style --- .../Storages/WindowView/StorageWindowView.cpp | 217 +++++++++--------- .../Storages/WindowView/StorageWindowView.h | 6 +- 2 files changed, 109 insertions(+), 114 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index b31144bc329..7a34a30b9c8 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -97,13 +97,13 @@ namespace is_hop = true; window_function = node.clone(); timestamp_column_name = node.arguments->children[0]->getColumnName(); - auto ptr_ = node.clone(); - std::static_pointer_cast(ptr_)->setAlias(""); - auto arrayJoin = makeASTFunction("arrayJoin", ptr_); - arrayJoin->alias = node.alias; - node_ptr = arrayJoin; - window_column_name = arrayJoin->getColumnName(); - window_column_alias = arrayJoin->alias; + auto ptr = node.clone(); + std::static_pointer_cast(ptr)->setAlias(""); + auto array_join = makeASTFunction("arrayJoin", ptr); + array_join->alias = node.alias; + node_ptr = array_join; + window_column_name = array_join->getColumnName(); + window_column_alias = array_join->alias; } else if (serializeAST(node) != serializeAST(*window_function)) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); @@ -225,19 +225,19 @@ namespace auto function_equal = makeASTFunction( "less", std::make_shared("____w_end"), std::make_shared(timestamp)); - auto alterCommand = std::make_shared(); - alterCommand->type = ASTAlterCommand::DELETE; - alterCommand->predicate = function_equal; - alterCommand->children.push_back(alterCommand->predicate); + auto alter_command = std::make_shared(); + alter_command->type = ASTAlterCommand::DELETE; + alter_command->predicate = function_equal; + alter_command->children.push_back(alter_command->predicate); - auto alterCommandList = std::make_shared(); - alterCommandList->add(alterCommand); + auto alter_command_list = std::make_shared(); + alter_command_list->add(alter_command); - auto alterQuery = std::make_shared(); - alterQuery->database = inner_table_id.database_name; - alterQuery->table = inner_table_id.table_name; - alterQuery->set(alterQuery->command_list, alterCommandList); - return alterQuery; + auto alter_query = std::make_shared(); + alter_query->database = inner_table_id.database_name; + alter_query->table = inner_table_id.table_name; + alter_query->set(alter_query->command_list, alter_command_list); + return alter_query; } std::shared_ptr generateFetchColumnsQuery(const StorageID & inner_storage) @@ -246,9 +246,6 @@ namespace auto select = std::make_shared(); select->children.push_back(std::make_shared()); res_query->setExpression(ASTSelectQuery::Expression::SELECT, select); - - auto tableInSelectQuery = std::make_shared(); - auto tableInSelectQueryElement = std::make_shared(); res_query->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); auto tables = res_query->tables(); auto tables_elem = std::make_shared(); @@ -360,8 +357,8 @@ bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, Pipes StorageWindowView::blocksToPipes(BlocksList & blocks, Block & sample_block) { Pipes pipes; - for (auto & block_ : blocks) - pipes.emplace_back(std::make_shared(sample_block, Chunk(block_.getColumns(), block_.rows()))); + for (auto & block : blocks) + pipes.emplace_back(std::make_shared(sample_block, Chunk(block.getColumns(), block.rows()))); return pipes; } @@ -402,11 +399,11 @@ inline void StorageWindowView::cleanCache() else { std::lock_guard lock(mutex); - mergeable_blocks.remove_if([w_bound](Block & block_) + mergeable_blocks.remove_if([w_bound](Block & block) { - auto & column_ = block_.getByName("____w_end").column; - const auto & data = static_cast(*column_).getData(); - for (size_t i = 0; i < column_->size(); ++i) + auto & column = block.getByName("____w_end").column; + const auto & data = static_cast(*column).getData(); + for (size_t i = 0; i < column->size(); ++i) { if (data[i] >= w_bound) return false; @@ -433,12 +430,12 @@ inline void StorageWindowView::fire(UInt32 watermark) if (target_table_id.empty()) { in_stream->readPrefix(); - while (auto block_ = in_stream->read()) + while (auto block = in_stream->read()) { for (auto & watch_stream : watch_streams) { - if (auto watch_stream_ = watch_stream.lock()) - watch_stream_->addBlock(block_); + if (auto watch_stream_ptr = watch_stream.lock()) + watch_stream_ptr->addBlock(block); } } in_stream->readSuffix(); @@ -448,18 +445,18 @@ inline void StorageWindowView::fire(UInt32 watermark) try { StoragePtr target_table = getTargetStorage(); - auto _lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); + auto lock = target_table->lockStructureForShare(true, global_context.getCurrentQueryId()); auto out_stream = target_table->write(getInnerQuery(), global_context); in_stream->readPrefix(); out_stream->writePrefix(); - while (auto block_ = in_stream->read()) + while (auto block = in_stream->read()) { for (auto & watch_stream : watch_streams) { - if (auto watch_stream_ = watch_stream.lock()) - watch_stream_->addBlock(block_); + if (const auto & watch_stream_ptr = watch_stream.lock()) + watch_stream_ptr->addBlock(block); } - out_stream->write(std::move(block_)); + out_stream->write(std::move(block)); } in_stream->readSuffix(); out_stream->writeSuffix(); @@ -481,7 +478,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto new_columns_list = std::make_shared(); - auto sample_block_ + auto t_sample_block = InterpreterSelectQuery(getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) .getSampleBlock(); @@ -496,13 +493,13 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery columns_list->children.push_back(column_window); } - for (auto & column_ : sample_block_.getColumnsWithTypeAndName()) + for (auto & column : t_sample_block.getColumnsWithTypeAndName()) { ParserIdentifierWithOptionalParameters parser; - String sql = column_.type->getName(); + String sql = column.type->getName(); ASTPtr ast = parseQuery(parser, sql.data(), sql.data() + sql.size(), "data type", 0); auto column_dec = std::make_shared(); - column_dec->name = column_.name; + column_dec->name = column.name; column_dec->type = ast; columns_list->children.push_back(column_dec); } @@ -568,13 +565,13 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) { - IntervalKind window_kind_; + IntervalKind window_interval_kind; if (is_tumble) - window_kind_ = window_kind; + window_interval_kind = window_kind; else - window_kind_ = hop_kind; + window_interval_kind = hop_kind; - switch (window_kind_) + switch (window_interval_kind) { #define CASE_WINDOW_KIND(KIND) \ case IntervalKind::KIND: \ @@ -687,7 +684,7 @@ void StorageWindowView::threadFuncCleanCache() } } if (!shutdown_called) - cleanCacheTask->scheduleAfter(RESCHEDULE_MS); + clean_cache_task->scheduleAfter(RESCHEDULE_MS); } void StorageWindowView::threadFuncFireProc() @@ -707,7 +704,7 @@ void StorageWindowView::threadFuncFireProc() fire_signal_condition.wait_for(lock, std::chrono::microseconds(static_cast(next_fire_signal) * 1000000 - timestamp_usec)); } if (!shutdown_called) - fireTask->scheduleAfter(RESCHEDULE_MS); + fire_task->scheduleAfter(RESCHEDULE_MS); } void StorageWindowView::threadFuncFireEvent() @@ -726,7 +723,7 @@ void StorageWindowView::threadFuncFireEvent() } } if (!shutdown_called) - fireTask->scheduleAfter(RESCHEDULE_MS); + fire_task->scheduleAfter(RESCHEDULE_MS); } BlockInputStreams StorageWindowView::watch( @@ -778,9 +775,7 @@ StorageWindowView::StorageWindowView( if (query.select->list_of_selects->children.size() != 1) throw Exception("UNION is not supported for Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); - auto inner_query_ = query.select->list_of_selects->children.at(0); - - ASTSelectQuery & select_query = typeid_cast(*inner_query_); + ASTSelectQuery & select_query = typeid_cast(*query.select->list_of_selects->children.at(0)); String select_database_name = local_context.getCurrentDatabase(); String select_table_name; extractDependentTable(select_query, select_database_name, select_table_name); @@ -895,27 +890,27 @@ StorageWindowView::StorageWindowView( { // write expressions - ColumnsWithTypeAndName columns__; - columns__.emplace_back( + ColumnsWithTypeAndName t_columns; + t_columns.emplace_back( nullptr, std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), window_column_name); - columns__.emplace_back(nullptr, std::make_shared(), "____timestamp"); + t_columns.emplace_back(nullptr, std::make_shared(), "____timestamp"); const auto & function_tuple = FunctionFactory::instance().get("tupleElement", global_context); - writeExpressions = std::make_shared(columns__, global_context); - writeExpressions->add(ExpressionAction::addColumn( + write_expressions = std::make_shared(t_columns, global_context); + write_expressions->add(ExpressionAction::addColumn( {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); - writeExpressions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); - writeExpressions->add(ExpressionAction::removeColumn("____tuple_arg")); + write_expressions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); + write_expressions->add(ExpressionAction::removeColumn("____tuple_arg")); } - cleanCacheTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); + clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) - fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); + fire_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); else - fireTask = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); }); - cleanCacheTask->deactivate(); - fireTask->deactivate(); + fire_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); }); + clean_cache_task->deactivate(); + fire_task->deactivate(); } @@ -927,17 +922,17 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parse stage mergeable ASTPtr result = query.clone(); ASTPtr expr_list = result; - StageMergeableVisitorData stageMergeableData; - InDepthNodeVisitor, true>(stageMergeableData).visit(expr_list); - if (!stageMergeableData.is_tumble && !stageMergeableData.is_hop) + StageMergeableVisitorData stage_mergeable_data; + InDepthNodeVisitor, true>(stage_mergeable_data).visit(expr_list); + if (!stage_mergeable_data.is_tumble && !stage_mergeable_data.is_hop) throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); - window_column_name = stageMergeableData.window_column_name; - window_column_alias = stageMergeableData.window_column_alias; - timestamp_column_name = stageMergeableData.timestamp_column_name; - is_tumble = stageMergeableData.is_tumble; + window_column_name = stage_mergeable_data.window_column_name; + window_column_alias = stage_mergeable_data.window_column_alias; + timestamp_column_name = stage_mergeable_data.timestamp_column_name; + is_tumble = stage_mergeable_data.is_tumble; // parser window function - ASTFunction & window_function = typeid_cast(*stageMergeableData.window_function); + ASTFunction & window_function = typeid_cast(*stage_mergeable_data.window_function); const auto & arguments = window_function.arguments->children; const auto & arg1 = std::static_pointer_cast(arguments.at(1)); if (!arg1 || !startsWith(arg1->name, "toInterval")) @@ -980,87 +975,87 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; - source_stream = std::make_shared(source_stream, window_view.writeExpressions); + source_stream = std::make_shared(source_stream, window_view.write_expressions); source_stream = std::make_shared( source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); } else { - UInt32 max_fired_watermark_ = 0; + UInt32 t_max_fired_watermark = 0; if (window_view.allowed_lateness) { - UInt32 max_timestamp_ = 0; - UInt32 max_watermark_ = 0; + UInt32 t_max_timstamp = 0; + UInt32 t_max_watermark = 0; { std::lock_guard lock(window_view.fire_signal_mutex); - max_fired_watermark_ = window_view.max_fired_watermark; - max_watermark_ = window_view.max_watermark; - max_timestamp_ = window_view.max_timestamp; + t_max_fired_watermark = window_view.max_fired_watermark; + t_max_watermark = window_view.max_watermark; + t_max_timstamp = window_view.max_timestamp; } - if (max_timestamp_!= 0) + if (t_max_timstamp!= 0) { UInt32 lateness_bound - = addTime(max_timestamp_, window_view.lateness_kind, -1 * window_view.lateness_num_units, window_view.time_zone); + = addTime(t_max_timstamp, window_view.lateness_kind, -1 * window_view.lateness_num_units, window_view.time_zone); if (window_view.is_watermark_bounded) { UInt32 watermark_lower_bound = window_view.is_tumble - ? addTime(max_watermark_, window_view.window_kind, -1 * window_view.window_num_units, window_view.time_zone) - : addTime(max_watermark_, window_view.hop_kind, -1 * window_view.hop_num_units, window_view.time_zone); + ? addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units, window_view.time_zone) + : addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units, window_view.time_zone); if (watermark_lower_bound < lateness_bound) lateness_bound = watermark_lower_bound; } - ColumnsWithTypeAndName columns__; - columns__.emplace_back(nullptr, std::make_shared(), window_view.timestamp_column_name); - ExpressionActionsPtr filterExpressions = std::make_shared(columns__, context); - filterExpressions->add( + ColumnsWithTypeAndName columns; + columns.emplace_back(nullptr, std::make_shared(), window_view.timestamp_column_name); + ExpressionActionsPtr filter_expressions = std::make_shared(columns, context); + filter_expressions->add( ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(lateness_bound)), std::make_shared(), "____lateness_bound"})); const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); - filterExpressions->add(ExpressionAction::applyFunction( + filter_expressions->add(ExpressionAction::applyFunction( function_greater, Names{window_view.timestamp_column_name, "____lateness_bound"}, "____filter")); - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), filterExpressions, "____filter", true)); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), filter_expressions, "____filter", true)); } } - UInt32 max_timestamp__ = 0; + UInt32 t_max_timstamp = 0; if (!window_view.is_tumble || window_view.is_watermark_bounded || window_view.allowed_lateness) { auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); - for (auto& timestamp_ : timestamp_data) + for (auto& timestamp : timestamp_data) { - if (timestamp_ > max_timestamp__) - max_timestamp__ = timestamp_; + if (timestamp > t_max_timstamp) + t_max_timstamp = timestamp; } } InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; - source_stream = std::make_shared(source_stream, window_view.writeExpressions); + source_stream = std::make_shared(source_stream, window_view.write_expressions); source_stream = std::make_shared( source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); if (!window_view.is_tumble) source_stream - = std::make_shared(source_stream, window_view, window_view.getWindowUpperBound(max_timestamp__)); + = std::make_shared(source_stream, window_view, window_view.getWindowUpperBound(t_max_timstamp)); else source_stream = std::make_shared(source_stream, window_view); if (window_view.is_watermark_bounded || window_view.allowed_lateness) - std::static_pointer_cast(source_stream)->setMaxTimestamp(max_timestamp__); + std::static_pointer_cast(source_stream)->setMaxTimestamp(t_max_timstamp); - if (window_view.allowed_lateness && max_fired_watermark_ != 0) - std::static_pointer_cast(source_stream)->setAllowedLateness(max_fired_watermark_); + if (window_view.allowed_lateness && t_max_fired_watermark != 0) + std::static_pointer_cast(source_stream)->setAllowedLateness(t_max_fired_watermark); } if (!window_view.inner_table_id.empty()) { auto & inner_storage = window_view.getInnerStorage(); - auto lock_ = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); + auto lock = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); auto stream = inner_storage->write(window_view.getInnerQuery(), context); copyData(*source_stream, *stream); } @@ -1069,8 +1064,8 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con source_stream->readPrefix(); { std::lock_guard lock(window_view.mutex); - while (Block block_ = source_stream->read()) - window_view.mergeable_blocks.push_back(std::move(block_)); + while (Block t_block = source_stream->read()) + window_view.mergeable_blocks.push_back(std::move(t_block)); } source_stream->readSuffix(); } @@ -1079,8 +1074,8 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con void StorageWindowView::startup() { // Start the working thread - cleanCacheTask->activateAndSchedule(); - fireTask->activateAndSchedule(); + clean_cache_task->activateAndSchedule(); + fire_task->activateAndSchedule(); } void StorageWindowView::shutdown() @@ -1088,8 +1083,8 @@ void StorageWindowView::shutdown() bool expected = false; if (!shutdown_called.compare_exchange_strong(expected, true)) return; - cleanCacheTask->deactivate(); - fireTask->deactivate(); + clean_cache_task->deactivate(); + fire_task->deactivate(); } StorageWindowView::~StorageWindowView() @@ -1159,21 +1154,21 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma pipes = blocksToPipes(mergeable_blocks, getMergeableHeader()); } - ColumnsWithTypeAndName columns_; - columns_.emplace_back(nullptr, std::make_shared(), "____w_end"); + ColumnsWithTypeAndName t_columns; + t_columns.emplace_back(nullptr, std::make_shared(), "____w_end"); - ExpressionActionsPtr actions_ = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), + ExpressionActionsPtr actions = std::make_shared(t_columns, global_context); + actions->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), std::make_shared(), "____watermark"})); const auto & function_equals = FunctionFactory::instance().get("equals", global_context); - ExpressionActionsPtr apply_function_actions = std::make_shared(columns_, global_context); - actions_->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____watermark"}, "____filter")); - actions_->add(ExpressionAction::removeColumn("____w_end")); - actions_->add(ExpressionAction::removeColumn("____watermark")); + ExpressionActionsPtr apply_function_actions = std::make_shared(t_columns, global_context); + actions->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____watermark"}, "____filter")); + actions->add(ExpressionAction::removeColumn("____w_end")); + actions->add(ExpressionAction::removeColumn("____watermark")); for (auto & pipe : pipes) - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), actions_, + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), actions, "____filter", true)); auto proxy_storage = std::make_shared( diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 87460d5280d..9783fa76a5c 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -105,10 +105,10 @@ private: mutable StoragePtr inner_storage; mutable StoragePtr target_storage; - BackgroundSchedulePool::TaskHolder cleanCacheTask; - BackgroundSchedulePool::TaskHolder fireTask; + BackgroundSchedulePool::TaskHolder clean_cache_task; + BackgroundSchedulePool::TaskHolder fire_task; - ExpressionActionsPtr writeExpressions; + ExpressionActionsPtr write_expressions; ASTPtr innerQueryParser(ASTSelectQuery & inner_query); From 427d982d0c0a7a6509fa6b374a2da4e7047c8300 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 26 Mar 2020 01:56:49 +0800 Subject: [PATCH 040/609] remove ____w_end marker --- .../Storages/WindowView/StorageWindowView.cpp | 87 ++++++++----------- .../Storages/WindowView/StorageWindowView.h | 3 +- .../WindowView/WatermarkBlockInputStream.h | 13 ++- 3 files changed, 49 insertions(+), 54 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 7a34a30b9c8..8470c262d28 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -220,26 +220,6 @@ namespace String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } - ASTPtr generateDeleteRetiredQuery(StorageID inner_table_id, UInt32 timestamp) - { - auto function_equal = makeASTFunction( - "less", std::make_shared("____w_end"), std::make_shared(timestamp)); - - auto alter_command = std::make_shared(); - alter_command->type = ASTAlterCommand::DELETE; - alter_command->predicate = function_equal; - alter_command->children.push_back(alter_command->predicate); - - auto alter_command_list = std::make_shared(); - alter_command_list->add(alter_command); - - auto alter_query = std::make_shared(); - alter_query->database = inner_table_id.database_name; - alter_query->table = inner_table_id.table_name; - alter_query->set(alter_query->command_list, alter_command_list); - return alter_query; - } - std::shared_ptr generateFetchColumnsQuery(const StorageID & inner_storage) { auto res_query = std::make_shared(); @@ -297,6 +277,27 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa DB::ErrorCodes::LOGICAL_ERROR); } +ASTPtr StorageWindowView::generateCleanCacheQuery(UInt32 timestamp) +{ + auto function_tuple + = makeASTFunction("tupleElement", std::make_shared(window_column_name), std::make_shared("2")); + auto function_equal = makeASTFunction("less", function_tuple, std::make_shared(timestamp)); + + auto alter_command = std::make_shared(); + alter_command->type = ASTAlterCommand::DELETE; + alter_command->predicate = function_equal; + alter_command->children.push_back(alter_command->predicate); + + auto alter_command_list = std::make_shared(); + alter_command_list->add(alter_command); + + auto alter_query = std::make_shared(); + alter_query->database = inner_table_id.database_name; + alter_query->table = inner_table_id.table_name; + alter_query->set(alter_query->command_list, alter_command_list); + return alter_query; +} + void StorageWindowView::checkTableCanBeDropped() const { auto table_id = getStorageID(); @@ -392,17 +393,18 @@ inline void StorageWindowView::cleanCache() if (!inner_table_id.empty()) { - auto sql = generateDeleteRetiredQuery(inner_table_id, w_bound); + auto sql = generateCleanCacheQuery(w_bound); InterpreterAlterQuery alt_query(sql, global_context); alt_query.execute(); } else { std::lock_guard lock(mutex); - mergeable_blocks.remove_if([w_bound](Block & block) + mergeable_blocks.remove_if([&](Block & block) { - auto & column = block.getByName("____w_end").column; - const auto & data = static_cast(*column).getData(); + auto & column = block.getByName(window_column_name).column; + const ColumnTuple & column_tuple = typeid_cast(*column); + const ColumnUInt32::Container & data = static_cast(*column_tuple.getColumnPtr(1)).getData(); for (size_t i = 0; i < column->size(); ++i) { if (data[i] >= w_bound) @@ -503,10 +505,6 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery column_dec->type = ast; columns_list->children.push_back(column_dec); } - auto column_wend = std::make_shared(); - column_wend->name = "____w_end"; - column_wend->type = std::make_shared("DateTime"); - columns_list->children.push_back(column_wend); if (inner_create_query.storage->ttl_table) throw Exception("TTL is not supported for inner table in Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); @@ -888,22 +886,6 @@ StorageWindowView::StorageWindowView( fetch_column_query = generateFetchColumnsQuery(inner_table_id); } - { - // write expressions - ColumnsWithTypeAndName t_columns; - t_columns.emplace_back( - nullptr, - std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), - window_column_name); - t_columns.emplace_back(nullptr, std::make_shared(), "____timestamp"); - const auto & function_tuple = FunctionFactory::instance().get("tupleElement", global_context); - write_expressions = std::make_shared(t_columns, global_context); - write_expressions->add(ExpressionAction::addColumn( - {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); - write_expressions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); - write_expressions->add(ExpressionAction::removeColumn("____tuple_arg")); - } - clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) fire_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); @@ -975,7 +957,6 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; - source_stream = std::make_shared(source_stream, window_view.write_expressions); source_stream = std::make_shared( source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); } @@ -1035,15 +1016,14 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; - source_stream = std::make_shared(source_stream, window_view.write_expressions); source_stream = std::make_shared( source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); if (!window_view.is_tumble) source_stream - = std::make_shared(source_stream, window_view, window_view.getWindowUpperBound(t_max_timstamp)); + = std::make_shared(source_stream, window_view, window_view.window_column_name, window_view.getWindowUpperBound(t_max_timstamp)); else - source_stream = std::make_shared(source_stream, window_view); + source_stream = std::make_shared(source_stream, window_view, window_view.window_column_name); if (window_view.is_watermark_bounded || window_view.allowed_lateness) std::static_pointer_cast(source_stream)->setMaxTimestamp(t_max_timstamp); @@ -1155,9 +1135,18 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma } ColumnsWithTypeAndName t_columns; - t_columns.emplace_back(nullptr, std::make_shared(), "____w_end"); + t_columns.emplace_back( + nullptr, + std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), + window_column_name); ExpressionActionsPtr actions = std::make_shared(t_columns, global_context); + const auto & function_tuple = FunctionFactory::instance().get("tupleElement", global_context); + actions->add(ExpressionAction::addColumn( + {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); + actions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); + actions->add(ExpressionAction::removeColumn("____tuple_arg")); + actions->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), std::make_shared(), "____watermark"})); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 9783fa76a5c..4597b26fe6f 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -108,11 +108,10 @@ private: BackgroundSchedulePool::TaskHolder clean_cache_task; BackgroundSchedulePool::TaskHolder fire_task; - ExpressionActionsPtr write_expressions; - ASTPtr innerQueryParser(ASTSelectQuery & inner_query); std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); + ASTPtr generateCleanCacheQuery(UInt32 timestamp); UInt32 getWindowLowerBound(UInt32 time_sec); UInt32 getWindowUpperBound(UInt32 time_sec); diff --git a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h index 0b43582a532..3346a8bb493 100644 --- a/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/dbms/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -13,11 +14,13 @@ class WatermarkBlockInputStream : public IBlockInputStream public: WatermarkBlockInputStream( BlockInputStreamPtr input_, - StorageWindowView& storage_) + StorageWindowView& storage_, + String& window_column_name_) : allowed_lateness(false) , update_timestamp(false) , watermark_specified(false) , storage(storage_) + , window_column_name(window_column_name_) , lateness_upper_bound(0) , max_timestamp(0) , max_watermark(0) @@ -28,11 +31,13 @@ public: WatermarkBlockInputStream( BlockInputStreamPtr input_, StorageWindowView& storage_, + String& window_column_name_, UInt32 max_watermark_) : allowed_lateness(false) , update_timestamp(false) , watermark_specified(true) , storage(storage_) + , window_column_name(window_column_name_) , lateness_upper_bound(0) , max_timestamp(0) , max_watermark(max_watermark_) @@ -66,8 +71,9 @@ protected: if (!res) return res; - auto & column_wend = res.getByName("____w_end").column; - const ColumnUInt32::Container & wend_data = static_cast(*column_wend).getData(); + auto & column_window = res.getByName(window_column_name).column; + const ColumnTuple & column_tuple = typeid_cast(*column_window); + const ColumnUInt32::Container & wend_data = static_cast(*column_tuple.getColumnPtr(1)).getData(); for (size_t i = 0; i < wend_data.size(); ++i) { if (!watermark_specified && wend_data[i] > max_watermark) @@ -94,6 +100,7 @@ private: bool watermark_specified; std::set late_signals; StorageWindowView & storage; + String window_column_name; UInt32 lateness_upper_bound; UInt32 max_timestamp; UInt32 max_watermark; From 17d259d43d29e62f60cdc587825af6a197849beb Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 27 Mar 2020 18:13:52 +0800 Subject: [PATCH 041/609] remove duplicated lines in tests --- .../queries/0_stateless/01053_window_view_proc_now_tumble_to.sql | 1 - .../0_stateless/01054_window_view_proc_now_tumble_inner_to.sql | 1 - .../queries/0_stateless/01055_window_view_proc_now_hop_to.sql | 1 - .../0_stateless/01056_window_view_proc_now_hop_inner_to.sql | 1 - .../queries/0_stateless/01057_window_view_proc_tumble_to.sql | 1 - .../0_stateless/01058_window_view_proc_tumble_inner_to.sql | 1 - dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql | 1 - .../queries/0_stateless/01060_window_view_proc_hop_inner_to.sql | 1 - .../0_stateless/01062_window_view_event_strict_asc_tumble_to.sql | 1 - .../01063_window_view_event_strict_asc_tumble_inner_to.sql | 1 - .../0_stateless/01064_window_view_event_strict_asc_hop_to.sql | 1 - .../01065_window_view_event_strict_asc_hop_inner_to.sql | 1 - .../0_stateless/01067_window_view_event_asc_tumble_to.sql | 1 - .../0_stateless/01068_window_view_event_asc_tumble_inner_to.sql | 1 - .../queries/0_stateless/01069_window_view_event_asc_hop_to.sql | 1 - .../0_stateless/01070_window_view_event_asc_hop_inner_to.sql | 1 - .../0_stateless/01072_window_view_event_bounded_tumble_to.sql | 1 - .../01073_window_view_event_bounded_tumble_inner_to.sql | 1 - .../0_stateless/01074_window_view_event_bounded_hop_to.sql | 1 - .../0_stateless/01075_window_view_event_bounded_hop_inner_to.sql | 1 - .../01077_window_view_event_strict_asc_tumble_to_lateness.sql | 1 - .../01078_window_view_event_asc_tumble_to_lateness.sql | 1 - .../01079_window_view_event_bounded_tumble_to_lateness.sql | 1 - 23 files changed, 23 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql index 4ebda16f5cf..8bc5dfb5c04 100755 --- a/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql index a184d43ebbc..8e78a74c95c 100755 --- a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql index 1c4b0cdbc26..b14db138406 100755 --- a/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql index 89c78784ab3..ad93163870d 100755 --- a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql index c64e5c49c86..ef84ee95079 100755 --- a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql index f4226f24cec..fa3600de95a 100755 --- a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql index 51c819dac29..acf8e8dc5c6 100755 --- a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql index 0a6a4fff2cb..4d802af103a 100755 --- a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql index 6a675d26df9..df10f8b892f 100644 --- a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql index f4ad0b6dcd6..bdad1d5a4b2 100644 --- a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql index eedc24d19fc..24b35f6b3b3 100644 --- a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql index 34502703f14..0be01e39769 100644 --- a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql index 16c9b9d6e10..4be8b453df6 100644 --- a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql index e50b94dd269..40f2634d0c6 100644 --- a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql index f6c0e036868..8b20610c94c 100644 --- a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql index e889761f3dd..b4e8cfdf6bb 100644 --- a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql index 8cddd87d8f4..69138a7be89 100644 --- a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql +++ b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql index ef84d472b1e..e8288077793 100644 --- a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql index b1f90a66765..337f2bb6c1b 100644 --- a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql +++ b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql index 1f2179182e1..98b12f6c4c2 100644 --- a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql +++ b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql index 2882be59ef4..4bbee2f9c80 100644 --- a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql +++ b/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql index 31eaf397e47..5351baf24f5 100644 --- a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql +++ b/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; diff --git a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql index 96c1d2d0099..3d3a7c37beb 100644 --- a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql +++ b/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql @@ -1,5 +1,4 @@ SET allow_experimental_window_view = 1; -SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; From 220c29600d7b026a505a8914297a91544b274afc Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 27 Mar 2020 23:46:29 +0800 Subject: [PATCH 042/609] rename test name --- ...ons.reference => 01049_window_view_window_functions.reference} | 0 ...indow_functions.sql => 01049_window_view_window_functions.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{01049_window_functions.reference => 01049_window_view_window_functions.reference} (100%) rename dbms/tests/queries/0_stateless/{01049_window_functions.sql => 01049_window_view_window_functions.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.reference b/dbms/tests/queries/0_stateless/01049_window_view_window_functions.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01049_window_functions.reference rename to dbms/tests/queries/0_stateless/01049_window_view_window_functions.reference diff --git a/dbms/tests/queries/0_stateless/01049_window_functions.sql b/dbms/tests/queries/0_stateless/01049_window_view_window_functions.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01049_window_functions.sql rename to dbms/tests/queries/0_stateless/01049_window_view_window_functions.sql From 5a23306ebba57dc708a9e8773effdef9efbf5648 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 28 Mar 2020 00:44:09 +0800 Subject: [PATCH 043/609] use AggregatingMergeTree as default inner table. --- .../Storages/WindowView/StorageWindowView.cpp | 288 +++++++++--------- .../Storages/WindowView/StorageWindowView.h | 6 +- ...7_window_view_inner_table_parser.reference | 22 ++ .../01047_window_view_inner_table_parser.sql | 61 ++++ ...ndow_view_inner_table_generation.reference | 22 ++ ...048_window_view_inner_table_generation.sql | 68 +++++ ..._window_view_proc_now_tumble_to.reference} | 0 ... 01052_window_view_proc_now_tumble_to.sql} | 0 ...053_window_view_proc_now_hop_to.reference} | 0 ... => 01053_window_view_proc_now_hop_to.sql} | 0 ...4_window_view_proc_now_tumble_inner_to.sql | 17 -- ...1054_window_view_proc_tumble_to.reference} | 0 ...l => 01054_window_view_proc_tumble_to.sql} | 0 ...> 01055_window_view_proc_hop_to.reference} | 0 ....sql => 01055_window_view_proc_hop_to.sql} | 0 ...py => 01056_window_view_proc_hop_watch.py} | 0 ...1056_window_view_proc_hop_watch.reference} | 0 ...1056_window_view_proc_now_hop_inner_to.sql | 17 -- ...view_event_strict_asc_tumble_to.reference} | 0 ...indow_view_event_strict_asc_tumble_to.sql} | 0 ...01057_window_view_proc_tumble_to.reference | 2 - ...ow_view_event_strict_asc_hop_to.reference} | 0 ...8_window_view_event_strict_asc_hop_to.sql} | 0 ...window_view_proc_tumble_inner_to.reference | 2 - ...01058_window_view_proc_tumble_inner_to.sql | 17 -- ...window_view_event_strict_asc_hop_watch.py} | 0 ...view_event_strict_asc_hop_watch.reference} | 0 .../01059_window_view_proc_hop_to.reference | 2 - ...window_view_event_asc_tumble_to.reference} | 0 ...01060_window_view_event_asc_tumble_to.sql} | 0 ...60_window_view_proc_hop_inner_to.reference | 2 - .../01060_window_view_proc_hop_inner_to.sql | 17 -- ...61_window_view_event_asc_hop_to.reference} | 0 ...=> 01061_window_view_event_asc_hop_to.sql} | 0 ... 01062_window_view_event_asc_hop_watch.py} | 0 ...window_view_event_asc_hop_watch.reference} | 0 ...ow_view_event_bounded_tumble_to.reference} | 0 ...3_window_view_event_bounded_tumble_to.sql} | 0 ...event_strict_asc_tumble_inner_to.reference | 4 - ..._view_event_strict_asc_tumble_inner_to.sql | 24 -- ...indow_view_event_bounded_hop_to.reference} | 0 ...1064_window_view_event_bounded_hop_to.sql} | 0 ...65_window_view_event_bounded_hop_watch.py} | 0 ...ow_view_event_bounded_hop_watch.reference} | 0 ...ew_event_strict_asc_hop_inner_to.reference | 7 - ...dow_view_event_strict_asc_hop_inner_to.sql | 24 -- ...t_strict_asc_tumble_to_lateness.reference} | 0 ...w_event_strict_asc_tumble_to_lateness.sql} | 0 ...ew_event_asc_tumble_to_lateness.reference} | 0 ...dow_view_event_asc_tumble_to_lateness.sql} | 0 ..._window_view_event_asc_tumble_inner_to.sql | 25 -- ...vent_bounded_tumble_to_lateness.reference} | 0 ...view_event_bounded_tumble_to_lateness.sql} | 0 ...070_window_view_event_asc_hop_inner_to.sql | 25 -- ...dow_view_event_bounded_tumble_to.reference | 4 - ...ew_event_bounded_tumble_inner_to.reference | 4 - ...dow_view_event_bounded_tumble_inner_to.sql | 25 -- ...window_view_event_bounded_hop_to.reference | 7 - ..._view_event_bounded_hop_inner_to.reference | 7 - ...window_view_event_bounded_hop_inner_to.sql | 25 -- 60 files changed, 310 insertions(+), 414 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference create mode 100644 dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql create mode 100644 dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference create mode 100644 dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.sql rename dbms/tests/queries/0_stateless/{01053_window_view_proc_now_tumble_to.reference => 01052_window_view_proc_now_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01053_window_view_proc_now_tumble_to.sql => 01052_window_view_proc_now_tumble_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01054_window_view_proc_now_tumble_inner_to.reference => 01053_window_view_proc_now_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01055_window_view_proc_now_hop_to.sql => 01053_window_view_proc_now_hop_to.sql} (100%) delete mode 100755 dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql rename dbms/tests/queries/0_stateless/{01055_window_view_proc_now_hop_to.reference => 01054_window_view_proc_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01057_window_view_proc_tumble_to.sql => 01054_window_view_proc_tumble_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01056_window_view_proc_now_hop_inner_to.reference => 01055_window_view_proc_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01059_window_view_proc_hop_to.sql => 01055_window_view_proc_hop_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01061_window_view_proc_hop_watch.py => 01056_window_view_proc_hop_watch.py} (100%) rename dbms/tests/queries/0_stateless/{01061_window_view_proc_hop_watch.reference => 01056_window_view_proc_hop_watch.reference} (100%) delete mode 100755 dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01062_window_view_event_strict_asc_tumble_to.reference => 01057_window_view_event_strict_asc_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01062_window_view_event_strict_asc_tumble_to.sql => 01057_window_view_event_strict_asc_tumble_to.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.reference rename dbms/tests/queries/0_stateless/{01064_window_view_event_strict_asc_hop_to.reference => 01058_window_view_event_strict_asc_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01064_window_view_event_strict_asc_hop_to.sql => 01058_window_view_event_strict_asc_hop_to.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.reference delete mode 100755 dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql rename dbms/tests/queries/0_stateless/{01066_window_view_event_strict_asc_hop_watch.py => 01059_window_view_event_strict_asc_hop_watch.py} (100%) rename dbms/tests/queries/0_stateless/{01066_window_view_event_strict_asc_hop_watch.reference => 01059_window_view_event_strict_asc_hop_watch.reference} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.reference rename dbms/tests/queries/0_stateless/{01067_window_view_event_asc_tumble_to.reference => 01060_window_view_event_asc_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01067_window_view_event_asc_tumble_to.sql => 01060_window_view_event_asc_tumble_to.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.reference delete mode 100755 dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01069_window_view_event_asc_hop_to.reference => 01061_window_view_event_asc_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01069_window_view_event_asc_hop_to.sql => 01061_window_view_event_asc_hop_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01071_window_view_event_asc_hop_watch.py => 01062_window_view_event_asc_hop_watch.py} (100%) rename dbms/tests/queries/0_stateless/{01071_window_view_event_asc_hop_watch.reference => 01062_window_view_event_asc_hop_watch.reference} (100%) rename dbms/tests/queries/0_stateless/{01068_window_view_event_asc_tumble_inner_to.reference => 01063_window_view_event_bounded_tumble_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01072_window_view_event_bounded_tumble_to.sql => 01063_window_view_event_bounded_tumble_to.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql rename dbms/tests/queries/0_stateless/{01070_window_view_event_asc_hop_inner_to.reference => 01064_window_view_event_bounded_hop_to.reference} (100%) rename dbms/tests/queries/0_stateless/{01074_window_view_event_bounded_hop_to.sql => 01064_window_view_event_bounded_hop_to.sql} (100%) rename dbms/tests/queries/0_stateless/{01076_window_view_event_bounded_hop_watch.py => 01065_window_view_event_bounded_hop_watch.py} (100%) rename dbms/tests/queries/0_stateless/{01076_window_view_event_bounded_hop_watch.reference => 01065_window_view_event_bounded_hop_watch.reference} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql rename dbms/tests/queries/0_stateless/{01077_window_view_event_strict_asc_tumble_to_lateness.reference => 01066_window_view_event_strict_asc_tumble_to_lateness.reference} (100%) rename dbms/tests/queries/0_stateless/{01077_window_view_event_strict_asc_tumble_to_lateness.sql => 01066_window_view_event_strict_asc_tumble_to_lateness.sql} (100%) rename dbms/tests/queries/0_stateless/{01078_window_view_event_asc_tumble_to_lateness.reference => 01067_window_view_event_asc_tumble_to_lateness.reference} (100%) rename dbms/tests/queries/0_stateless/{01078_window_view_event_asc_tumble_to_lateness.sql => 01067_window_view_event_asc_tumble_to_lateness.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql rename dbms/tests/queries/0_stateless/{01079_window_view_event_bounded_tumble_to_lateness.reference => 01068_window_view_event_bounded_tumble_to_lateness.reference} (100%) rename dbms/tests/queries/0_stateless/{01079_window_view_event_bounded_tumble_to_lateness.sql => 01068_window_view_event_bounded_tumble_to_lateness.sql} (100%) delete mode 100644 dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql delete mode 100644 dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference delete mode 100644 dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 8470c262d28..62fbe5349c5 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include @@ -146,6 +147,7 @@ namespace { String window_column_name; String window_column_alias; + Aliases * aliases; }; static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } @@ -159,19 +161,22 @@ namespace } private: - static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data & data) + static void visit(const ASTFunction & node, ASTPtr & node_ptr, Data &) { - if (node.name == "TUMBLE" || node.name == "HOP") - { - if (queryToString(node) == data.window_column_name) - node_ptr = std::make_shared(data.window_column_name); - } + if (node.name == "tuple") + return; + if (node.name == "HOP") + node_ptr = std::make_shared("arrayJoin(" + node.getColumnName() + ")"); + else + node_ptr = std::make_shared(node.getColumnName()); } static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data) { - if (node.name == data.window_column_alias) - node_ptr = std::make_shared(data.window_column_name); + if (node.getColumnName() == data.window_column_alias) + dynamic_cast(node_ptr.get())->name = data.window_column_name; + else if (auto it = data.aliases->find(node.getColumnName()); it != data.aliases->end()) + dynamic_cast(node_ptr.get())->name = it->second->getColumnName(); } }; @@ -327,9 +332,7 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &) { auto table_id = getStorageID(); DatabaseCatalog::instance().removeDependency(select_table_id, table_id); - - if (!inner_table_id.empty()) - executeDropQuery(ASTDropQuery::Kind::Drop, global_context, inner_table_id); + executeDropQuery(ASTDropQuery::Kind::Drop, global_context, inner_table_id); std::lock_guard lock(mutex); is_dropped = true; @@ -338,20 +341,11 @@ void StorageWindowView::drop(TableStructureWriteLockHolder &) void StorageWindowView::truncate(const ASTPtr &, const Context &, TableStructureWriteLockHolder &) { - if (!inner_table_id.empty()) - executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, inner_table_id); - else - { - std::lock_guard lock(mutex); - mergeable_blocks.clear(); - } + executeDropQuery(ASTDropQuery::Kind::Truncate, global_context, inner_table_id); } bool StorageWindowView::optimize(const ASTPtr & query, const ASTPtr & partition, bool final, bool deduplicate, const Context & context) { - if (inner_table_id.empty()) - throw Exception( - "OPTIMIZE only supported when creating WINDOW VIEW within INNER table.", ErrorCodes::INCORRECT_QUERY); return getInnerStorage()->optimize(query, partition, final, deduplicate, context); } @@ -391,28 +385,9 @@ inline void StorageWindowView::cleanCache() w_bound = is_tumble ? addTime(w_bound, window_kind, -1 * window_num_units, time_zone) : addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone); - if (!inner_table_id.empty()) - { - auto sql = generateCleanCacheQuery(w_bound); - InterpreterAlterQuery alt_query(sql, global_context); - alt_query.execute(); - } - else - { - std::lock_guard lock(mutex); - mergeable_blocks.remove_if([&](Block & block) - { - auto & column = block.getByName(window_column_name).column; - const ColumnTuple & column_tuple = typeid_cast(*column); - const ColumnUInt32::Container & data = static_cast(*column_tuple.getColumnPtr(1)).getData(); - for (size_t i = 0; i < column->size(); ++i) - { - if (data[i] >= w_bound) - return false; - } - return true; - }); - } + auto sql = generateCleanCacheQuery(w_bound); + InterpreterAlterQuery alt_query(sql, global_context); + alt_query.execute(); std::lock_guard lock(fire_signal_mutex); watch_streams.remove_if([](std::weak_ptr & ptr) { return ptr.expired(); }); @@ -471,17 +446,21 @@ inline void StorageWindowView::fire(UInt32 watermark) fire_condition.notify_all(); } -std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name) +std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name) { /// We will create a query to create an internal table. - auto manual_create_query = std::make_shared(); - manual_create_query->database = database_name; - manual_create_query->table = table_name; + auto inner_create_query = std::make_shared(); + inner_create_query->database = database_name; + inner_create_query->table = table_name; - auto new_columns_list = std::make_shared(); + auto inner_select_query = dynamic_pointer_cast(getInnerQuery()); + + Aliases aliases; + QueryAliasesVisitor::Data query_aliases_data{aliases}; + QueryAliasesVisitor(query_aliases_data).visit(inner_select_query); auto t_sample_block - = InterpreterSelectQuery(getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) + = InterpreterSelectQuery(inner_select_query, global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) .getSampleBlock(); auto columns_list = std::make_shared(); @@ -506,59 +485,100 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery columns_list->children.push_back(column_dec); } - if (inner_create_query.storage->ttl_table) - throw Exception("TTL is not supported for inner table in Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); - ReplaceFunctionWindowMatcher::Data query_data; query_data.window_column_name = window_column_name; query_data.window_column_alias = window_column_alias; + query_data.aliases = &aliases; ReplaceFunctionWindowMatcher::Visitor visitor(query_data); ReplaceFuncNowVisitorData parser_proc_time_data; InDepthNodeVisitor, true> time_now_visitor(parser_proc_time_data); - auto storage = std::make_shared(); - storage->set(storage->engine, inner_create_query.storage->engine->clone()); - if (inner_create_query.storage->partition_by) + auto new_storage = std::make_shared(); + if (storage == nullptr) { - auto partition_by = inner_create_query.storage->partition_by->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(partition_by); - visitor.visit(partition_by); - storage->set(storage->partition_by, partition_by); - } - if (inner_create_query.storage->primary_key) - { - auto primary_key = inner_create_query.storage->primary_key->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(primary_key); - visitor.visit(primary_key); - storage->set(storage->primary_key, primary_key); - } - if (inner_create_query.storage->order_by) - { - auto order_by = inner_create_query.storage->order_by->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(order_by); - visitor.visit(order_by); - storage->set(storage->order_by, order_by); - } - if (inner_create_query.storage->sample_by) - { - auto sample_by = inner_create_query.storage->sample_by->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(sample_by); - visitor.visit(sample_by); - storage->set(storage->sample_by, sample_by); - } - if (inner_create_query.storage->settings) - storage->set(storage->settings, inner_create_query.storage->settings->clone()); + new_storage->set(new_storage->engine, makeASTFunction("AggregatingMergeTree")); - new_columns_list->set(new_columns_list->columns, columns_list); - manual_create_query->set(manual_create_query->columns_list, new_columns_list); - manual_create_query->set(manual_create_query->storage, storage); + // auto inner_select_query = std::dynamic_pointer_cast(getInnerQuery()); + for (auto & child : inner_select_query->groupBy()->children) + if (auto * ast_with_alias = dynamic_cast(child.get())) + ast_with_alias->setAlias(""); - return manual_create_query; + auto order_by = std::make_shared(); + order_by->name = "tuple"; + order_by->arguments = inner_select_query->groupBy(); + order_by->children.push_back(order_by->arguments); + + ASTPtr order_by_ptr = order_by; + if (is_time_column_func_now) + time_now_visitor.visit(order_by_ptr); + visitor.visit(order_by_ptr); + + for (auto & child : order_by->arguments->children) + { + if (child->getColumnName() == window_column_name) + { + ASTPtr tmp = child; + child = order_by->arguments->children[0]; + order_by->arguments->children[0] = tmp; + break; + } + } + new_storage->set(new_storage->order_by, order_by_ptr); + new_storage->set(new_storage->primary_key, std::make_shared(window_column_name)); + } + else + { + if (storage->ttl_table) + throw Exception("TTL is not supported for inner table in Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + if (!endsWith(storage->engine->name, "MergeTree")) + throw Exception( + "The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support", + ErrorCodes::INCORRECT_QUERY); + + new_storage->set(new_storage->engine, storage->engine->clone()); + if (storage->partition_by) + { + auto partition_by = storage->partition_by->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(partition_by); + visitor.visit(partition_by); + new_storage->set(new_storage->partition_by, partition_by); + } + if (storage->primary_key) + { + auto primary_key = storage->primary_key->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(primary_key); + visitor.visit(primary_key); + new_storage->set(new_storage->primary_key, primary_key); + } + if (storage->order_by) + { + auto order_by = storage->order_by->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(order_by); + visitor.visit(order_by); + new_storage->set(new_storage->order_by, order_by); + } + if (storage->sample_by) + { + auto sample_by = storage->sample_by->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(sample_by); + visitor.visit(sample_by); + new_storage->set(new_storage->sample_by, sample_by); + } + if (storage->settings) + new_storage->set(new_storage->settings, storage->settings->clone()); + } + + auto new_columns = std::make_shared(); + new_columns->set(new_columns->columns, columns_list); + inner_create_query->set(inner_create_query->columns_list, new_columns); + inner_create_query->set(inner_create_query->storage, new_storage); + + return inner_create_query; } inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) @@ -862,29 +882,22 @@ StorageWindowView::StorageWindowView( throw Exception("Value for ALLOWED_LATENESS function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } - if (query.storage) + if (attach_) { - if (attach_) - { - inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); - } - else - { - if (!endsWith(query.storage->engine->name, "MergeTree")) - throw Exception( - "The ENGINE of WindowView must be MergeTree family of table engines including the engines with replication support", - ErrorCodes::INCORRECT_QUERY); - - auto manual_create_query - = generateInnerTableCreateQuery(query, table_id_.database_name, generateInnerTableName(table_id_.table_name)); - InterpreterCreateQuery create_interpreter(manual_create_query, local_context); - create_interpreter.setInternal(true); - create_interpreter.execute(); - inner_storage = DatabaseCatalog::instance().getTable(StorageID(manual_create_query->database, manual_create_query->table)); - inner_table_id = inner_storage->getStorageID(); - } - fetch_column_query = generateFetchColumnsQuery(inner_table_id); + inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); } + else + { + auto inner_create_query + = generateInnerTableCreateQuery(query.storage, table_id_.database_name, generateInnerTableName(table_id_.table_name)); + + InterpreterCreateQuery create_interpreter(inner_create_query, local_context); + create_interpreter.setInternal(true); + create_interpreter.execute(); + inner_storage = DatabaseCatalog::instance().getTable(StorageID(inner_create_query->database, inner_create_query->table)); + inner_table_id = inner_storage->getStorageID(); + } + fetch_column_query = generateFetchColumnsQuery(inner_table_id); clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) @@ -1032,23 +1045,10 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con std::static_pointer_cast(source_stream)->setAllowedLateness(t_max_fired_watermark); } - if (!window_view.inner_table_id.empty()) - { - auto & inner_storage = window_view.getInnerStorage(); - auto lock = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); - auto stream = inner_storage->write(window_view.getInnerQuery(), context); - copyData(*source_stream, *stream); - } - else - { - source_stream->readPrefix(); - { - std::lock_guard lock(window_view.mutex); - while (Block t_block = source_stream->read()) - window_view.mergeable_blocks.push_back(std::move(t_block)); - } - source_stream->readSuffix(); - } + auto & inner_storage = window_view.getInnerStorage(); + auto lock = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); + auto stream = inner_storage->write(window_view.getInnerQuery(), context); + copyData(*source_stream, *stream); } void StorageWindowView::startup() @@ -1093,16 +1093,9 @@ StoragePtr StorageWindowView::getParentStorage() const return parent_storage; } -Block & StorageWindowView::getMergeableHeader() const -{ - if (!mergeable_sample_block) - mergeable_sample_block = mergeable_blocks.front().cloneEmpty(); - return mergeable_sample_block; -} - StoragePtr & StorageWindowView::getInnerStorage() const { - if (inner_storage == nullptr && !inner_table_id.empty()) + if (inner_storage == nullptr) inner_storage = DatabaseCatalog::instance().getTable(inner_table_id); return inner_storage; } @@ -1118,21 +1111,12 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma { Pipes pipes; - if (!inner_table_id.empty()) - { - auto & storage = getInnerStorage(); - InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); - QueryPipeline pipeline; - BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); - for (auto & stream : streams) - pipes.emplace_back(std::make_shared(std::move(stream))); - } - else - { - if (mergeable_blocks.empty()) - return std::make_shared(getHeader()); - pipes = blocksToPipes(mergeable_blocks, getMergeableHeader()); - } + auto & storage = getInnerStorage(); + InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); + QueryPipeline pipeline; + BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); + for (auto & stream : streams) + pipes.emplace_back(std::make_shared(std::move(stream))); ColumnsWithTypeAndName t_columns; t_columns.emplace_back( diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 4597b26fe6f..e9a0c0c7e45 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -63,7 +63,6 @@ private: bool is_tumble; // false if is hop std::atomic shutdown_called{false}; mutable Block sample_block; - mutable Block mergeable_sample_block; UInt64 clean_interval; const DateLUTImpl & time_zone; UInt32 max_timestamp = 0; @@ -78,7 +77,6 @@ private: std::list> watch_streams; std::condition_variable_any fire_signal_condition; std::condition_variable fire_condition; - BlocksList mergeable_blocks; /// Mutex for the blocks and ready condition std::mutex mutex; @@ -110,7 +108,7 @@ private: ASTPtr innerQueryParser(ASTSelectQuery & inner_query); - std::shared_ptr generateInnerTableCreateQuery(const ASTCreateQuery & inner_create_query, const String & database_name, const String & table_name); + std::shared_ptr generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name); ASTPtr generateCleanCacheQuery(UInt32 timestamp); UInt32 getWindowLowerBound(UInt32 time_sec); @@ -138,8 +136,6 @@ private: Block & getHeader() const; - Block & getMergeableHeader() const; - StorageWindowView( const StorageID & table_id_, Context & local_context, diff --git a/dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference b/dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference new file mode 100644 index 00000000000..afa64024e56 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference @@ -0,0 +1,22 @@ +---TUMBLE--- +||---WINDOW COLUMN NAME--- +CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192 +||---WINDOW COLUMN ALIAS--- +CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192 +||---IDENTIFIER--- +CREATE TABLE default.`.inner.wv` (`b` Int32, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b) SETTINGS index_granularity = 8192 +||---FUNCTION--- +CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +||---PARTITION--- +CREATE TABLE default.`.inner.wv` (`TUMBLE(____timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` ORDER BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192 +---HOP--- +||---WINDOW COLUMN NAME--- +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192 +||---WINDOW COLUMN ALIAS--- +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192 +||---IDENTIFIER--- +CREATE TABLE default.`.inner.wv` (`b` Int32, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192 +||---FUNCTION--- +CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +||---PARTITION--- +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY `arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192 diff --git a/dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql b/dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql new file mode 100644 index 00000000000..5df48b10f24 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql @@ -0,0 +1,61 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS mt; + +CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +SELECT '---TUMBLE---'; +SELECT '||---WINDOW COLUMN NAME---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), TUMBLE_END(wid) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---WINDOW COLUMN ALIAS---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---IDENTIFIER---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---FUNCTION---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---PARTITION---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, TUMBLE(now(), INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; +SHOW CREATE TABLE `.inner.wv`; + + +SELECT '---HOP---'; +SELECT '||---WINDOW COLUMN NAME---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---WINDOW COLUMN ALIAS---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---IDENTIFIER---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---FUNCTION---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---PARTITION---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE `.inner.wv`; + +DROP TABLE wv; +DROP TABLE mt; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference b/dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference new file mode 100644 index 00000000000..d35af28a9d5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference @@ -0,0 +1,22 @@ +---TUMBLE--- +||---WINDOW COLUMN NAME--- +CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY tuple(`TUMBLE(timestamp, toIntervalSecond(\'1\'))`) SETTINGS index_granularity = 8192 +||---WINDOW COLUMN ALIAS--- +CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY tuple(`TUMBLE(timestamp, toIntervalSecond(\'1\'))`) SETTINGS index_granularity = 8192 +||---IDENTIFIER--- +CREATE TABLE default.`.inner.wv` (`b` Int32, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `b` Int32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b) SETTINGS index_granularity = 8192 +||---FUNCTION--- +CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +---HOP--- +||---WINDOW COLUMN NAME--- +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY tuple(`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`) SETTINGS index_granularity = 8192 +||---WINDOW COLUMN ALIAS--- +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY tuple(`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`) SETTINGS index_granularity = 8192 +||---IDENTIFIER--- +CREATE TABLE default.`.inner.wv` (`b` Int32, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `b` Int32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192 +||---FUNCTION--- +CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192 diff --git a/dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.sql b/dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.sql new file mode 100644 index 00000000000..e3b137546e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01048_window_view_inner_table_generation.sql @@ -0,0 +1,68 @@ +SET allow_experimental_window_view = 1; + +DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS `.inner.wv`; + +CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); + +SELECT '---TUMBLE---'; +SELECT '||---WINDOW COLUMN NAME---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---WINDOW COLUMN ALIAS---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---IDENTIFIER---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, b; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---FUNCTION---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); +SHOW CREATE TABLE `.inner.wv`; + + +SELECT '---HOP---'; +SELECT '||---WINDOW COLUMN NAME---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---WINDOW COLUMN ALIAS---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---IDENTIFIER---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b; +SHOW CREATE TABLE `.inner.wv`; + +SELECT '||---FUNCTION---'; +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE `.inner.wv`; + +DROP TABLE IF EXISTS wv; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); +SHOW CREATE TABLE `.inner.wv`; + +DROP TABLE wv; +DROP TABLE mt; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.reference b/dbms/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.reference rename to dbms/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql b/dbms/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01053_window_view_proc_now_tumble_to.sql rename to dbms/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.reference rename to dbms/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql b/dbms/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.sql rename to dbms/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql deleted file mode 100755 index 8e78a74c95c..00000000000 --- a/dbms/tests/queries/0_stateless/01054_window_view_proc_now_tumble_inner_to.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree PARTITION BY wid ORDER BY tuple(TUMBLE(now(), INTERVAL '1' SECOND)) AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND) AS wid; - -INSERT INTO mt VALUES (1); -SELECT sleep(2); -SELECT count from dst; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.reference b/dbms/tests/queries/0_stateless/01054_window_view_proc_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01055_window_view_proc_now_hop_to.reference rename to dbms/tests/queries/0_stateless/01054_window_view_proc_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql b/dbms/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.sql rename to dbms/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01055_window_view_proc_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.reference rename to dbms/tests/queries/0_stateless/01055_window_view_proc_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql b/dbms/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.sql rename to dbms/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py b/dbms/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py similarity index 100% rename from dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.py rename to dbms/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py diff --git a/dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.reference b/dbms/tests/queries/0_stateless/01056_window_view_proc_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01061_window_view_proc_hop_watch.reference rename to dbms/tests/queries/0_stateless/01056_window_view_proc_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql deleted file mode 100755 index ad93163870d..00000000000 --- a/dbms/tests/queries/0_stateless/01056_window_view_proc_now_hop_inner_to.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv to dst ENGINE=AggregatingMergeTree ORDER BY tuple(wid) AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; - -INSERT INTO mt VALUES (1); -SELECT sleep(2); -SELECT count from dst; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.reference b/dbms/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.reference rename to dbms/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01062_window_view_event_strict_asc_tumble_to.sql rename to dbms/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.reference b/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01057_window_view_proc_tumble_to.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.reference b/dbms/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.reference rename to dbms/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01064_window_view_event_strict_asc_hop_to.sql rename to dbms/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql deleted file mode 100755 index fa3600de95a..00000000000 --- a/dbms/tests/queries/0_stateless/01058_window_view_proc_tumble_inner_to.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; - -INSERT INTO mt VALUES (1, now()); -SELECT sleep(2); -SELECT count from dst; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py b/dbms/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py similarity index 100% rename from dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.py rename to dbms/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py diff --git a/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.reference b/dbms/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_hop_watch.reference rename to dbms/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.reference b/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01059_window_view_proc_hop_to.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.reference b/dbms/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.reference rename to dbms/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql b/dbms/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to.sql rename to dbms/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.reference deleted file mode 100644 index 0d66ea1aee9..00000000000 --- a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.reference +++ /dev/null @@ -1,2 +0,0 @@ -0 -1 diff --git a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql deleted file mode 100755 index 4d802af103a..00000000000 --- a/dbms/tests/queries/0_stateless/01060_window_view_proc_hop_inner_to.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND) AS wid; - -INSERT INTO mt VALUES (1, now()); -SELECT sleep(2); -SELECT count from dst; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.reference b/dbms/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.reference rename to dbms/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql b/dbms/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01069_window_view_event_asc_hop_to.sql rename to dbms/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py b/dbms/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py similarity index 100% rename from dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.py rename to dbms/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py diff --git a/dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.reference b/dbms/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01071_window_view_event_asc_hop_watch.reference rename to dbms/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.reference rename to dbms/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.reference diff --git a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.sql rename to dbms/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference deleted file mode 100644 index 50ab245a54d..00000000000 --- a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 -3 1990-01-01 12:00:05 -1 1990-01-01 12:00:10 -2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql deleted file mode 100644 index bdad1d5a4b2..00000000000 --- a/dbms/tests/queries/0_stateless/01063_window_view_event_strict_asc_tumble_inner_to.sql +++ /dev/null @@ -1,24 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; - -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); - -SELECT sleep(1); -SELECT * from dst order by w_end; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.reference rename to dbms/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.reference diff --git a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql b/dbms/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.sql rename to dbms/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql diff --git a/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py b/dbms/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py similarity index 100% rename from dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.py rename to dbms/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py diff --git a/dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.reference b/dbms/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01076_window_view_event_bounded_hop_watch.reference rename to dbms/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.reference diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference deleted file mode 100644 index 166a57070b0..00000000000 --- a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.reference +++ /dev/null @@ -1,7 +0,0 @@ -0 -2 1990-01-01 12:00:02 -2 1990-01-01 12:00:04 -1 1990-01-01 12:00:06 -1 1990-01-01 12:00:08 -2 1990-01-01 12:00:12 -1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql deleted file mode 100644 index 0be01e39769..00000000000 --- a/dbms/tests/queries/0_stateless/01065_window_view_event_strict_asc_hop_inner_to.sql +++ /dev/null @@ -1,24 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree() order by wid WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; - -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); - -SELECT sleep(1); -SELECT * from dst order by w_end; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.reference b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.reference rename to dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.reference diff --git a/dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01077_window_view_event_strict_asc_tumble_to_lateness.sql rename to dbms/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql diff --git a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.reference b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.reference rename to dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.reference diff --git a/dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01078_window_view_event_asc_tumble_to_lateness.sql rename to dbms/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql diff --git a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql deleted file mode 100644 index 40f2634d0c6..00000000000 --- a/dbms/tests/queries/0_stateless/01068_window_view_event_asc_tumble_inner_to.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; - -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); - -SELECT sleep(1); -SELECT * from dst order by w_end; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.reference b/dbms/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.reference rename to dbms/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.reference diff --git a/dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql b/dbms/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01079_window_view_event_bounded_tumble_to_lateness.sql rename to dbms/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql diff --git a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql deleted file mode 100644 index b4e8cfdf6bb..00000000000 --- a/dbms/tests/queries/0_stateless/01070_window_view_event_asc_hop_inner_to.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; - -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); - -SELECT sleep(1); -SELECT * from dst order by w_end; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference b/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference deleted file mode 100644 index 1096602f00f..00000000000 --- a/dbms/tests/queries/0_stateless/01072_window_view_event_bounded_tumble_to.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 -3 1990-01-01 12:00:05 -2 1990-01-01 12:00:10 -2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference deleted file mode 100644 index 1096602f00f..00000000000 --- a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 -3 1990-01-01 12:00:05 -2 1990-01-01 12:00:10 -2 1990-01-01 12:00:15 diff --git a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql b/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql deleted file mode 100644 index e8288077793..00000000000 --- a/dbms/tests/queries/0_stateless/01073_window_view_event_bounded_tumble_inner_to.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=MergeTree ORDER BY tuple() WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; - -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); - -SELECT sleep(1); -SELECT * from dst order by w_end; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; diff --git a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference b/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference deleted file mode 100644 index 047b7bfbec9..00000000000 --- a/dbms/tests/queries/0_stateless/01074_window_view_event_bounded_hop_to.reference +++ /dev/null @@ -1,7 +0,0 @@ -0 -2 1990-01-01 12:00:02 -2 1990-01-01 12:00:04 -1 1990-01-01 12:00:06 -2 1990-01-01 12:00:08 -2 1990-01-01 12:00:12 -1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference deleted file mode 100644 index 047b7bfbec9..00000000000 --- a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.reference +++ /dev/null @@ -1,7 +0,0 @@ -0 -2 1990-01-01 12:00:02 -2 1990-01-01 12:00:04 -1 1990-01-01 12:00:06 -2 1990-01-01 12:00:08 -2 1990-01-01 12:00:12 -1 1990-01-01 12:00:14 diff --git a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql b/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql deleted file mode 100644 index 98b12f6c4c2..00000000000 --- a/dbms/tests/queries/0_stateless/01075_window_view_event_bounded_hop_inner_to.sql +++ /dev/null @@ -1,25 +0,0 @@ -SET allow_experimental_window_view = 1; - -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; - -CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); -CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst ENGINE=AggregatingMergeTree ORDER BY wid WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND) AS wid; - -INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:05'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:06'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); - -SELECT sleep(1); -SELECT * from dst order by w_end; - -DROP TABLE wv; -DROP TABLE mt; -DROP TABLE dst; From 107ddb21b3ad965388d22a207408a274ad04d4da Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 28 Mar 2020 10:08:21 +0800 Subject: [PATCH 044/609] fix build --- dbms/src/Storages/WindowView/StorageWindowView.cpp | 3 +-- dbms/src/Storages/WindowView/StorageWindowView.h | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 62fbe5349c5..57ee9b460dc 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -453,7 +453,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery inner_create_query->database = database_name; inner_create_query->table = table_name; - auto inner_select_query = dynamic_pointer_cast(getInnerQuery()); + auto inner_select_query = std::dynamic_pointer_cast(getInnerQuery()); Aliases aliases; QueryAliasesVisitor::Data query_aliases_data{aliases}; @@ -499,7 +499,6 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery { new_storage->set(new_storage->engine, makeASTFunction("AggregatingMergeTree")); - // auto inner_select_query = std::dynamic_pointer_cast(getInnerQuery()); for (auto & child : inner_select_query->groupBy()->children) if (auto * ast_with_alias = dynamic_cast(child.get())) ast_with_alias->setAlias(""); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index e9a0c0c7e45..51a6bea1667 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -15,7 +15,7 @@ class IAST; class WindowViewBlockInputStream; using ASTPtr = std::shared_ptr; -class StorageWindowView : public ext::shared_ptr_helper, public IStorage +class StorageWindowView final : public ext::shared_ptr_helper, public IStorage { friend struct ext::shared_ptr_helper; friend class TimestampTransformation; From 8f84599ba3bc8bf7788239069e431dc2d4dabed5 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 31 Mar 2020 13:48:01 +0800 Subject: [PATCH 045/609] update fetch column logic --- .../Storages/WindowView/StorageWindowView.cpp | 75 +++++++------------ .../Storages/WindowView/StorageWindowView.h | 2 +- 2 files changed, 27 insertions(+), 50 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 57ee9b460dc..4ab2b4f582b 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -224,25 +224,6 @@ namespace } String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } - - std::shared_ptr generateFetchColumnsQuery(const StorageID & inner_storage) - { - auto res_query = std::make_shared(); - auto select = std::make_shared(); - select->children.push_back(std::make_shared()); - res_query->setExpression(ASTSelectQuery::Expression::SELECT, select); - res_query->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); - auto tables = res_query->tables(); - auto tables_elem = std::make_shared(); - auto table_expr = std::make_shared(); - tables->children.push_back(tables_elem); - tables_elem->table_expression = table_expr; - tables_elem->children.push_back(table_expr); - table_expr->database_and_table_name = createTableIdentifier(inner_storage.database_name, inner_storage.table_name); - table_expr->children.push_back(table_expr->database_and_table_name); - - return res_query; - } } static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) @@ -285,7 +266,7 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa ASTPtr StorageWindowView::generateCleanCacheQuery(UInt32 timestamp) { auto function_tuple - = makeASTFunction("tupleElement", std::make_shared(window_column_name), std::make_shared("2")); + = makeASTFunction("tupleElement", std::make_shared(window_column_name), std::make_shared(Field{UInt8(2)})); auto function_equal = makeASTFunction("less", function_tuple, std::make_shared(timestamp)); auto alter_command = std::make_shared(); @@ -453,7 +434,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery inner_create_query->database = database_name; inner_create_query->table = table_name; - auto inner_select_query = std::dynamic_pointer_cast(getInnerQuery()); + auto inner_select_query = std::static_pointer_cast(getInnerQuery()); Aliases aliases; QueryAliasesVisitor::Data query_aliases_data{aliases}; @@ -896,7 +877,6 @@ StorageWindowView::StorageWindowView( inner_storage = DatabaseCatalog::instance().getTable(StorageID(inner_create_query->database, inner_create_query->table)); inner_table_id = inner_storage->getStorageID(); } - fetch_column_query = generateFetchColumnsQuery(inner_table_id); clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) @@ -1099,6 +1079,29 @@ StoragePtr & StorageWindowView::getInnerStorage() const return inner_storage; } +ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 watermark) const +{ + auto res_query = std::make_shared(); + auto select = std::make_shared(); + select->children.push_back(std::make_shared()); + res_query->setExpression(ASTSelectQuery::Expression::SELECT, select); + res_query->setExpression(ASTSelectQuery::Expression::TABLES, std::make_shared()); + auto tables_elem = std::make_shared(); + auto table_expr = std::make_shared(); + res_query->tables()->children.push_back(tables_elem); + tables_elem->table_expression = table_expr; + tables_elem->children.push_back(table_expr); + table_expr->database_and_table_name = createTableIdentifier(inner_table_id.database_name, inner_table_id.table_name); + table_expr->children.push_back(table_expr->database_and_table_name); + + auto func_tuple + = makeASTFunction("tupleElement", std::make_shared(window_column_name), std::make_shared(Field{UInt8(2)})); + auto func_equals = makeASTFunction("equals", func_tuple, std::make_shared(watermark)); + res_query->setExpression(ASTSelectQuery::Expression::WHERE, func_equals); + + return res_query; +} + StoragePtr & StorageWindowView::getTargetStorage() const { if (target_storage == nullptr && !target_table_id.empty()) @@ -1111,38 +1114,12 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma Pipes pipes; auto & storage = getInnerStorage(); - InterpreterSelectQuery fetch(fetch_column_query, global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); + InterpreterSelectQuery fetch(getFetchColumnQuery(watermark), global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); QueryPipeline pipeline; BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); for (auto & stream : streams) pipes.emplace_back(std::make_shared(std::move(stream))); - ColumnsWithTypeAndName t_columns; - t_columns.emplace_back( - nullptr, - std::make_shared(DataTypes{std::make_shared(), std::make_shared()}), - window_column_name); - - ExpressionActionsPtr actions = std::make_shared(t_columns, global_context); - const auto & function_tuple = FunctionFactory::instance().get("tupleElement", global_context); - actions->add(ExpressionAction::addColumn( - {std::make_shared()->createColumnConst(1, toField(2)), std::make_shared(), "____tuple_arg"})); - actions->add(ExpressionAction::applyFunction(function_tuple, Names{window_column_name, "____tuple_arg"}, "____w_end")); - actions->add(ExpressionAction::removeColumn("____tuple_arg")); - - actions->add(ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(watermark)), - std::make_shared(), - "____watermark"})); - const auto & function_equals = FunctionFactory::instance().get("equals", global_context); - ExpressionActionsPtr apply_function_actions = std::make_shared(t_columns, global_context); - actions->add(ExpressionAction::applyFunction(function_equals, Names{"____w_end", "____watermark"}, "____filter")); - actions->add(ExpressionAction::removeColumn("____w_end")); - actions->add(ExpressionAction::removeColumn("____watermark")); - - for (auto & pipe : pipes) - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), actions, - "____filter", true)); - auto proxy_storage = std::make_shared( StorageID(global_context.getCurrentDatabase(), "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index 51a6bea1667..f5c88dc0a18 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -55,7 +55,6 @@ public: private: ASTPtr inner_query; ASTPtr final_query; - ASTPtr fetch_column_query; Context & global_context; bool is_proctime{true}; @@ -127,6 +126,7 @@ private: ASTPtr getInnerQuery() const { return inner_query->clone(); } ASTPtr getFinalQuery() const { return final_query->clone(); } + ASTPtr getFetchColumnQuery(UInt32 watermark) const; StoragePtr getParentStorage() const; From 36383d83f1958e9099f437535daa8c2dcb586782 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 31 Mar 2020 14:28:04 +0800 Subject: [PATCH 046/609] use max_fired_watermark to cleanup cache in proc time --- .../Storages/WindowView/StorageWindowView.cpp | 26 ++++++++----------- 1 file changed, 11 insertions(+), 15 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index 4ab2b4f582b..3cb82871a3e 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -341,31 +341,26 @@ Pipes StorageWindowView::blocksToPipes(BlocksList & blocks, Block & sample_block inline void StorageWindowView::cleanCache() { UInt32 w_bound; - if (is_proctime) - { - w_bound = getWindowUpperBound(std::time(nullptr)); - } - else { std::lock_guard lock(fire_signal_mutex); - if (max_watermark == 0) - return; w_bound = max_fired_watermark; if (w_bound == 0) return; - if (allowed_lateness) + if (!is_proctime) { - UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, time_zone); - lateness_bound = getWindowLowerBound(lateness_bound); - if (lateness_bound < w_bound) - w_bound = lateness_bound; + if (max_watermark == 0) + return; + if (allowed_lateness) + { + UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, time_zone); + lateness_bound = getWindowLowerBound(lateness_bound); + if (lateness_bound < w_bound) + w_bound = lateness_bound; + } } } - w_bound = is_tumble ? addTime(w_bound, window_kind, -1 * window_num_units, time_zone) - : addTime(w_bound, hop_kind, -1 * hop_num_units, time_zone); - auto sql = generateCleanCacheQuery(w_bound); InterpreterAlterQuery alt_query(sql, global_context); alt_query.execute(); @@ -694,6 +689,7 @@ void StorageWindowView::threadFuncFireProc() while (next_fire_signal <= timestamp_now) { fire(next_fire_signal); + max_fired_watermark = next_fire_signal; next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, time_zone); } From 9df20c091748075abe3ab6a644f25f5e43d77e55 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 31 Mar 2020 22:02:53 +0800 Subject: [PATCH 047/609] add wv_context --- .../Storages/WindowView/StorageWindowView.cpp | 35 ++++++++++--------- .../Storages/WindowView/StorageWindowView.h | 3 +- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/dbms/src/Storages/WindowView/StorageWindowView.cpp b/dbms/src/Storages/WindowView/StorageWindowView.cpp index b8dd01bb61e..5eddd0732b6 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.cpp +++ b/dbms/src/Storages/WindowView/StorageWindowView.cpp @@ -295,7 +295,7 @@ void StorageWindowView::checkTableCanBeDropped() const } } -static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, const StorageID & target_table_id) +static void executeDropQuery(ASTDropQuery::Kind kind, Context & context, const StorageID & target_table_id) { if (DatabaseCatalog::instance().tryGetTable(target_table_id)) { @@ -304,7 +304,7 @@ static void executeDropQuery(ASTDropQuery::Kind kind, Context & global_context, drop_query->table = target_table_id.table_name; drop_query->kind = kind; ASTPtr ast_drop_query = drop_query; - InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); + InterpreterDropQuery drop_interpreter(ast_drop_query, context); drop_interpreter.execute(); } } @@ -362,7 +362,7 @@ inline void StorageWindowView::cleanCache() } auto sql = generateCleanCacheQuery(w_bound); - InterpreterAlterQuery alt_query(sql, global_context); + InterpreterAlterQuery alt_query(sql, wv_context); alt_query.execute(); std::lock_guard lock(fire_signal_mutex); @@ -398,8 +398,8 @@ inline void StorageWindowView::fire(UInt32 watermark) try { StoragePtr target_table = getTargetStorage(); - auto lock = target_table->lockStructureForShare(global_context.getCurrentQueryId()); - auto out_stream = target_table->write(getInnerQuery(), global_context); + auto lock = target_table->lockStructureForShare(wv_context.getCurrentQueryId()); + auto out_stream = target_table->write(getInnerQuery(), wv_context); in_stream->readPrefix(); out_stream->writePrefix(); while (auto block = in_stream->read()) @@ -436,7 +436,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery QueryAliasesVisitor(query_aliases_data).visit(inner_select_query); auto t_sample_block - = InterpreterSelectQuery(inner_select_query, global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) + = InterpreterSelectQuery(inner_select_query, wv_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::WithMergeableState)) .getSampleBlock(); auto columns_list = std::make_shared(); @@ -759,8 +759,11 @@ StorageWindowView::StorageWindowView( bool attach_) : IStorage(table_id_) , global_context(local_context.getGlobalContext()) + , wv_context(Context(global_context)) , time_zone(DateLUT::instance()) { + wv_context.makeQueryContext(); + setColumns(columns_); if (!query.select) @@ -770,7 +773,7 @@ StorageWindowView::StorageWindowView( throw Exception("UNION is not supported for Window View", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); ASTSelectQuery & select_query = typeid_cast(*query.select->list_of_selects->children.at(0)); - String select_database_name = local_context.getCurrentDatabase(); + String select_database_name = global_context.getCurrentDatabase(); String select_table_name; extractDependentTable(select_query, select_database_name, select_table_name); select_table_id = StorageID(select_database_name, select_table_name); @@ -798,7 +801,7 @@ StorageWindowView::StorageWindowView( if (!query.to_table.empty()) target_table_id = StorageID(query.to_database, query.to_table); - clean_interval = local_context.getSettingsRef().window_view_clean_interval.totalSeconds(); + clean_interval = global_context.getSettingsRef().window_view_clean_interval.totalSeconds(); next_fire_signal = getWindowUpperBound(std::time(nullptr)); if (query.is_watermark_strictly_ascending || query.is_watermark_ascending || query.is_watermark_bounded) @@ -867,18 +870,18 @@ StorageWindowView::StorageWindowView( auto inner_create_query = generateInnerTableCreateQuery(query.storage, table_id_.database_name, generateInnerTableName(table_id_.table_name)); - InterpreterCreateQuery create_interpreter(inner_create_query, local_context); + InterpreterCreateQuery create_interpreter(inner_create_query, wv_context); create_interpreter.setInternal(true); create_interpreter.execute(); inner_storage = DatabaseCatalog::instance().getTable(StorageID(inner_create_query->database, inner_create_query->table)); inner_table_id = inner_storage->getStorageID(); } - clean_cache_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); + clean_cache_task = wv_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) - fire_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); + fire_task = wv_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); else - fire_task = global_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); }); + fire_task = wv_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireEvent(); }); clean_cache_task->deactivate(); fire_task->deactivate(); } @@ -1053,7 +1056,7 @@ Block & StorageWindowView::getHeader() const if (!sample_block) { sample_block = InterpreterSelectQuery( - getInnerQuery(), global_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) + getInnerQuery(), wv_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) .getSampleBlock(); for (size_t i = 0; i < sample_block.columns(); ++i) sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); @@ -1110,16 +1113,16 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma Pipes pipes; auto & storage = getInnerStorage(); - InterpreterSelectQuery fetch(getFetchColumnQuery(watermark), global_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); + InterpreterSelectQuery fetch(getFetchColumnQuery(watermark), wv_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); QueryPipeline pipeline; BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); for (auto & stream : streams) pipes.emplace_back(std::make_shared(std::move(stream))); auto proxy_storage = std::make_shared( - StorageID(global_context.getCurrentDatabase(), "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); + StorageID(wv_context.getCurrentDatabase(), "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); - InterpreterSelectQuery select(getFinalQuery(), global_context, proxy_storage, QueryProcessingStage::Complete); + InterpreterSelectQuery select(getFinalQuery(), wv_context, proxy_storage, QueryProcessingStage::Complete); BlockInputStreamPtr data = std::make_shared(select.execute().in); /// Squashing is needed here because the view query can generate a lot of blocks diff --git a/dbms/src/Storages/WindowView/StorageWindowView.h b/dbms/src/Storages/WindowView/StorageWindowView.h index f5c88dc0a18..9945a1b0201 100644 --- a/dbms/src/Storages/WindowView/StorageWindowView.h +++ b/dbms/src/Storages/WindowView/StorageWindowView.h @@ -56,7 +56,8 @@ private: ASTPtr inner_query; ASTPtr final_query; - Context & global_context; + Context global_context; + Context wv_context; bool is_proctime{true}; bool is_time_column_func_now; bool is_tumble; // false if is hop From d16fbcfb9fe55bc7c40436df5c06059d5584ca9e Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 8 Jun 2020 14:27:30 +0800 Subject: [PATCH 048/609] Reuse intermediate results in hop window --- src/Functions/FunctionsWindow.cpp | 1 + src/Functions/FunctionsWindow.h | 187 +++++++-- .../ReplaceWindowColumnBlockInputStream.h | 77 ++++ src/Storages/WindowView/StorageWindowView.cpp | 394 +++++++++++------- src/Storages/WindowView/StorageWindowView.h | 12 +- .../WindowView/WatermarkBlockInputStream.h | 51 ++- ...7_window_view_inner_table_parser.reference | 10 +- ...ndow_view_inner_table_generation.reference | 12 +- ...049_window_view_window_functions.reference | 2 +- 9 files changed, 544 insertions(+), 202 deletions(-) create mode 100644 src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h diff --git a/src/Functions/FunctionsWindow.cpp b/src/Functions/FunctionsWindow.cpp index b99183044ce..063e39992f5 100644 --- a/src/Functions/FunctionsWindow.cpp +++ b/src/Functions/FunctionsWindow.cpp @@ -7,6 +7,7 @@ void registerFunctionsWindow(FunctionFactory& factory) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index c5c3419136b..1d97fec8c81 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -12,6 +12,7 @@ #include #include #include +#include #include "IFunctionImpl.h" @@ -47,6 +48,7 @@ enum WindowFunctionName TUMBLE_START, TUMBLE_END, HOP, + HOP_SLICE, HOP_START, HOP_END }; @@ -379,8 +381,7 @@ namespace "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name + ". This argument is optional and must be a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared( - std::make_shared(DataTypes{std::make_shared(), std::make_shared()})); + return std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); } static ColumnPtr @@ -445,19 +446,11 @@ namespace { const auto & time_data = time_column.getData(); size_t size = time_column.size(); - int max_wid_nums = window_num_units / hop_num_units + (window_num_units % hop_num_units != 0); - auto column_offsets = ColumnArray::ColumnOffsets::create(size); - IColumn::Offsets & out_offsets = column_offsets->getData(); - - auto start = ColumnUInt32::create(); - auto end = ColumnUInt32::create(); + auto start = ColumnUInt32::create(size); + auto end = ColumnUInt32::create(size); ColumnUInt32::Container & start_data = start->getData(); ColumnUInt32::Container & end_data = end->getData(); - start_data.reserve(max_wid_nums * size); - end_data.reserve(max_wid_nums * size); - out_offsets.reserve(size); - IColumn::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { UInt32 wstart = static_cast(ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone)); @@ -473,23 +466,131 @@ namespace wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); } while (wend_ > time_data[i]); - UInt32 wstart_ = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); - wend_ = wend_latest; - - while (wstart_ <= time_data[i]) - { - start_data.push_back(wstart_); - end_data.push_back(wend_); - wstart_ = AddTime::execute(wstart_, hop_num_units, time_zone); - wend_ = AddTime::execute(wstart_, window_num_units, time_zone); - ++current_offset; - } - out_offsets[i] = current_offset; + end_data[i] = wend_latest; + start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); } - MutableColumns tuple_columns; - tuple_columns.emplace_back(std::move(start)); - tuple_columns.emplace_back(std::move(end)); - return ColumnArray::create(ColumnTuple::create(std::move(tuple_columns)), std::move(column_offsets)); + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } + }; + + template <> + struct WindowImpl + { + static constexpr auto name = "HOP_SLICE"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() != 3 && arguments.size() != 4) + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 3.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (!WhichDataType(arguments[0].type).isDateTime()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[1].type).isInterval()) + throw Exception( + "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (!WhichDataType(arguments[2].type).isInterval()) + throw Exception( + "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) + throw Exception( + "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return std::make_shared(); + } + + static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) + { + const auto & time_column = block.getByPosition(arguments[0]); + const auto & hop_interval_column = block.getByPosition(arguments[1]); + const auto & window_interval_column = block.getByPosition(arguments[2]); + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(block, arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<0>(hop_interval) != std::get<0>(window_interval)) + throw Exception( + "Interval type of window and hop column of function " + function_name + ", must be same.", ErrorCodes::ILLEGAL_COLUMN); + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval.", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) + { + case IntervalKind::Second: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr + execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + { + Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); + + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + + auto end = ColumnUInt32::create(size); + ColumnUInt32::Container & end_data = end->getData(); + for (size_t i = 0; i < size; ++i) + { + UInt32 wstart = static_cast(ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone)); + UInt32 wend = AddTime::execute(wstart, hop_num_units, time_zone); + + UInt32 wend_ = wend; + UInt32 wend_latest; + + do + { + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); + } while (wend_ > time_data[i]); + + end_data[i] = wend_latest; + } + return end; } }; @@ -503,9 +604,9 @@ namespace if (arguments.size() == 1) { auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isArray()) + if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Illegal type of first argument of function " + function_name + " should be tuple or array", + "Illegal type of first argument of function " + function_name + " should be Tuple, Array, UInt8 or UInt32", ErrorCodes::ILLEGAL_COLUMN); return std::make_shared(); } @@ -531,7 +632,7 @@ namespace { throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 1 or 3.", + + ", should be 1, 3 or 4.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } } @@ -542,10 +643,15 @@ namespace const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); + if (arguments.size() == 1) + { + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; + } else - result_column_ = block.getByPosition(arguments[0]).column; + result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); return executeWindowBound(result_column_, 0, function_name); } }; @@ -566,10 +672,16 @@ namespace const auto & time_column = block.getByPosition(arguments[0]); const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); + if (arguments.size() == 1) + { + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; + } else - result_column_ = block.getByPosition(arguments[0]).column; + result_column_ = WindowImpl::dispatchForColumns(block, arguments, function_name); + return executeWindowBound(result_column_, 1, function_name); } }; @@ -600,6 +712,7 @@ using FunctionTumble = FunctionWindow; using FunctionTumbleStart = FunctionWindow; using FunctionTumbleEnd = FunctionWindow; using FunctionHop = FunctionWindow; +using FunctionHopSlice = FunctionWindow; using FunctionHopStart = FunctionWindow; using FunctionHopEnd = FunctionWindow; } diff --git a/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h b/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h new file mode 100644 index 00000000000..21e77761238 --- /dev/null +++ b/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h @@ -0,0 +1,77 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class ReplaceWindowColumnBlockInputStream : public IBlockInputStream +{ +public: + ReplaceWindowColumnBlockInputStream( + BlockInputStreamPtr input_, + String window_column_name_, + UInt32 window_start_, + UInt32 window_end_) + : window_column_name(window_column_name_) + { + children.push_back(input_); + window_column_type = std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); + window_value.emplace_back(window_start_); + window_value.emplace_back(window_end_); + + position = -1; + Block header = children.back()->getHeader(); + for (auto & column_ : header.getColumnsWithTypeAndName()) + { + if (startsWith(column_.name, "HOP_SLICE")) + { + position = header.getPositionByName(column_.name); + hop_slice_name = column_.name; + break; + } + } + if(position < 0) + throw Exception("Not found column HOP_SLICE", ErrorCodes::LOGICAL_ERROR); + } + + String getName() const override { return "ReplaceWindowColumn"; } + + Block getHeader() const override + { + Block res = children.back()->getHeader(); + res.erase(position); + res.insert(position, {window_column_type->createColumn(), window_column_type, window_column_name}); + return res; + } + +protected: + Block readImpl() override + { + Block res = children.back()->read(); + if (!res) + return res; + + res.erase(position); + res.insert(position, {window_column_type->createColumnConst(res.rows(), window_value)->convertToFullColumnIfConst(), window_column_type, window_column_name}); + return res; + } + +private: + String window_column_name; + String hop_slice_name; + DataTypePtr window_column_type; + int position; + Tuple window_value; +}; + +} diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index da52d994305..493f7e12017 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1,3 +1,5 @@ +#include +#include #include #include #include @@ -41,6 +43,7 @@ #include #include +#include #include #include #include @@ -65,13 +68,14 @@ namespace { const auto RESCHEDULE_MS = 500; - struct StageMergeableVisitorData + struct MergeableQueryVisitorData { using TypeToVisit = ASTFunction; ASTPtr window_function; - String window_column_name; - String window_column_alias; + String window_id_name; + String window_id_alias; + String serialized_window_function; String timestamp_column_name; bool is_tumble = false; bool is_hop = false; @@ -83,58 +87,86 @@ namespace if (!window_function) { is_tumble = true; - window_column_name = node.getColumnName(); - window_column_alias = node.alias; + window_id_name = node.getColumnName(); + window_id_alias = node.alias; window_function = node.clone(); + window_function->setAlias(""); + serialized_window_function = serializeAST(*window_function); timestamp_column_name = node.arguments->children[0]->getColumnName(); } - else if (serializeAST(node) != serializeAST(*window_function)) - throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + else + { + auto temp_node = node.clone(); + temp_node->setAlias(""); + if (serializeAST(*temp_node) != serialized_window_function) + throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + } } else if (node.name == "HOP") { + is_hop = true; + std::static_pointer_cast(node_ptr)->name = "HOP_SLICE"; if (!window_function) { - is_hop = true; window_function = node.clone(); - timestamp_column_name = node.arguments->children[0]->getColumnName(); - auto ptr = node.clone(); - std::static_pointer_cast(ptr)->setAlias(""); - auto array_join = makeASTFunction("arrayJoin", ptr); - array_join->alias = node.alias; - node_ptr = array_join; - window_column_name = array_join->getColumnName(); - window_column_alias = array_join->alias; + window_function->setAlias(""); + serialized_window_function = serializeAST(*window_function); } else if (serializeAST(node) != serializeAST(*window_function)) - throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + { + auto temp_node = node.clone(); + temp_node->setAlias(""); + if (serializeAST(*temp_node) != serialized_window_function) + throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + } + + timestamp_column_name = node.arguments->children[0]->getColumnName(); + window_id_name = node.getColumnName(); + window_id_alias = node.alias; } } }; - struct ReplaceFuncNowVisitorData + struct ReplaceFunctionHopSliceVisitorData + { + using TypeToVisit = ASTFunction; + + void visit(const ASTFunction & node, ASTPtr & node_ptr) + { + if (node.name == "HOP_SLICE") + std::static_pointer_cast(node_ptr)->name = "HOP"; + } + }; + + struct ReplaceFunctionNowVisitorData { using TypeToVisit = ASTFunction; bool is_time_column_func_now = false; - String window_column_name; + String window_id_name; void visit(ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "TUMBLE") + if (node.name == "TUMBLE" || node.name == "HOP" || node.name == "HOP_SLICE") { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") { is_time_column_func_now = true; node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); - window_column_name = node.getColumnName(); + window_id_name = node.getColumnName(); } } - else if (node.name == "HOP") - { - if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") - is_time_column_func_now = true; - } + } + }; + + struct ReplaceFunctionHopVisitorData + { + using TypeToVisit = ASTFunction; + + void visit(ASTFunction & node, ASTPtr & node_ptr) + { + if (node.name == "HOP") + std::static_pointer_cast(node_ptr)->name = "HOP_SLICE"; } }; @@ -145,8 +177,8 @@ namespace struct Data { - String window_column_name; - String window_column_alias; + String window_id_name; + String window_id_alias; Aliases * aliases; }; @@ -165,16 +197,14 @@ namespace { if (node.name == "tuple") return; - if (node.name == "HOP") - node_ptr = std::make_shared("arrayJoin(" + node.getColumnName() + ")"); else node_ptr = std::make_shared(node.getColumnName()); } static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data) { - if (node.getColumnName() == data.window_column_alias) - dynamic_cast(node_ptr.get())->name = data.window_column_name; + if (node.getColumnName() == data.window_id_alias) + dynamic_cast(node_ptr.get())->name = data.window_id_name; else if (auto it = data.aliases->find(node.getColumnName()); it != data.aliases->end()) dynamic_cast(node_ptr.get())->name = it->second->getColumnName(); } @@ -201,28 +231,6 @@ namespace __builtin_unreachable(); } - UInt32 addTime(UInt32 time_sec, IntervalKind::Kind window_kind, int window_num_units, const DateLUTImpl & time_zone) - { - switch (window_kind) - { -#define CASE_WINDOW_KIND(KIND) \ - case IntervalKind::KIND: \ - { \ - return AddTime::execute(time_sec, window_num_units, time_zone); \ - } - CASE_WINDOW_KIND(Second) - CASE_WINDOW_KIND(Minute) - CASE_WINDOW_KIND(Hour) - CASE_WINDOW_KIND(Day) - CASE_WINDOW_KIND(Week) - CASE_WINDOW_KIND(Month) - CASE_WINDOW_KIND(Quarter) - CASE_WINDOW_KIND(Year) -#undef CASE_WINDOW_KIND - } - __builtin_unreachable(); - } - String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } } @@ -265,9 +273,15 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa ASTPtr StorageWindowView::generateCleanCacheQuery(UInt32 timestamp) { - auto function_tuple - = makeASTFunction("tupleElement", std::make_shared(window_column_name), std::make_shared(Field{UInt8(2)})); - auto function_equal = makeASTFunction("less", function_tuple, std::make_shared(timestamp)); + ASTPtr function_equal; + if (is_tumble) + { + auto function_tuple = makeASTFunction( + "tupleElement", std::make_shared(window_id_name), std::make_shared(Field{UInt8(2)})); + function_equal = makeASTFunction("less", function_tuple, std::make_shared(timestamp)); + } + else + function_equal = makeASTFunction("less", std::make_shared(window_column_name), std::make_shared(timestamp)); auto alter_command = std::make_shared(); alter_command->type = ASTAlterCommand::DELETE; @@ -353,7 +367,7 @@ inline void StorageWindowView::cleanCache() return; if (allowed_lateness) { - UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, time_zone); + UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units); lateness_bound = getWindowLowerBound(lateness_bound); if (lateness_bound < w_bound) w_bound = lateness_bound; @@ -361,6 +375,7 @@ inline void StorageWindowView::cleanCache() } } + w_bound = addTime(w_bound, window_kind, -1 * window_num_units); auto sql = generateCleanCacheQuery(w_bound); InterpreterAlterQuery alt_query(sql, wv_context); alt_query.execute(); @@ -395,29 +410,22 @@ inline void StorageWindowView::fire(UInt32 watermark) } else { - try + StoragePtr target_table = getTargetStorage(); + auto lock = target_table->lockStructureForShare(true, wv_context.getCurrentQueryId()); + auto out_stream = target_table->write(getFinalQuery(), wv_context); + in_stream->readPrefix(); + out_stream->writePrefix(); + while (auto block = in_stream->read()) { - StoragePtr target_table = getTargetStorage(); - auto lock = target_table->lockStructureForShare(true, wv_context.getCurrentQueryId()); - auto out_stream = target_table->write(getInnerQuery(), wv_context); - in_stream->readPrefix(); - out_stream->writePrefix(); - while (auto block = in_stream->read()) + for (auto & watch_stream : watch_streams) { - for (auto & watch_stream : watch_streams) - { - if (const auto & watch_stream_ptr = watch_stream.lock()) - watch_stream_ptr->addBlock(block); - } - out_stream->write(std::move(block)); + if (const auto & watch_stream_ptr = watch_stream.lock()) + watch_stream_ptr->addBlock(block); } - in_stream->readSuffix(); - out_stream->writeSuffix(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); + out_stream->write(std::move(block)); } + in_stream->readSuffix(); + out_stream->writeSuffix(); } fire_condition.notify_all(); } @@ -441,12 +449,15 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto columns_list = std::make_shared(); - if (is_time_column_func_now && is_tumble) + if (is_time_column_func_now) { auto column_window = std::make_shared(); - column_window->name = window_column_name; - column_window->type - = makeASTFunction("Tuple", std::make_shared("DateTime"), std::make_shared("DateTime")); + column_window->name = window_id_name; + if (is_tumble) + column_window->type + = makeASTFunction("Tuple", std::make_shared("DateTime"), std::make_shared("DateTime")); + else + column_window->type = std::make_shared("UInt32"); columns_list->children.push_back(column_window); } @@ -462,13 +473,15 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery } ReplaceFunctionWindowMatcher::Data query_data; - query_data.window_column_name = window_column_name; - query_data.window_column_alias = window_column_alias; + query_data.window_id_name = window_id_name; + query_data.window_id_alias = window_id_alias; query_data.aliases = &aliases; ReplaceFunctionWindowMatcher::Visitor visitor(query_data); - ReplaceFuncNowVisitorData parser_proc_time_data; - InDepthNodeVisitor, true> time_now_visitor(parser_proc_time_data); + ReplaceFunctionNowVisitorData time_now_data; + ReplaceFunctionHopVisitorData func_hop_data; + InDepthNodeVisitor, true> time_now_visitor(time_now_data); + InDepthNodeVisitor, true> func_hop_visitor(func_hop_data); auto new_storage = std::make_shared(); if (storage == nullptr) @@ -491,7 +504,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery for (auto & child : order_by->arguments->children) { - if (child->getColumnName() == window_column_name) + if (child->getColumnName() == window_id_name) { ASTPtr tmp = child; child = order_by->arguments->children[0]; @@ -500,7 +513,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery } } new_storage->set(new_storage->order_by, order_by_ptr); - new_storage->set(new_storage->primary_key, std::make_shared(window_column_name)); + new_storage->set(new_storage->primary_key, std::make_shared(window_id_name)); } else { @@ -517,6 +530,8 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto partition_by = storage->partition_by->clone(); if (is_time_column_func_now) time_now_visitor.visit(partition_by); + if (!is_tumble) + func_hop_visitor.visit(partition_by); visitor.visit(partition_by); new_storage->set(new_storage->partition_by, partition_by); } @@ -525,6 +540,8 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto primary_key = storage->primary_key->clone(); if (is_time_column_func_now) time_now_visitor.visit(primary_key); + if (!is_tumble) + func_hop_visitor.visit(primary_key); visitor.visit(primary_key); new_storage->set(new_storage->primary_key, primary_key); } @@ -533,6 +550,8 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto order_by = storage->order_by->clone(); if (is_time_column_func_now) time_now_visitor.visit(order_by); + if (!is_tumble) + func_hop_visitor.visit(order_by); visitor.visit(order_by); new_storage->set(new_storage->order_by, order_by); } @@ -541,6 +560,8 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto sample_by = storage->sample_by->clone(); if (is_time_column_func_now) time_now_visitor.visit(sample_by); + if (!is_tumble) + func_hop_visitor.visit(sample_by); visitor.visit(sample_by); new_storage->set(new_storage->sample_by, sample_by); } @@ -556,6 +577,27 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery return inner_create_query; } +inline UInt32 StorageWindowView::addTime(UInt32 time_sec, IntervalKind::Kind kind, Int64 num_units) const +{ + switch (kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: { \ + return AddTime::execute(time_sec, num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); +} + inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) { IntervalKind window_interval_kind; @@ -572,7 +614,11 @@ inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) if (is_tumble) \ return ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ else \ - return ToStartOfTransform::execute(time_sec, hop_num_units, time_zone); \ + {\ + UInt32 w_start = ToStartOfTransform::execute(time_sec, hop_num_units, time_zone); \ + UInt32 w_end = AddTime::execute(w_start, hop_num_units, time_zone);\ + return AddTime::execute(w_end, -1 * window_num_units, time_zone);\ + }\ } CASE_WINDOW_KIND(Second) CASE_WINDOW_KIND(Minute) @@ -589,13 +635,27 @@ inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) { - switch (window_kind) + IntervalKind window_interval_kind; + if (is_tumble) + window_interval_kind = window_kind; + else + window_interval_kind = hop_kind; + + switch (window_interval_kind) { #define CASE_WINDOW_KIND(KIND) \ case IntervalKind::KIND: \ { \ - UInt32 start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ - return AddTime::execute(start, window_num_units, time_zone); \ + if (is_tumble) \ + {\ + UInt32 w_start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ + return AddTime::execute(w_start, window_num_units, time_zone); \ + }\ + else \ + {\ + UInt32 w_start = ToStartOfTransform::execute(time_sec, hop_num_units, time_zone); \ + return AddTime::execute(w_start, hop_num_units, time_zone);\ + }\ } CASE_WINDOW_KIND(Second) CASE_WINDOW_KIND(Minute) @@ -642,19 +702,30 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) { fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; - max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); + if(is_tumble) + max_watermark = addTime(max_watermark, window_kind, window_num_units); + else + max_watermark = addTime(max_watermark, hop_kind, hop_num_units); } } else // strictly || bounded { - UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units, time_zone); + UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units); updated = max_watermark_bias <= watermark; while (max_watermark_bias <= max_timestamp) { fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; - max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); - max_watermark_bias = addTime(max_watermark, window_kind, window_num_units, time_zone); + if(is_tumble) + { + max_watermark = addTime(max_watermark, window_kind, window_num_units); + max_watermark_bias = addTime(max_watermark, window_kind, window_num_units); + } + else + { + max_watermark = addTime(max_watermark, hop_kind, hop_num_units); + max_watermark_bias = addTime(max_watermark, hop_kind, hop_num_units); + } } } if (updated) @@ -688,9 +759,16 @@ void StorageWindowView::threadFuncFireProc() UInt32 timestamp_now = std::time(nullptr); while (next_fire_signal <= timestamp_now) { - fire(next_fire_signal); + try + { + fire(next_fire_signal); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } max_fired_watermark = next_fire_signal; - next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, time_zone); + next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units); } next_fire_signal = getWindowUpperBound(timestamp_now); @@ -779,12 +857,19 @@ StorageWindowView::StorageWindowView( select_table_id = StorageID(select_database_name, select_table_name); inner_query = innerQueryParser(select_query); - final_query = inner_query->clone(); - ReplaceFuncNowVisitorData final_query_data; - InDepthNodeVisitor, true>(final_query_data).visit(final_query); - is_time_column_func_now = final_query_data.is_time_column_func_now; - if (is_time_column_func_now && is_tumble) - window_column_name = final_query_data.window_column_name; + mergeable_query = inner_query->clone(); + + ReplaceFunctionNowVisitorData func_now_data; + InDepthNodeVisitor, true>(func_now_data).visit(mergeable_query); + is_time_column_func_now = func_now_data.is_time_column_func_now; + if (is_time_column_func_now) + window_id_name = func_now_data.window_id_name; + + final_query = mergeable_query->clone(); + + ReplaceFunctionHopSliceVisitorData final_query_data; + InDepthNodeVisitor, true>(final_query_data).visit(final_query); + is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; is_watermark_ascending = query.is_watermark_ascending; is_watermark_bounded = query.is_watermark_bounded; @@ -877,6 +962,8 @@ StorageWindowView::StorageWindowView( inner_table_id = inner_storage->getStorageID(); } + window_column_name = std::regex_replace(window_id_name, std::regex("HOP_SLICE"), "HOP"); + clean_cache_task = wv_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) fire_task = wv_context.getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); @@ -895,12 +982,12 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parse stage mergeable ASTPtr result = query.clone(); ASTPtr expr_list = result; - StageMergeableVisitorData stage_mergeable_data; - InDepthNodeVisitor, true>(stage_mergeable_data).visit(expr_list); + MergeableQueryVisitorData stage_mergeable_data; + InDepthNodeVisitor, true>(stage_mergeable_data).visit(expr_list); if (!stage_mergeable_data.is_tumble && !stage_mergeable_data.is_hop) throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); - window_column_name = stage_mergeable_data.window_column_name; - window_column_alias = stage_mergeable_data.window_column_alias; + window_id_name = stage_mergeable_data.window_id_name; + window_id_alias = stage_mergeable_data.window_id_alias; timestamp_column_name = stage_mergeable_data.timestamp_column_name; is_tumble = stage_mergeable_data.is_tumble; @@ -918,14 +1005,18 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) if (!is_tumble) { + hop_kind = window_kind; + hop_num_units = window_num_units; const auto & arg2 = std::static_pointer_cast(arguments.at(2)); if (!arg2 || !startsWith(arg2->name, "toInterval")) throw Exception("Illegal type of last argument of function " + arg2->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - hop_kind = strToIntervalKind(arg2->name.substr(10)); + window_kind = strToIntervalKind(arg2->name.substr(10)); const auto & interval_units_p2 = std::static_pointer_cast(arg2->children.front()->children.front()); - hop_num_units = stoi(interval_units_p2->value.get()); - if (hop_num_units <= 0) + window_num_units = stoi(interval_units_p2->value.get()); + if (window_num_units <= 0) throw Exception("Interval value for WINDOW function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + slice_num_units= std::gcd(hop_num_units, window_num_units); } return result; } @@ -939,13 +1030,10 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (window_view.is_proctime) { fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); - if (window_view.is_tumble) - { - UInt32 timestamp_now = std::time(nullptr); - pipe.addSimpleTransform(std::make_shared>( - pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); - } - InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); + UInt32 timestamp_now = std::time(nullptr); + pipe.addSimpleTransform(std::make_shared>( + pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); + InterpreterSelectQuery select_block(window_view.getMergeableQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; source_stream = std::make_shared( @@ -968,12 +1056,12 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (t_max_timstamp!= 0) { UInt32 lateness_bound - = addTime(t_max_timstamp, window_view.lateness_kind, -1 * window_view.lateness_num_units, window_view.time_zone); + = window_view.addTime(t_max_timstamp, window_view.lateness_kind, -1 * window_view.lateness_num_units); if (window_view.is_watermark_bounded) { UInt32 watermark_lower_bound = window_view.is_tumble - ? addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units, window_view.time_zone) - : addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units, window_view.time_zone); + ? window_view.addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units) + : window_view.addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units); if (watermark_lower_bound < lateness_bound) lateness_bound = watermark_lower_bound; } @@ -1004,17 +1092,22 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con } } - InterpreterSelectQuery select_block(window_view.getFinalQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select_block(window_view.getMergeableQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().in; source_stream = std::make_shared( source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); - if (!window_view.is_tumble) - source_stream - = std::make_shared(source_stream, window_view, window_view.window_column_name, window_view.getWindowUpperBound(t_max_timstamp)); + if (window_view.is_tumble) + { + source_stream = std::make_shared(source_stream, window_view, window_view.window_id_name); + } else - source_stream = std::make_shared(source_stream, window_view, window_view.window_column_name); + { + source_stream + = std::make_shared(source_stream, window_view, window_view.window_id_name, window_view.getWindowUpperBound(t_max_timstamp)); + std::static_pointer_cast(source_stream)->setHopWindow(); + } if (window_view.is_watermark_bounded || window_view.allowed_lateness) std::static_pointer_cast(source_stream)->setMaxTimestamp(t_max_timstamp); @@ -1025,7 +1118,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con auto & inner_storage = window_view.getInnerStorage(); auto lock = inner_storage->lockStructureForShare(true, context.getCurrentQueryId()); - auto stream = inner_storage->write(window_view.getInnerQuery(), context); + auto stream = inner_storage->write(window_view.getMergeableQuery(), context); copyData(*source_stream, *stream); } @@ -1056,7 +1149,7 @@ Block & StorageWindowView::getHeader() const if (!sample_block) { sample_block = InterpreterSelectQuery( - getInnerQuery(), wv_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) + getFinalQuery(), wv_context, getParentStorage(), SelectQueryOptions(QueryProcessingStage::Complete)) .getSampleBlock(); for (size_t i = 0; i < sample_block.columns(); ++i) sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); @@ -1078,7 +1171,7 @@ StoragePtr & StorageWindowView::getInnerStorage() const return inner_storage; } -ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 watermark) const +ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 w_start, UInt32 w_end) const { auto res_query = std::make_shared(); auto select = std::make_shared(); @@ -1093,10 +1186,24 @@ ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 watermark) const table_expr->database_and_table_name = createTableIdentifier(inner_table_id.database_name, inner_table_id.table_name); table_expr->children.push_back(table_expr->database_and_table_name); - auto func_tuple - = makeASTFunction("tupleElement", std::make_shared(window_column_name), std::make_shared(Field{UInt8(2)})); - auto func_equals = makeASTFunction("equals", func_tuple, std::make_shared(watermark)); - res_query->setExpression(ASTSelectQuery::Expression::WHERE, func_equals); + if (is_tumble) + { + auto func_tuple + = makeASTFunction("tupleElement", std::make_shared(window_id_name), std::make_shared(Field{UInt8(2)})); + auto func_equals = makeASTFunction("equals", func_tuple, std::make_shared(w_end)); + res_query->setExpression(ASTSelectQuery::Expression::WHERE, func_equals); + } + else + { + auto func_array = makeASTFunction("array"); + while (w_end > w_start) + { + func_array ->arguments->children.push_back(std::make_shared(w_end)); + w_end = addTime(w_end, window_kind, -1 * slice_num_units); + } + auto func_has = makeASTFunction("has", func_array, std::make_shared(window_id_name)); + res_query->setExpression(ASTSelectQuery::Expression::WHERE, func_has); + } return res_query; } @@ -1110,24 +1217,23 @@ StoragePtr & StorageWindowView::getTargetStorage() const BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 watermark) { - Pipes pipes; + UInt32 w_start = addTime(watermark, window_kind, -1 * window_num_units); - auto & storage = getInnerStorage(); - InterpreterSelectQuery fetch(getFetchColumnQuery(watermark), wv_context, storage, SelectQueryOptions(QueryProcessingStage::FetchColumns)); - QueryPipeline pipeline; - BlockInputStreams streams = fetch.executeWithMultipleStreams(pipeline); - for (auto & stream : streams) - pipes.emplace_back(std::make_shared(std::move(stream))); + InterpreterSelectQuery fetch(getFetchColumnQuery(w_start, watermark), wv_context, getInnerStorage(), SelectQueryOptions(QueryProcessingStage::FetchColumns)); + BlockInputStreamPtr in_stream = fetch.execute().in; + + if (!is_tumble) + in_stream = std::make_shared(in_stream, window_column_name, w_start, watermark); + + Pipes pipes; + pipes.emplace_back(std::make_shared(std::move(in_stream))); auto proxy_storage = std::make_shared( - StorageID(wv_context.getCurrentDatabase(), "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); + StorageID(getStorageID().database_name, "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); InterpreterSelectQuery select(getFinalQuery(), wv_context, proxy_storage, QueryProcessingStage::Complete); - BlockInputStreamPtr data = std::make_shared(select.execute().in); + BlockInputStreamPtr data = select.execute().in; - /// Squashing is needed here because the view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). data = std::make_shared( data, global_context.getSettingsRef().min_insert_block_size_rows, global_context.getSettingsRef().min_insert_block_size_bytes); diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 9945a1b0201..1739fed21f9 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -54,6 +54,7 @@ public: private: ASTPtr inner_query; + ASTPtr mergeable_query; ASTPtr final_query; Context global_context; @@ -63,6 +64,7 @@ private: bool is_tumble; // false if is hop std::atomic shutdown_called{false}; mutable Block sample_block; + mutable Block mergeable_header; UInt64 clean_interval; const DateLUTImpl & time_zone; UInt32 max_timestamp = 0; @@ -90,10 +92,12 @@ private: IntervalKind::Kind lateness_kind; Int64 window_num_units; Int64 hop_num_units; + Int64 slice_num_units; Int64 watermark_num_units = 0; Int64 lateness_num_units = 0; + String window_id_name; + String window_id_alias; String window_column_name; - String window_column_alias; String timestamp_column_name; StorageID select_table_id = StorageID::createEmpty(); @@ -111,6 +115,7 @@ private: std::shared_ptr generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name); ASTPtr generateCleanCacheQuery(UInt32 timestamp); + UInt32 addTime(UInt32 time_sec, IntervalKind::Kind kind, Int64 num_units) const; UInt32 getWindowLowerBound(UInt32 time_sec); UInt32 getWindowUpperBound(UInt32 time_sec); @@ -126,8 +131,9 @@ private: static Pipes blocksToPipes(BlocksList & blocks, Block & sample_block); ASTPtr getInnerQuery() const { return inner_query->clone(); } + ASTPtr getMergeableQuery() const { return mergeable_query->clone(); } ASTPtr getFinalQuery() const { return final_query->clone(); } - ASTPtr getFetchColumnQuery(UInt32 watermark) const; + ASTPtr getFetchColumnQuery(UInt32 w_start, UInt32 w_end) const; StoragePtr getParentStorage() const; @@ -137,6 +143,8 @@ private: Block & getHeader() const; + Block & getMergeableHeader() const; + StorageWindowView( const StorageID & table_id_, Context & local_context, diff --git a/src/Storages/WindowView/WatermarkBlockInputStream.h b/src/Storages/WindowView/WatermarkBlockInputStream.h index 3346a8bb493..e9c2f8da028 100644 --- a/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -19,6 +19,7 @@ public: : allowed_lateness(false) , update_timestamp(false) , watermark_specified(false) + , is_tumble(true) , storage(storage_) , window_column_name(window_column_name_) , lateness_upper_bound(0) @@ -36,6 +37,7 @@ public: : allowed_lateness(false) , update_timestamp(false) , watermark_specified(true) + , is_tumble(true) , storage(storage_) , window_column_name(window_column_name_) , lateness_upper_bound(0) @@ -52,6 +54,12 @@ public: return children.back()->getHeader(); } + void setHopWindow() + { + is_tumble = false; + slice_num_units = std::gcd(storage.hop_num_units, storage.window_num_units); + } + void setAllowedLateness(UInt32 upper_bound) { allowed_lateness = true; @@ -72,14 +80,41 @@ protected: return res; auto & column_window = res.getByName(window_column_name).column; - const ColumnTuple & column_tuple = typeid_cast(*column_window); - const ColumnUInt32::Container & wend_data = static_cast(*column_tuple.getColumnPtr(1)).getData(); - for (size_t i = 0; i < wend_data.size(); ++i) + if (is_tumble) { - if (!watermark_specified && wend_data[i] > max_watermark) - max_watermark = wend_data[i]; - if (allowed_lateness && wend_data[i] <= lateness_upper_bound) - late_signals.insert(wend_data[i]); + const ColumnTuple & column_tuple = typeid_cast(*column_window); + const ColumnUInt32::Container & wend_data = static_cast(*column_tuple.getColumnPtr(1)).getData(); + for (size_t i = 0; i < wend_data.size(); ++i) + { + if (!watermark_specified && wend_data[i] > max_watermark) + max_watermark = wend_data[i]; + if (allowed_lateness && wend_data[i] <= lateness_upper_bound) + late_signals.insert(wend_data[i]); + } + } + else + { + const ColumnUInt32::Container & slice_data = static_cast(*column_window).getData(); + for (size_t i = 0; i < slice_data.size(); ++i) + { + UInt32 w_start = storage.addTime(slice_data[i], storage.hop_kind, -1 * slice_num_units); + w_start = storage.getWindowLowerBound(w_start); + UInt32 w_start_latest; + do + { + w_start_latest = w_start; + w_start = storage.addTime(w_start, storage.hop_kind, storage.hop_num_units); + } while (w_start < slice_data[i]); + + UInt32 w_end = storage.addTime(w_start_latest, storage.window_kind, storage.window_num_units); + + if (!watermark_specified && w_end > max_watermark) + { + max_watermark = w_end; + } + if (allowed_lateness && w_end <= lateness_upper_bound) + late_signals.insert(w_end); + } } return res; } @@ -98,11 +133,13 @@ private: bool allowed_lateness; bool update_timestamp; bool watermark_specified; + bool is_tumble; std::set late_signals; StorageWindowView & storage; String window_column_name; UInt32 lateness_upper_bound; UInt32 max_timestamp; UInt32 max_watermark; + Int64 slice_num_units; }; } diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference index afa64024e56..17974774595 100644 --- a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference +++ b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference @@ -11,12 +11,12 @@ CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `TUMBLE(timestamp, toInter CREATE TABLE default.`.inner.wv` (`TUMBLE(____timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` ORDER BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` SETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` SETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree ORDER BY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` SETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv` (`b` Int32, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`b` Int32, `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b) SETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY `arrayJoin(HOP(now(), toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree PARTITION BY `HOP_SLICE(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY `HOP_SLICE(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` SETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference b/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference index d35af28a9d5..587f3ec3364 100644 --- a/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference +++ b/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference @@ -11,12 +11,12 @@ CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `TUMBLE(timestamp, toInter CREATE TABLE default.`.inner.wv` (`TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))` ORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY tuple(`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY tuple(`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`) SETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY tuple(`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY tuple(`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`) SETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv` (`b` Int32, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `b` Int32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, b) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`b` Int32, `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `b` Int32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b) SETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv` (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` Tuple(DateTime, DateTime), `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))` ORDER BY (`arrayJoin(HOP(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\')))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`plus(a, b)` Int64, `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv` (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, `plus(a, b)` Int64, `count(a)` AggregateFunction(count, Int32)) ENGINE = AggregatingMergeTree() PRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` ORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`) SETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index 544fa875dfe..93fa40adf9c 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -9,7 +9,7 @@ 2020-01-10 00:00:00 2020-01-10 00:00:00 --HOP-- -[('2020-01-07 00:00:00','2020-01-10 00:00:00'),('2020-01-08 00:00:00','2020-01-11 00:00:00'),('2020-01-09 00:00:00','2020-01-12 00:00:00')] +('2020-01-07 00:00:00','2020-01-10 00:00:00') 2020-01-07 00:00:00 2020-01-06 00:00:00 2020-01-07 00:00:00 From f3c1474dc427d1f66a7d865e023d258e570b802a Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 9 Jun 2020 19:37:33 +0800 Subject: [PATCH 049/609] fix code style --- .../WindowView/ReplaceWindowColumnBlockInputStream.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h b/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h index 21e77761238..77c3c455197 100644 --- a/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h +++ b/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h @@ -40,7 +40,7 @@ public: break; } } - if(position < 0) + if (position < 0) throw Exception("Not found column HOP_SLICE", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 7ef6575569b..803fd0735c9 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -701,7 +701,7 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) { fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; - if(is_tumble) + if (is_tumble) max_watermark = addTime(max_watermark, window_kind, window_num_units); else max_watermark = addTime(max_watermark, hop_kind, hop_num_units); @@ -715,7 +715,7 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) { fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; - if(is_tumble) + if (is_tumble) { max_watermark = addTime(max_watermark, window_kind, window_num_units); max_watermark_bias = addTime(max_watermark, window_kind, window_num_units); @@ -1228,7 +1228,7 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma auto proxy_storage = std::make_shared( StorageID(getStorageID().database_name, "WindowViewProxyStorage"), getParentStorage()->getColumns(), std::move(pipes), QueryProcessingStage::WithMergeableState); - + SelectQueryOptions query_options(QueryProcessingStage::Complete); query_options.ignore_limits = true; query_options.ignore_quota = true; From ea0d4ed8d797ed46cf1d43f4a8726f01f350b6b6 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 9 Jun 2020 23:57:47 +0800 Subject: [PATCH 050/609] fix clang build --- src/Functions/FunctionsWindow.h | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 14 +++++++------- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 1d97fec8c81..c6f38e97137 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -507,7 +507,7 @@ namespace return std::make_shared(); } - static ColumnPtr + [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 803fd0735c9..980fb4f2e92 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -131,7 +131,7 @@ namespace { using TypeToVisit = ASTFunction; - void visit(const ASTFunction & node, ASTPtr & node_ptr) + static void visit(const ASTFunction & node, ASTPtr & node_ptr) { if (node.name == "HOP_SLICE") std::static_pointer_cast(node_ptr)->name = "HOP"; @@ -163,7 +163,7 @@ namespace { using TypeToVisit = ASTFunction; - void visit(ASTFunction & node, ASTPtr & node_ptr) + static void visit(ASTFunction & node, ASTPtr & node_ptr) { if (node.name == "HOP") std::static_pointer_cast(node_ptr)->name = "HOP_SLICE"; @@ -536,13 +536,13 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery } if (storage->primary_key) { - auto primary_key_ = storage->primary_key->clone(); + auto tmp_primary_key = storage->primary_key->clone(); if (is_time_column_func_now) - time_now_visitor.visit(primary_key_); + time_now_visitor.visit(tmp_primary_key); if (!is_tumble) - func_hop_visitor.visit(primary_key_); - visitor.visit(primary_key_); - new_storage->set(new_storage->primary_key, primary_key_); + func_hop_visitor.visit(tmp_primary_key); + visitor.visit(tmp_primary_key); + new_storage->set(new_storage->primary_key, tmp_primary_key); } if (storage->order_by) { From 985a648fe6e28051acde9b99fcc09002bbc56463 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 9 Jun 2020 23:58:27 +0800 Subject: [PATCH 051/609] code reuse --- src/Storages/WindowView/WatermarkBlockInputStream.h | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/src/Storages/WindowView/WatermarkBlockInputStream.h b/src/Storages/WindowView/WatermarkBlockInputStream.h index e9c2f8da028..61638857462 100644 --- a/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -16,17 +16,9 @@ public: BlockInputStreamPtr input_, StorageWindowView& storage_, String& window_column_name_) - : allowed_lateness(false) - , update_timestamp(false) - , watermark_specified(false) - , is_tumble(true) - , storage(storage_) - , window_column_name(window_column_name_) - , lateness_upper_bound(0) - , max_timestamp(0) - , max_watermark(0) + : WatermarkBlockInputStream(input_, storage_, window_column_name_, 0) { - children.push_back(input_); + watermark_specified = false; } WatermarkBlockInputStream( @@ -43,6 +35,7 @@ public: , lateness_upper_bound(0) , max_timestamp(0) , max_watermark(max_watermark_) + , slice_num_units(0) { children.push_back(input_); } From 7b7032be8df8dc44ad71cdf4ad537f5f5847923d Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 17 Jun 2020 23:06:19 +0800 Subject: [PATCH 052/609] rename HOP_SLICE to WINDOW_ID --- src/Functions/FunctionsWindow.cpp | 2 +- src/Functions/FunctionsWindow.h | 44 +++++++++++++------ .../ReplaceWindowColumnBlockInputStream.h | 6 +-- src/Storages/WindowView/StorageWindowView.cpp | 38 ++++++++++------ ...7_window_view_inner_table_parser.reference | 10 ++--- ...ndow_view_inner_table_generation.reference | 12 ++--- 6 files changed, 68 insertions(+), 44 deletions(-) diff --git a/src/Functions/FunctionsWindow.cpp b/src/Functions/FunctionsWindow.cpp index 063e39992f5..11dc5981a5b 100644 --- a/src/Functions/FunctionsWindow.cpp +++ b/src/Functions/FunctionsWindow.cpp @@ -7,10 +7,10 @@ void registerFunctionsWindow(FunctionFactory& factory) { factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index c6f38e97137..e8ff40513b7 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -48,9 +48,9 @@ enum WindowFunctionName TUMBLE_START, TUMBLE_END, HOP, - HOP_SLICE, HOP_START, - HOP_END + HOP_END, + WINDOW_ID }; namespace ErrorCodes { @@ -477,26 +477,19 @@ namespace }; template <> - struct WindowImpl + struct WindowImpl { - static constexpr auto name = "HOP_SLICE"; + static constexpr auto name = "WINDOW_ID"; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - if (arguments.size() != 3 && arguments.size() != 4) - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 3.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception( "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); if (!WhichDataType(arguments[1].type).isInterval()) throw Exception( "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[2].type).isInterval()) + if (arguments.size() >= 3 && !WhichDataType(arguments[2].type).isInterval()) throw Exception( "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) @@ -504,11 +497,16 @@ namespace "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name + ". This argument is optional and must be a constant string with timezone name", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (arguments.size() > 4) + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should not larger than 4.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return std::make_shared(); } [[maybe_unused]] static ColumnPtr - dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) + dispatchForHopColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { const auto & time_column = block.getByPosition(arguments[0]); const auto & hop_interval_column = block.getByPosition(arguments[1]); @@ -592,6 +590,24 @@ namespace } return end; } + + [[maybe_unused]] static ColumnPtr + dispatchForTumbleColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) + { + ColumnPtr column = WindowImpl::dispatchForColumns(block, arguments, function_name); + return executeWindowBound(column, 1, function_name); + } + + [[maybe_unused]] static ColumnPtr + dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) + { + const auto & third_column = block.getByPosition(arguments[2]); + + if (arguments.size() == 2 || (arguments.size() == 3 && WhichDataType(third_column.type).isString())) + return dispatchForTumbleColumns(block, arguments, function_name); + else + return dispatchForHopColumns(block, arguments, function_name); + } }; template <> @@ -712,7 +728,7 @@ using FunctionTumble = FunctionWindow; using FunctionTumbleStart = FunctionWindow; using FunctionTumbleEnd = FunctionWindow; using FunctionHop = FunctionWindow; -using FunctionHopSlice = FunctionWindow; +using FunctionWindowId = FunctionWindow; using FunctionHopStart = FunctionWindow; using FunctionHopEnd = FunctionWindow; } diff --git a/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h b/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h index 77c3c455197..90d1c8e177c 100644 --- a/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h +++ b/src/Storages/WindowView/ReplaceWindowColumnBlockInputStream.h @@ -33,15 +33,14 @@ public: Block header = children.back()->getHeader(); for (auto & column_ : header.getColumnsWithTypeAndName()) { - if (startsWith(column_.name, "HOP_SLICE")) + if (startsWith(column_.name, "WINDOW_ID")) { position = header.getPositionByName(column_.name); - hop_slice_name = column_.name; break; } } if (position < 0) - throw Exception("Not found column HOP_SLICE", ErrorCodes::LOGICAL_ERROR); + throw Exception("Not found column WINDOW_ID", ErrorCodes::LOGICAL_ERROR); } String getName() const override { return "ReplaceWindowColumn"; } @@ -68,7 +67,6 @@ protected: private: String window_column_name; - String hop_slice_name; DataTypePtr window_column_type; int position; Tuple window_value; diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 980fb4f2e92..12768378228 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -105,7 +105,7 @@ namespace else if (node.name == "HOP") { is_hop = true; - std::static_pointer_cast(node_ptr)->name = "HOP_SLICE"; + std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; if (!window_function) { window_function = node.clone(); @@ -127,14 +127,20 @@ namespace } }; - struct ReplaceFunctionHopSliceVisitorData + struct ReplaceFunctionWindowIdVisitorData { using TypeToVisit = ASTFunction; + bool is_tumble; - static void visit(const ASTFunction & node, ASTPtr & node_ptr) + void visit(const ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "HOP_SLICE") - std::static_pointer_cast(node_ptr)->name = "HOP"; + if (node.name == "WINDOW_ID") + { + if (is_tumble) + std::static_pointer_cast(node_ptr)->name = "TUMBLE"; + else + std::static_pointer_cast(node_ptr)->name = "HOP"; + } } }; @@ -147,7 +153,7 @@ namespace void visit(ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "TUMBLE" || node.name == "HOP" || node.name == "HOP_SLICE") + if (node.name == "TUMBLE" || node.name == "HOP" || node.name == "WINDOW_ID") { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") { @@ -166,7 +172,7 @@ namespace static void visit(ASTFunction & node, ASTPtr & node_ptr) { if (node.name == "HOP") - std::static_pointer_cast(node_ptr)->name = "HOP_SLICE"; + std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; } }; @@ -460,7 +466,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery columns_list->children.push_back(column_window); } - for (auto & column : t_sample_block.getColumnsWithTypeAndName()) + for (const auto & column : t_sample_block.getColumnsWithTypeAndName()) { ParserIdentifierWithOptionalParameters parser; String sql = column.type->getName(); @@ -672,7 +678,7 @@ inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) inline void StorageWindowView::addFireSignal(std::set & signals) { std::lock_guard lock(fire_signal_mutex); - for (auto & signal : signals) + for (const auto & signal : signals) fire_signal.push_back(signal); fire_signal_condition.notify_all(); } @@ -866,8 +872,9 @@ StorageWindowView::StorageWindowView( final_query = mergeable_query->clone(); - ReplaceFunctionHopSliceVisitorData final_query_data; - InDepthNodeVisitor, true>(final_query_data).visit(final_query); + ReplaceFunctionWindowIdVisitorData final_query_data; + final_query_data.is_tumble = is_tumble; + InDepthNodeVisitor, true>(final_query_data).visit(final_query); is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; is_watermark_ascending = query.is_watermark_ascending; @@ -960,7 +967,10 @@ StorageWindowView::StorageWindowView( inner_table_id = inner_storage->getStorageID(); } - window_column_name = std::regex_replace(window_id_name, std::regex("HOP_SLICE"), "HOP"); + if (is_tumble) + window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "TUMBLE"); + else + window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); clean_cache_task = wv_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); if (is_proctime) @@ -1081,9 +1091,9 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con UInt32 t_max_timstamp = 0; if (!window_view.is_tumble || window_view.is_watermark_bounded || window_view.allowed_lateness) { - auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; + const auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); - for (auto& timestamp : timestamp_data) + for (const auto & timestamp : timestamp_data) { if (timestamp > t_max_timstamp) t_max_timstamp = timestamp; diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference index 441eb351935..4863417b5c7 100644 --- a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference +++ b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference @@ -11,12 +11,12 @@ CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `TUMBLE(times CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `HOP_SLICE(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `HOP_SLICE(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference b/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference index 697307e4cd2..fc183ebec09 100644 --- a/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference +++ b/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference @@ -11,12 +11,12 @@ CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `TUMBLE(times CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `b` Int32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `b` Int32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`HOP_SLICE(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 From 2f1e5c015f6a3788b1c5ac06c24bf5e388a10027 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 17 Jun 2020 23:22:17 +0800 Subject: [PATCH 053/609] remove unused code --- src/Functions/FunctionsWindow.h | 28 ++----------------- src/Storages/WindowView/StorageWindowView.cpp | 1 - 2 files changed, 2 insertions(+), 27 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index e8ff40513b7..9785a3a7c23 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -1,9 +1,7 @@ #pragma once -#include #include #include -#include #include #include #include @@ -152,32 +150,10 @@ namespace ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); return col_tuple->getColumnPtr(index); } - else if (const ColumnArray * col_array = checkAndGetColumn(column.get()); col_array) - { - const ColumnTuple * col_tuple_ = checkAndGetColumn(&col_array->getData()); - if (!col_tuple_) - throw Exception( - "Illegal column for first argument of function " + function_name + ". Must be a Tuple or Array(Tuple)", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto & bound_column = col_tuple_->getColumn(index); - const ColumnUInt32::Container & bound_data = static_cast(bound_column).getData(); - const auto & column_offsets = col_array->getOffsets(); - auto res = ColumnUInt32::create(); - ColumnUInt32::Container & res_data = res->getData(); - res_data.reserve(column_offsets.size()); - IColumn::Offset current_offset = 0; - for (size_t i = 0; i < column_offsets.size(); ++i) - { - res_data.push_back(bound_data[current_offset]); - current_offset = column_offsets[i]; - } - return res; - } else { throw Exception( - "Illegal column for first argument of function " + function_name + ". Must be a Tuple or Array(Tuple)", + "Illegal column for first argument of function " + function_name + ". Must be Tuple", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } } @@ -622,7 +598,7 @@ namespace auto type_ = WhichDataType(arguments[0].type); if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Illegal type of first argument of function " + function_name + " should be Tuple, Array, UInt8 or UInt32", + "Illegal type of first argument of function " + function_name + " should be Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); return std::make_shared(); } diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 12768378228..5c921c0157f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -7,7 +7,6 @@ #include #include #include -#include #include #include #include From dd3ee6af3599875c91231e4899752ed736ef6372 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 19 Jun 2020 17:35:18 +0800 Subject: [PATCH 054/609] Unified the processing logic of TUMBLE and HOP window --- src/Functions/FunctionsWindow.h | 17 ++- src/Storages/WindowView/StorageWindowView.cpp | 141 ++++++------------ .../WindowView/WatermarkBlockInputStream.h | 64 +------- ...7_window_view_inner_table_parser.reference | 10 +- ...=> 01048_window_view_sql_parser.reference} | 12 +- ...n.sql => 01048_window_view_sql_parser.sql} | 0 6 files changed, 73 insertions(+), 171 deletions(-) rename tests/queries/0_stateless/{01048_window_view_inner_table_generation.reference => 01048_window_view_sql_parser.reference} (57%) rename tests/queries/0_stateless/{01048_window_view_inner_table_generation.sql => 01048_window_view_sql_parser.sql} (100%) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 9785a3a7c23..55c8bd55389 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -261,9 +261,10 @@ namespace { if (arguments.size() == 1) { - if (!WhichDataType(arguments[0].type).isTuple()) + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Illegal type of first argument of function " + function_name + " should be tuple", ErrorCodes::ILLEGAL_COLUMN); + "Illegal type of first argument of function " + function_name + " should be tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); return std::make_shared(); } else if (arguments.size() == 2 || arguments.size() == 3) @@ -577,12 +578,16 @@ namespace [[maybe_unused]] static ColumnPtr dispatchForColumns(Block & block, const ColumnNumbers & arguments, const String & function_name) { - const auto & third_column = block.getByPosition(arguments[2]); - - if (arguments.size() == 2 || (arguments.size() == 3 && WhichDataType(third_column.type).isString())) + if (arguments.size() == 2) return dispatchForTumbleColumns(block, arguments, function_name); else - return dispatchForHopColumns(block, arguments, function_name); + { + const auto & third_column = block.getByPosition(arguments[2]); + if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) + return dispatchForTumbleColumns(block, arguments, function_name); + else + return dispatchForHopColumns(block, arguments, function_name); + } } }; diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 5c921c0157f..68848084619 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1,14 +1,11 @@ #include #include -#include #include #include #include -#include #include #include #include -#include #include #include #include @@ -32,7 +29,6 @@ #include #include #include -#include #include #include #include @@ -81,11 +77,13 @@ namespace void visit(const ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "TUMBLE") + if (node.name == "TUMBLE" || node.name == "HOP") { + is_tumble = node.name == "TUMBLE"; + is_hop = node.name == "HOP"; if (!window_function) { - is_tumble = true; + std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; window_id_name = node.getColumnName(); window_id_alias = node.alias; window_function = node.clone(); @@ -101,32 +99,10 @@ namespace throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); } } - else if (node.name == "HOP") - { - is_hop = true; - std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; - if (!window_function) - { - window_function = node.clone(); - window_function->setAlias(""); - serialized_window_function = serializeAST(*window_function); - } - else if (serializeAST(node) != serializeAST(*window_function)) - { - auto temp_node = node.clone(); - temp_node->setAlias(""); - if (serializeAST(*temp_node) != serialized_window_function) - throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); - } - - timestamp_column_name = node.arguments->children[0]->getColumnName(); - window_id_name = node.getColumnName(); - window_id_alias = node.alias; - } } }; - struct ReplaceFunctionWindowIdVisitorData + struct ReplaceWindowIdVisitorData { using TypeToVisit = ASTFunction; bool is_tumble; @@ -152,7 +128,7 @@ namespace void visit(ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "TUMBLE" || node.name == "HOP" || node.name == "WINDOW_ID") + if (node.name == "WINDOW_ID") { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") { @@ -164,13 +140,13 @@ namespace } }; - struct ReplaceFunctionHopVisitorData + struct ReplaceFunctionWindowVisitorData { using TypeToVisit = ASTFunction; static void visit(ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "HOP") + if (node.name == "HOP" || node.name == "TUMBLE") std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; } }; @@ -279,14 +255,7 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa ASTPtr StorageWindowView::generateCleanCacheQuery(UInt32 timestamp) { ASTPtr function_equal; - if (is_tumble) - { - auto function_tuple = makeASTFunction( - "tupleElement", std::make_shared(window_id_name), std::make_shared(Field{UInt8(2)})); - function_equal = makeASTFunction("less", function_tuple, std::make_shared(timestamp)); - } - else - function_equal = makeASTFunction("less", std::make_shared(window_id_name), std::make_shared(timestamp)); + function_equal = makeASTFunction("less", std::make_shared(window_id_name), std::make_shared(timestamp)); auto alter_command = std::make_shared(); alter_command->type = ASTAlterCommand::DELETE; @@ -457,11 +426,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery { auto column_window = std::make_shared(); column_window->name = window_id_name; - if (is_tumble) - column_window->type - = makeASTFunction("Tuple", std::make_shared("DateTime"), std::make_shared("DateTime")); - else - column_window->type = std::make_shared("UInt32"); + column_window->type = std::make_shared("UInt32"); columns_list->children.push_back(column_window); } @@ -483,9 +448,9 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery ReplaceFunctionWindowMatcher::Visitor visitor(query_data); ReplaceFunctionNowVisitorData time_now_data; - ReplaceFunctionHopVisitorData func_hop_data; + ReplaceFunctionWindowVisitorData func_hop_data; InDepthNodeVisitor, true> time_now_visitor(time_now_data); - InDepthNodeVisitor, true> func_hop_visitor(func_hop_data); + InDepthNodeVisitor, true> func_window_visitor(func_hop_data); auto new_storage = std::make_shared(); if (storage == nullptr) @@ -534,8 +499,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto partition_by = storage->partition_by->clone(); if (is_time_column_func_now) time_now_visitor.visit(partition_by); - if (!is_tumble) - func_hop_visitor.visit(partition_by); + func_window_visitor.visit(partition_by); visitor.visit(partition_by); new_storage->set(new_storage->partition_by, partition_by); } @@ -544,8 +508,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto tmp_primary_key = storage->primary_key->clone(); if (is_time_column_func_now) time_now_visitor.visit(tmp_primary_key); - if (!is_tumble) - func_hop_visitor.visit(tmp_primary_key); + func_window_visitor.visit(tmp_primary_key); visitor.visit(tmp_primary_key); new_storage->set(new_storage->primary_key, tmp_primary_key); } @@ -554,8 +517,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto order_by = storage->order_by->clone(); if (is_time_column_func_now) time_now_visitor.visit(order_by); - if (!is_tumble) - func_hop_visitor.visit(order_by); + func_window_visitor.visit(order_by); visitor.visit(order_by); new_storage->set(new_storage->order_by, order_by); } @@ -564,8 +526,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto sample_by = storage->sample_by->clone(); if (is_time_column_func_now) time_now_visitor.visit(sample_by); - if (!is_tumble) - func_hop_visitor.visit(sample_by); + func_window_visitor.visit(sample_by); visitor.visit(sample_by); new_storage->set(new_storage->sample_by, sample_by); } @@ -694,7 +655,7 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) std::lock_guard lock(fire_signal_mutex); if (max_watermark == 0) { - max_watermark = watermark; + max_watermark = getWindowUpperBound(watermark - 1); return; } @@ -706,10 +667,8 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) { fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; - if (is_tumble) - max_watermark = addTime(max_watermark, window_kind, window_num_units); - else - max_watermark = addTime(max_watermark, hop_kind, hop_num_units); + max_watermark + = is_tumble ? addTime(max_watermark, window_kind, window_num_units) : addTime(max_watermark, hop_kind, hop_num_units); } } else // strictly || bounded @@ -871,9 +830,9 @@ StorageWindowView::StorageWindowView( final_query = mergeable_query->clone(); - ReplaceFunctionWindowIdVisitorData final_query_data; + ReplaceWindowIdVisitorData final_query_data; final_query_data.is_tumble = is_tumble; - InDepthNodeVisitor, true>(final_query_data).visit(final_query); + InDepthNodeVisitor, true>(final_query_data).visit(final_query); is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; is_watermark_ascending = query.is_watermark_ascending; @@ -1051,19 +1010,19 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con UInt32 t_max_fired_watermark = 0; if (window_view.allowed_lateness) { - UInt32 t_max_timstamp = 0; + UInt32 t_max_timestamp = 0; UInt32 t_max_watermark = 0; { std::lock_guard lock(window_view.fire_signal_mutex); t_max_fired_watermark = window_view.max_fired_watermark; t_max_watermark = window_view.max_watermark; - t_max_timstamp = window_view.max_timestamp; + t_max_timestamp = window_view.max_timestamp; } - if (t_max_timstamp!= 0) + if (t_max_timestamp!= 0) { UInt32 lateness_bound - = window_view.addTime(t_max_timstamp, window_view.lateness_kind, -1 * window_view.lateness_num_units); + = window_view.addTime(t_max_timestamp, window_view.lateness_kind, -1 * window_view.lateness_num_units); if (window_view.is_watermark_bounded) { UInt32 watermark_lower_bound = window_view.is_tumble @@ -1087,37 +1046,28 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con } } - UInt32 t_max_timstamp = 0; - if (!window_view.is_tumble || window_view.is_watermark_bounded || window_view.allowed_lateness) - { - const auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; - const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); - for (const auto & timestamp : timestamp_data) - { - if (timestamp > t_max_timstamp) - t_max_timstamp = timestamp; - } - } - InterpreterSelectQuery select_block(window_view.getMergeableQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().getInputStream(); source_stream = std::make_shared( source_stream, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); - - if (window_view.is_tumble) - { - source_stream = std::make_shared(source_stream, window_view, window_view.window_id_name); - } - else - { - source_stream - = std::make_shared(source_stream, window_view, window_view.window_id_name, window_view.getWindowUpperBound(t_max_timstamp)); - std::static_pointer_cast(source_stream)->setHopWindow(); - } + source_stream = std::make_shared(source_stream, window_view, window_view.window_id_name); if (window_view.is_watermark_bounded || window_view.allowed_lateness) - std::static_pointer_cast(source_stream)->setMaxTimestamp(t_max_timstamp); + { + UInt32 t_max_timestamp = 0; + if (window_view.is_watermark_bounded || window_view.allowed_lateness) + { + const auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; + const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); + for (const auto & timestamp : timestamp_data) + { + if (timestamp > t_max_timestamp) + t_max_timestamp = timestamp; + } + } + std::static_pointer_cast(source_stream)->setMaxTimestamp(t_max_timestamp); + } if (window_view.allowed_lateness && t_max_fired_watermark != 0) std::static_pointer_cast(source_stream)->setAllowedLateness(t_max_fired_watermark); @@ -1195,10 +1145,8 @@ ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 w_start, UInt32 w_end) cons if (is_tumble) { - auto func_tuple - = makeASTFunction("tupleElement", std::make_shared(window_id_name), std::make_shared(Field{UInt8(2)})); - auto func_equals = makeASTFunction("equals", func_tuple, std::make_shared(w_end)); - res_query->setExpression(ASTSelectQuery::Expression::WHERE, func_equals); + auto func_equals = makeASTFunction("equals", std::make_shared(window_id_name), std::make_shared(w_end)); + res_query->setExpression(ASTSelectQuery::Expression::PREWHERE, func_equals); } else { @@ -1209,7 +1157,7 @@ ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 w_start, UInt32 w_end) cons w_end = addTime(w_end, window_kind, -1 * slice_num_units); } auto func_has = makeASTFunction("has", func_array, std::make_shared(window_id_name)); - res_query->setExpression(ASTSelectQuery::Expression::WHERE, func_has); + res_query->setExpression(ASTSelectQuery::Expression::PREWHERE, func_has); } return res_query; @@ -1229,8 +1177,7 @@ BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 waterma InterpreterSelectQuery fetch(getFetchColumnQuery(w_start, watermark), *wv_context, getInnerStorage(), SelectQueryOptions(QueryProcessingStage::FetchColumns)); BlockInputStreamPtr in_stream = fetch.execute().getInputStream(); - if (!is_tumble) - in_stream = std::make_shared(in_stream, window_column_name, w_start, watermark); + in_stream = std::make_shared(in_stream, window_column_name, w_start, watermark); Pipes pipes; pipes.emplace_back(std::make_shared(std::move(in_stream))); diff --git a/src/Storages/WindowView/WatermarkBlockInputStream.h b/src/Storages/WindowView/WatermarkBlockInputStream.h index 61638857462..46e6a126d08 100644 --- a/src/Storages/WindowView/WatermarkBlockInputStream.h +++ b/src/Storages/WindowView/WatermarkBlockInputStream.h @@ -16,26 +16,13 @@ public: BlockInputStreamPtr input_, StorageWindowView& storage_, String& window_column_name_) - : WatermarkBlockInputStream(input_, storage_, window_column_name_, 0) - { - watermark_specified = false; - } - - WatermarkBlockInputStream( - BlockInputStreamPtr input_, - StorageWindowView& storage_, - String& window_column_name_, - UInt32 max_watermark_) : allowed_lateness(false) , update_timestamp(false) - , watermark_specified(true) - , is_tumble(true) , storage(storage_) , window_column_name(window_column_name_) , lateness_upper_bound(0) , max_timestamp(0) - , max_watermark(max_watermark_) - , slice_num_units(0) + , max_watermark(0) { children.push_back(input_); } @@ -47,12 +34,6 @@ public: return children.back()->getHeader(); } - void setHopWindow() - { - is_tumble = false; - slice_num_units = std::gcd(storage.hop_num_units, storage.window_num_units); - } - void setAllowedLateness(UInt32 upper_bound) { allowed_lateness = true; @@ -73,41 +54,13 @@ protected: return res; auto & column_window = res.getByName(window_column_name).column; - if (is_tumble) + const ColumnUInt32::Container & wend_data = static_cast(*column_window).getData(); + for (size_t i = 0; i < wend_data.size(); ++i) { - const ColumnTuple & column_tuple = typeid_cast(*column_window); - const ColumnUInt32::Container & wend_data = static_cast(*column_tuple.getColumnPtr(1)).getData(); - for (size_t i = 0; i < wend_data.size(); ++i) - { - if (!watermark_specified && wend_data[i] > max_watermark) - max_watermark = wend_data[i]; - if (allowed_lateness && wend_data[i] <= lateness_upper_bound) - late_signals.insert(wend_data[i]); - } - } - else - { - const ColumnUInt32::Container & slice_data = static_cast(*column_window).getData(); - for (size_t i = 0; i < slice_data.size(); ++i) - { - UInt32 w_start = storage.addTime(slice_data[i], storage.hop_kind, -1 * slice_num_units); - w_start = storage.getWindowLowerBound(w_start); - UInt32 w_start_latest; - do - { - w_start_latest = w_start; - w_start = storage.addTime(w_start, storage.hop_kind, storage.hop_num_units); - } while (w_start < slice_data[i]); - - UInt32 w_end = storage.addTime(w_start_latest, storage.window_kind, storage.window_num_units); - - if (!watermark_specified && w_end > max_watermark) - { - max_watermark = w_end; - } - if (allowed_lateness && w_end <= lateness_upper_bound) - late_signals.insert(w_end); - } + if (wend_data[i] > max_watermark) + max_watermark = wend_data[i]; + if (allowed_lateness && wend_data[i] <= lateness_upper_bound) + late_signals.insert(wend_data[i]); } return res; } @@ -125,14 +78,11 @@ protected: private: bool allowed_lateness; bool update_timestamp; - bool watermark_specified; - bool is_tumble; std::set late_signals; StorageWindowView & storage; String window_column_name; UInt32 lateness_upper_bound; UInt32 max_timestamp; UInt32 max_watermark; - Int64 slice_num_units; }; } diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference index 4863417b5c7..e96e5bb5ade 100644 --- a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference +++ b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference @@ -1,14 +1,14 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(____timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `TUMBLE(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference b/tests/queries/0_stateless/01048_window_view_sql_parser.reference similarity index 57% rename from tests/queries/0_stateless/01048_window_view_inner_table_generation.reference rename to tests/queries/0_stateless/01048_window_view_sql_parser.reference index fc183ebec09..da0cb55c523 100644 --- a/tests/queries/0_stateless/01048_window_view_inner_table_generation.reference +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.reference @@ -1,14 +1,14 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`TUMBLE(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`TUMBLE(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `b` Int32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `b` Int32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `TUMBLE(timestamp, toIntervalSecond(\'1\'))` Tuple(DateTime, DateTime), \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `TUMBLE(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`TUMBLE(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_inner_table_generation.sql b/tests/queries/0_stateless/01048_window_view_sql_parser.sql similarity index 100% rename from tests/queries/0_stateless/01048_window_view_inner_table_generation.sql rename to tests/queries/0_stateless/01048_window_view_sql_parser.sql From 20aab89ef3082a0c0e6f06fe72c0079d1856b033 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 20 Jun 2020 00:29:44 +0800 Subject: [PATCH 055/609] update tests --- src/Storages/WindowView/StorageWindowView.cpp | 2 +- ...7_window_view_inner_table_parser.reference | 20 ++++++++-------- .../01048_window_view_sql_parser.reference | 24 +++++++++---------- 3 files changed, 23 insertions(+), 23 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 68848084619..1f437216cc8 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -107,7 +107,7 @@ namespace using TypeToVisit = ASTFunction; bool is_tumble; - void visit(const ASTFunction & node, ASTPtr & node_ptr) + void visit(const ASTFunction & node, ASTPtr & node_ptr) const { if (node.name == "WINDOW_ID") { diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference index e96e5bb5ade..184a6c5ba51 100644 --- a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference +++ b/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference @@ -1,22 +1,22 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.reference b/tests/queries/0_stateless/01048_window_view_sql_parser.reference index da0cb55c523..31bec6c3bb5 100644 --- a/tests/queries/0_stateless/01048_window_view_sql_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.reference @@ -1,22 +1,22 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `b` Int32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32, \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `b` Int32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64, \n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32, \n `plus(a, b)` Int64, \n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 From 3d19d2d69d433016751691ebd111cb9bb39c5d37 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 22 Jun 2020 11:39:40 +0800 Subject: [PATCH 056/609] fix bad arguments error in window_id --- src/Functions/FunctionsWindow.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 55c8bd55389..4fb444f30fa 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -460,6 +460,13 @@ namespace [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { + if (arguments.size() < 2) + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should not smaller than 2.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } if (!WhichDataType(arguments[0].type).isDateTime()) throw Exception( "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); From 14150d452befd7b3c56c1ff04379a7ef1c3b4317 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 22 Jun 2020 20:51:32 +0800 Subject: [PATCH 057/609] update tests --- .../01062_window_view_event_asc_hop_watch.py | 17 ++++++++++------- ...01065_window_view_event_bounded_hop_watch.py | 16 +++++++++------- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py b/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py index 028cc62e10f..2a7c9711d2a 100755 --- a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py +++ b/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py @@ -34,17 +34,20 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('WATCH test.wv') + client1.expect(r'Progress') client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00');") - client1.expect('Progress: 0.00 rows.*\)') + client2.expect(prompt) client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05');") - client1.expect('1*1990-01-01 12:00:02' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') + client2.expect(prompt) + client1.expect('1*' + end_of_block) client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06');") - client1.expect('Progress: 1.00 rows.*\)') + client2.expect(prompt) + client1.expect(r'Progress') client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10');") - client1.expect('1*1990-01-01 12:00:06' + end_of_block) - client1.expect('2*1990-01-01 12:00:08' + end_of_block) - client1.expect('Progress: 3.00 rows.*\)') + client2.expect(prompt) + client1.expect('1*' + end_of_block) + client1.expect('2*' + end_of_block) + client1.expect(r'Progress') # send Ctrl-C client1.send('\x03', eol='') diff --git a/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py b/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py index d9d1603b922..5bf6ff057a9 100755 --- a/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py +++ b/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py @@ -34,17 +34,19 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('WATCH test.wv') + client1.expect(r'Progress') client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:00');") - client1.expect('Progress: 0.00 rows.*\)') + client2.expect(prompt) client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:05');") - client1.expect('1*1990-01-01 12:00:02' + end_of_block) - client1.expect('Progress: 1.00 rows.*\)') + client2.expect(prompt) + client1.expect('1*' + end_of_block) client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:06');") - client1.expect('Progress: 1.00 rows.*\)') + client2.expect(prompt) + client1.expect(r'Progress') client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10');") - client1.expect('1*1990-01-01 12:00:06' + end_of_block) - client1.expect('2*1990-01-01 12:00:08' + end_of_block) - client1.expect('Progress: 3.00 rows.*\)') + client2.expect(prompt) + client1.expect('1*' + end_of_block) + client1.expect('2*' + end_of_block) # send Ctrl-C client1.send('\x03', eol='') From fcdb930c64c0c2fcf7cc3f4fd7494695a134a751 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 13 Jul 2020 22:28:31 +0800 Subject: [PATCH 058/609] add column timestamp only when function now is used --- src/Storages/WindowView/StorageWindowView.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 1f437216cc8..f8fe75be4ce 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -996,9 +996,9 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (window_view.is_proctime) { fire_signal_lock = std::shared_lock(window_view.fire_signal_mutex); - UInt32 timestamp_now = std::time(nullptr); - pipe.addSimpleTransform(std::make_shared>( - pipe.getHeader(), std::make_shared(), timestamp_now, "____timestamp")); + if (window_view.is_time_column_func_now) + pipe.addSimpleTransform(std::make_shared>( + pipe.getHeader(), std::make_shared(), std::time(nullptr), "____timestamp")); InterpreterSelectQuery select_block(window_view.getMergeableQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().getInputStream(); From 3e77a5a982ba1f7e782966a071e6c41efa014410 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 13 Jul 2020 22:31:54 +0800 Subject: [PATCH 059/609] rename function cleanCache --- src/Storages/WindowView/StorageWindowView.cpp | 61 ++++++++++--------- src/Storages/WindowView/StorageWindowView.h | 7 ++- 2 files changed, 36 insertions(+), 32 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index f8fe75be4ce..239978b07c4 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -252,10 +252,35 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa DB::ErrorCodes::LOGICAL_ERROR); } -ASTPtr StorageWindowView::generateCleanCacheQuery(UInt32 timestamp) +UInt32 StorageWindowView::getCleanupBound() +{ + UInt32 w_bound; + { + std::lock_guard lock(fire_signal_mutex); + w_bound = max_fired_watermark; + if (w_bound == 0) + return 0; + + if (!is_proctime) + { + if (max_watermark == 0) + return 0; + if (allowed_lateness) + { + UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units); + lateness_bound = getWindowLowerBound(lateness_bound); + if (lateness_bound < w_bound) + w_bound = lateness_bound; + } + } + } + return w_bound; +} + +ASTPtr StorageWindowView::generateCleanupQuery() { ASTPtr function_equal; - function_equal = makeASTFunction("less", std::make_shared(window_id_name), std::make_shared(timestamp)); + function_equal = makeASTFunction("less", std::make_shared(window_id_name), std::make_shared(getCleanupBound())); auto alter_command = std::make_shared(); alter_command->type = ASTAlterCommand::DELETE; @@ -326,31 +351,9 @@ Pipes StorageWindowView::blocksToPipes(BlocksList & blocks, Block & sample_block return pipes; } -inline void StorageWindowView::cleanCache() +inline void StorageWindowView::cleanup() { - UInt32 w_bound; - { - std::lock_guard lock(fire_signal_mutex); - w_bound = max_fired_watermark; - if (w_bound == 0) - return; - - if (!is_proctime) - { - if (max_watermark == 0) - return; - if (allowed_lateness) - { - UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units); - lateness_bound = getWindowLowerBound(lateness_bound); - if (lateness_bound < w_bound) - w_bound = lateness_bound; - } - } - } - - w_bound = addTime(w_bound, window_kind, -1 * window_num_units); - InterpreterAlterQuery alt_query(generateCleanCacheQuery(w_bound), *wv_context); + InterpreterAlterQuery alt_query(generateCleanupQuery(), *wv_context); alt_query.execute(); std::lock_guard lock(fire_signal_mutex); @@ -695,14 +698,14 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) fire_signal_condition.notify_all(); } -void StorageWindowView::threadFuncCleanCache() +void StorageWindowView::threadFuncCleanup() { while (!shutdown_called) { try { sleep(clean_interval); - cleanCache(); + cleanup(); } catch (...) { @@ -930,7 +933,7 @@ StorageWindowView::StorageWindowView( else window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); - clean_cache_task = wv_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanCache(); }); + clean_cache_task = wv_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanup(); }); if (is_proctime) fire_task = wv_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncFireProc(); }); else diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 73b14110cd4..744f7e222c1 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -113,15 +113,16 @@ private: ASTPtr innerQueryParser(ASTSelectQuery & inner_query); std::shared_ptr generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name); - ASTPtr generateCleanCacheQuery(UInt32 timestamp); + UInt32 getCleanupBound(); + ASTPtr generateCleanupQuery(); UInt32 addTime(UInt32 time_sec, IntervalKind::Kind kind, Int64 num_units) const; UInt32 getWindowLowerBound(UInt32 time_sec); UInt32 getWindowUpperBound(UInt32 time_sec); void fire(UInt32 watermark); - void cleanCache(); - void threadFuncCleanCache(); + void cleanup(); + void threadFuncCleanup(); void threadFuncFireProc(); void threadFuncFireEvent(); void addFireSignal(std::set & signals); From 6191f883a58a9af536d75ac6983d38c1a7177701 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 15 Jul 2020 12:52:51 +0800 Subject: [PATCH 060/609] update tests --- .../01047_window_view_inner_table_parser.sql | 10 ++++++++++ .../0_stateless/01048_window_view_sql_parser.sql | 12 ++++++++++++ .../01050_window_view_sql_parser_tumble.sql | 6 ++++++ .../0_stateless/01051_window_view_sql_parser_hop.sql | 6 ++++++ .../01052_window_view_proc_now_tumble_to.sql | 1 + .../01053_window_view_proc_now_hop_to.sql | 1 + .../0_stateless/01054_window_view_proc_tumble_to.sql | 1 + .../0_stateless/01055_window_view_proc_hop_to.sql | 1 + .../0_stateless/01056_window_view_proc_hop_watch.py | 2 ++ .../01057_window_view_event_strict_asc_tumble_to.sql | 1 + .../01058_window_view_event_strict_asc_hop_to.sql | 1 + .../01059_window_view_event_strict_asc_hop_watch.py | 2 ++ .../01060_window_view_event_asc_tumble_to.sql | 1 + .../01061_window_view_event_asc_hop_to.sql | 1 + .../01062_window_view_event_asc_hop_watch.py | 2 ++ .../01063_window_view_event_bounded_tumble_to.sql | 1 + .../01064_window_view_event_bounded_hop_to.sql | 1 + .../01065_window_view_event_bounded_hop_watch.py | 2 ++ ...ndow_view_event_strict_asc_tumble_to_lateness.sql | 1 + ...1067_window_view_event_asc_tumble_to_lateness.sql | 1 + ..._window_view_event_bounded_tumble_to_lateness.sql | 1 + 21 files changed, 55 insertions(+) diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql b/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql index 5df48b10f24..6c607ba689e 100644 --- a/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql +++ b/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql @@ -7,26 +7,31 @@ CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), TUMBLE_END(wid) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---PARTITION---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, TUMBLE(now(), INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; SHOW CREATE TABLE `.inner.wv`; @@ -34,26 +39,31 @@ SHOW CREATE TABLE `.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---PARTITION---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.sql b/tests/queries/0_stateless/01048_window_view_sql_parser.sql index e3b137546e3..7c66fa21a10 100644 --- a/tests/queries/0_stateless/01048_window_view_sql_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.sql @@ -8,29 +8,35 @@ CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, b; SHOW CREATE TABLE `.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); SHOW CREATE TABLE `.inner.wv`; @@ -38,29 +44,35 @@ SHOW CREATE TABLE `.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b; SHOW CREATE TABLE `.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); SHOW CREATE TABLE `.inner.wv`; diff --git a/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql index 37b86314cfa..c8844af9e8d 100644 --- a/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql @@ -6,26 +6,32 @@ CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---WATERMARK---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---With w_end---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WithOut w_end---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WITH---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end, date_time FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WHERE---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---ORDER_BY---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; DROP TABLE mt; diff --git a/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql index eae0de10c4b..7c04abeefc5 100644 --- a/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql @@ -6,26 +6,32 @@ CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---WATERMARK---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---With w_end---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WithOut w_end---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WITH---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end, date_time FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WHERE---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---ORDER_BY---'; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; DROP TABLE mt; diff --git a/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql b/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql index 8bc5dfb5c04..d73f46260e0 100755 --- a/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql b/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql index b14db138406..50440dd18e0 100755 --- a/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql index ef84ee95079..aa25952d768 100755 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql index acf8e8dc5c6..b43f4c51b75 100755 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index 4b7d7055ac4..45d3e5084bf 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -27,6 +27,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS `.inner.wv`') + client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) diff --git a/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql b/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql index df10f8b892f..2508d5c2aa8 100644 --- a/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql +++ b/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql b/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql index 24b35f6b3b3..c027ebbaa1c 100644 --- a/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql +++ b/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py b/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py index a9ee0557736..dbc02e8a39e 100755 --- a/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py +++ b/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py @@ -27,6 +27,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS `.inner.wv`') + client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) diff --git a/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql b/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql index 4be8b453df6..cb8caa7915e 100644 --- a/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql +++ b/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql b/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql index 8b20610c94c..05c65312dff 100644 --- a/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql +++ b/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py b/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py index 2a7c9711d2a..fceb1f3dc4b 100755 --- a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py +++ b/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py @@ -27,6 +27,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS `.inner.wv`') + client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) diff --git a/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql b/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql index 69138a7be89..a1da3bfbe05 100644 --- a/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql +++ b/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql b/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql index 337f2bb6c1b..19a484c0f24 100644 --- a/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql +++ b/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py b/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py index 5bf6ff057a9..262175810d9 100755 --- a/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py +++ b/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py @@ -27,6 +27,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(prompt) client1.send('DROP TABLE IF EXISTS test.wv') client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS `.inner.wv`') + client1.expect(prompt) client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) diff --git a/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql b/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql index 4bbee2f9c80..d734d9e9d57 100644 --- a/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql +++ b/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql b/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql index 5351baf24f5..9044d396fdf 100644 --- a/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql +++ b/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); diff --git a/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql b/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql index 3d3a7c37beb..79de8d1bbe1 100644 --- a/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql +++ b/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql @@ -3,6 +3,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); From 563c4fb382c5def7c40548f327de492c4f0c6bb5 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 15 Jul 2020 12:53:38 +0800 Subject: [PATCH 061/609] remove BlocksListSource --- src/Storages/WindowView/BlocksListSource.h | 38 ------------------- src/Storages/WindowView/StorageWindowView.cpp | 1 - 2 files changed, 39 deletions(-) delete mode 100644 src/Storages/WindowView/BlocksListSource.h diff --git a/src/Storages/WindowView/BlocksListSource.h b/src/Storages/WindowView/BlocksListSource.h deleted file mode 100644 index 66cb475bc9e..00000000000 --- a/src/Storages/WindowView/BlocksListSource.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ -using BlocksListPtr = std::shared_ptr; - -/** A stream of blocks from a shared list of blocks - */ -class BlocksListSource : public SourceWithProgress -{ -public: - BlocksListSource(const BlocksListPtr & blocks_ptr_, Block header) - : SourceWithProgress(std::move(header)) - , blocks(blocks_ptr_), it(blocks_ptr_->begin()), end(blocks_ptr_->end()) {} - - String getName() const override { return "BlocksList"; } - -protected: - Chunk generate() override - { - if (it == end) - return {}; - - Block res = *it; - ++it; - return Chunk(res.getColumns(), res.rows()); - } - -private: - BlocksListPtr blocks; - BlocksList::iterator it; - const BlocksList::iterator end; -}; - -} diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 239978b07c4..a87e305d7c6 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -37,7 +37,6 @@ #include #include -#include #include #include #include From db8f64a5730b8d95777aa8f767da713f24dfa11e Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 15 Jul 2020 14:18:57 +0800 Subject: [PATCH 062/609] filter outdated data before aggregating --- src/Storages/WindowView/StorageWindowView.cpp | 92 ++++++++++--------- 1 file changed, 49 insertions(+), 43 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index a87e305d7c6..a2ebd3c2195 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -994,6 +994,51 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); BlockInputStreamPtr source_stream; + UInt32 lateness_bound = 0; + UInt32 t_max_watermark = 0; + UInt32 t_max_timestamp = 0; + UInt32 t_max_fired_watermark = 0; + { + std::lock_guard lock(window_view.fire_signal_mutex); + t_max_fired_watermark = window_view.max_fired_watermark; + t_max_watermark = window_view.max_watermark; + t_max_timestamp = window_view.max_timestamp; + } + + // Filter outdated data + if (window_view.allowed_lateness && t_max_timestamp != 0) + { + lateness_bound + = window_view.addTime(t_max_timestamp, window_view.lateness_kind, -1 * window_view.lateness_num_units); + if (window_view.is_watermark_bounded) + { + UInt32 watermark_lower_bound = window_view.is_tumble + ? window_view.addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units) + : window_view.addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units); + if (watermark_lower_bound < lateness_bound) + lateness_bound = watermark_lower_bound; + } + } + else + { + lateness_bound = t_max_fired_watermark; + } + + if (lateness_bound > 0) + { + ColumnsWithTypeAndName columns; + columns.emplace_back(nullptr, std::make_shared(), window_view.timestamp_column_name); + ExpressionActionsPtr filter_expressions = std::make_shared(columns, context); + filter_expressions->add(ExpressionAction::addColumn( + {std::make_shared()->createColumnConst(1, toField(lateness_bound)), + std::make_shared(), + "____lateness_bound"})); + const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); + filter_expressions->add(ExpressionAction::applyFunction( + function_greater, Names{window_view.timestamp_column_name, "____lateness_bound"}, "____filter")); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), filter_expressions, "____filter", true)); + } + std::shared_lock fire_signal_lock; if (window_view.is_proctime) { @@ -1009,45 +1054,6 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con } else { - UInt32 t_max_fired_watermark = 0; - if (window_view.allowed_lateness) - { - UInt32 t_max_timestamp = 0; - UInt32 t_max_watermark = 0; - { - std::lock_guard lock(window_view.fire_signal_mutex); - t_max_fired_watermark = window_view.max_fired_watermark; - t_max_watermark = window_view.max_watermark; - t_max_timestamp = window_view.max_timestamp; - } - - if (t_max_timestamp!= 0) - { - UInt32 lateness_bound - = window_view.addTime(t_max_timestamp, window_view.lateness_kind, -1 * window_view.lateness_num_units); - if (window_view.is_watermark_bounded) - { - UInt32 watermark_lower_bound = window_view.is_tumble - ? window_view.addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units) - : window_view.addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units); - if (watermark_lower_bound < lateness_bound) - lateness_bound = watermark_lower_bound; - } - - ColumnsWithTypeAndName columns; - columns.emplace_back(nullptr, std::make_shared(), window_view.timestamp_column_name); - ExpressionActionsPtr filter_expressions = std::make_shared(columns, context); - filter_expressions->add( - ExpressionAction::addColumn({std::make_shared()->createColumnConst(1, toField(lateness_bound)), - std::make_shared(), - "____lateness_bound"})); - const auto & function_greater = FunctionFactory::instance().get("greaterOrEquals", context); - filter_expressions->add(ExpressionAction::applyFunction( - function_greater, Names{window_view.timestamp_column_name, "____lateness_bound"}, "____filter")); - pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), filter_expressions, "____filter", true)); - } - } - InterpreterSelectQuery select_block(window_view.getMergeableQuery(), context, {std::move(pipe)}, QueryProcessingStage::WithMergeableState); source_stream = select_block.execute().getInputStream(); @@ -1057,18 +1063,18 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (window_view.is_watermark_bounded || window_view.allowed_lateness) { - UInt32 t_max_timestamp = 0; + UInt32 block_max_timestamp = 0; if (window_view.is_watermark_bounded || window_view.allowed_lateness) { const auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); for (const auto & timestamp : timestamp_data) { - if (timestamp > t_max_timestamp) - t_max_timestamp = timestamp; + if (timestamp > block_max_timestamp) + block_max_timestamp = timestamp; } } - std::static_pointer_cast(source_stream)->setMaxTimestamp(t_max_timestamp); + std::static_pointer_cast(source_stream)->setMaxTimestamp(block_max_timestamp); } if (window_view.allowed_lateness && t_max_fired_watermark != 0) From 3ac816a19165edaf72b187a3a1285193cbd3fd50 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 20 Jul 2020 02:56:55 +0800 Subject: [PATCH 063/609] rename visitor --- src/Storages/WindowView/StorageWindowView.cpp | 47 ++++++++++--------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 7a3ffc350b5..5630d150365 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -101,7 +101,7 @@ namespace } }; - struct ReplaceWindowIdVisitorData + struct ReplaceWindowIdData { using TypeToVisit = ASTFunction; bool is_tumble; @@ -118,7 +118,9 @@ namespace } }; - struct ReplaceFunctionNowVisitorData + using ReplaceWindowIdVisitor = InDepthNodeVisitor, true>; + + struct ReplaceFunctionNowData { using TypeToVisit = ASTFunction; @@ -139,7 +141,9 @@ namespace } }; - struct ReplaceFunctionWindowVisitorData + using ReplaceFunctionNowVisitor = InDepthNodeVisitor, true>; + + struct ReplaceFunctionWindowData { using TypeToVisit = ASTFunction; @@ -150,10 +154,12 @@ namespace } }; - class ReplaceFunctionWindowMatcher + using ReplaceFunctionWindowVisitor = InDepthNodeVisitor, true>; + + class ToIdentifierMatcher { public: - using Visitor = InDepthNodeVisitor; + using Visitor = InDepthNodeVisitor; struct Data { @@ -451,16 +457,15 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery columns_list->children.push_back(column_dec); } - ReplaceFunctionWindowMatcher::Data query_data; + ToIdentifierMatcher::Data query_data; query_data.window_id_name = window_id_name; query_data.window_id_alias = window_id_alias; - query_data.aliases = &aliases; - ReplaceFunctionWindowMatcher::Visitor visitor(query_data); + ToIdentifierMatcher::Visitor to_identifier_visitor(query_data); - ReplaceFunctionNowVisitorData time_now_data; - ReplaceFunctionWindowVisitorData func_hop_data; - InDepthNodeVisitor, true> time_now_visitor(time_now_data); - InDepthNodeVisitor, true> func_window_visitor(func_hop_data); + ReplaceFunctionNowData time_now_data; + ReplaceFunctionNowVisitor time_now_visitor(time_now_data); + ReplaceFunctionWindowData func_hop_data; + ReplaceFunctionWindowVisitor func_window_visitor(func_hop_data); auto new_storage = std::make_shared(); if (storage == nullptr) @@ -479,7 +484,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery ASTPtr order_by_ptr = order_by; if (is_time_column_func_now) time_now_visitor.visit(order_by_ptr); - visitor.visit(order_by_ptr); + to_identifier_visitor.visit(order_by_ptr); for (auto & child : order_by->arguments->children) { @@ -510,7 +515,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery if (is_time_column_func_now) time_now_visitor.visit(partition_by); func_window_visitor.visit(partition_by); - visitor.visit(partition_by); + to_identifier_visitor.visit(partition_by); new_storage->set(new_storage->partition_by, partition_by); } if (storage->primary_key) @@ -519,7 +524,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery if (is_time_column_func_now) time_now_visitor.visit(tmp_primary_key); func_window_visitor.visit(tmp_primary_key); - visitor.visit(tmp_primary_key); + to_identifier_visitor.visit(tmp_primary_key); new_storage->set(new_storage->primary_key, tmp_primary_key); } if (storage->order_by) @@ -528,7 +533,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery if (is_time_column_func_now) time_now_visitor.visit(order_by); func_window_visitor.visit(order_by); - visitor.visit(order_by); + to_identifier_visitor.visit(order_by); new_storage->set(new_storage->order_by, order_by); } if (storage->sample_by) @@ -537,7 +542,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery if (is_time_column_func_now) time_now_visitor.visit(sample_by); func_window_visitor.visit(sample_by); - visitor.visit(sample_by); + to_identifier_visitor.visit(sample_by); new_storage->set(new_storage->sample_by, sample_by); } if (storage->settings) @@ -834,17 +839,17 @@ StorageWindowView::StorageWindowView( mergeable_query = inner_query->clone(); - ReplaceFunctionNowVisitorData func_now_data; - InDepthNodeVisitor, true>(func_now_data).visit(mergeable_query); + ReplaceFunctionNowData func_now_data; + ReplaceFunctionNowVisitor(func_now_data).visit(mergeable_query); is_time_column_func_now = func_now_data.is_time_column_func_now; if (is_time_column_func_now) window_id_name = func_now_data.window_id_name; final_query = mergeable_query->clone(); - ReplaceWindowIdVisitorData final_query_data; + ReplaceWindowIdData final_query_data; final_query_data.is_tumble = is_tumble; - InDepthNodeVisitor, true>(final_query_data).visit(final_query); + ReplaceWindowIdVisitor(final_query_data).visit(final_query); is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; is_watermark_ascending = query.is_watermark_ascending; From 132345cd2099f29fb34267e94d85566bf50bd130 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 20 Jul 2020 03:00:21 +0800 Subject: [PATCH 064/609] change inner_query to local variable --- src/Storages/WindowView/StorageWindowView.cpp | 16 ++++++++-------- src/Storages/WindowView/StorageWindowView.h | 4 +--- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 5630d150365..c44b80de7dc 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -216,8 +216,6 @@ namespace return IntervalKind::Year; __builtin_unreachable(); } - - String generateInnerTableName(const String & table_name) { return ".inner." + table_name; } } static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) @@ -246,9 +244,9 @@ static void extractDependentTable(ASTSelectQuery & query, String & select_databa if (ast_select->list_of_selects->children.size() != 1) throw Exception("UNION is not supported for WINDOW VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); - auto & inner_query = ast_select->list_of_selects->children.at(0); + auto & inner_select_query = ast_select->list_of_selects->children.at(0); - extractDependentTable(inner_query->as(), select_database_name, select_table_name); + extractDependentTable(inner_select_query->as(), select_database_name, select_table_name); } else throw Exception( @@ -418,14 +416,14 @@ inline void StorageWindowView::fire(UInt32 watermark) fire_condition.notify_all(); } -std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name) +std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name) { /// We will create a query to create an internal table. auto inner_create_query = std::make_shared(); inner_create_query->database = database_name; inner_create_query->table = table_name; - auto inner_select_query = std::static_pointer_cast(getInnerQuery()); + auto inner_select_query = std::static_pointer_cast(inner_query); Aliases aliases; QueryAliasesVisitor::Data query_aliases_data{aliases}; @@ -835,7 +833,7 @@ StorageWindowView::StorageWindowView( String select_table_name; extractDependentTable(select_query, select_database_name, select_table_name); select_table_id = StorageID(select_database_name, select_table_name); - inner_query = innerQueryParser(select_query); + auto inner_query = innerQueryParser(select_query); mergeable_query = inner_query->clone(); @@ -926,6 +924,8 @@ StorageWindowView::StorageWindowView( throw Exception("Value for ALLOWED_LATENESS function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } + auto generateInnerTableName = [](const String & table_name) { return ".inner." + table_name; }; + if (attach_) { inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); @@ -933,7 +933,7 @@ StorageWindowView::StorageWindowView( else { auto inner_create_query - = generateInnerTableCreateQuery(query.storage, table_id_.database_name, generateInnerTableName(table_id_.table_name)); + = generateInnerTableCreateQuery(inner_query, query.storage, table_id_.database_name, generateInnerTableName(table_id_.table_name)); InterpreterCreateQuery create_interpreter(inner_create_query, *wv_context); create_interpreter.setInternal(true); diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 53533b2b7cf..0713bfffa79 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -59,7 +59,6 @@ public: static void writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context); private: - ASTPtr inner_query; ASTPtr mergeable_query; ASTPtr final_query; @@ -118,7 +117,7 @@ private: ASTPtr innerQueryParser(ASTSelectQuery & inner_query); - std::shared_ptr generateInnerTableCreateQuery(ASTStorage * storage, const String & database_name, const String & table_name); + std::shared_ptr generateInnerTableCreateQuery(const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name); UInt32 getCleanupBound(); ASTPtr generateCleanupQuery(); @@ -137,7 +136,6 @@ private: static Pipes blocksToPipes(BlocksList & blocks, Block & sample_block); - ASTPtr getInnerQuery() const { return inner_query->clone(); } ASTPtr getMergeableQuery() const { return mergeable_query->clone(); } ASTPtr getFinalQuery() const { return final_query->clone(); } ASTPtr getFetchColumnQuery(UInt32 w_start, UInt32 w_end) const; From c2763dadd7e2a61f49761ae42d3171a36901845e Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 20 Jul 2020 03:01:42 +0800 Subject: [PATCH 065/609] remove alias to name cast --- src/Storages/WindowView/StorageWindowView.cpp | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index c44b80de7dc..bcbc626e4c5 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -191,8 +191,6 @@ namespace { if (node.getColumnName() == data.window_id_alias) dynamic_cast(node_ptr.get())->name = data.window_id_name; - else if (auto it = data.aliases->find(node.getColumnName()); it != data.aliases->end()) - dynamic_cast(node_ptr.get())->name = it->second->getColumnName(); } }; @@ -425,10 +423,6 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery auto inner_select_query = std::static_pointer_cast(inner_query); - Aliases aliases; - QueryAliasesVisitor::Data query_aliases_data{aliases}; - QueryAliasesVisitor(query_aliases_data).visit(inner_select_query); - auto t_sample_block = InterpreterSelectQuery( inner_select_query, *wv_context, getParentStorage(), nullptr, SelectQueryOptions(QueryProcessingStage::WithMergeableState)) From 5bdf35f6bba8da3271455356ad632f4af3b274b9 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 20 Jul 2020 03:02:18 +0800 Subject: [PATCH 066/609] simplify code --- src/Storages/WindowView/StorageWindowView.cpp | 54 +++++++------------ 1 file changed, 18 insertions(+), 36 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index bcbc626e4c5..ddd49360ba0 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -501,42 +501,24 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery ErrorCodes::INCORRECT_QUERY); new_storage->set(new_storage->engine, storage->engine->clone()); - if (storage->partition_by) - { - auto partition_by = storage->partition_by->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(partition_by); - func_window_visitor.visit(partition_by); - to_identifier_visitor.visit(partition_by); - new_storage->set(new_storage->partition_by, partition_by); - } - if (storage->primary_key) - { - auto tmp_primary_key = storage->primary_key->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(tmp_primary_key); - func_window_visitor.visit(tmp_primary_key); - to_identifier_visitor.visit(tmp_primary_key); - new_storage->set(new_storage->primary_key, tmp_primary_key); - } - if (storage->order_by) - { - auto order_by = storage->order_by->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(order_by); - func_window_visitor.visit(order_by); - to_identifier_visitor.visit(order_by); - new_storage->set(new_storage->order_by, order_by); - } - if (storage->sample_by) - { - auto sample_by = storage->sample_by->clone(); - if (is_time_column_func_now) - time_now_visitor.visit(sample_by); - func_window_visitor.visit(sample_by); - to_identifier_visitor.visit(sample_by); - new_storage->set(new_storage->sample_by, sample_by); - } + + auto visit = [&](const IAST * ast, IAST * & field) { + if (ast) + { + auto node = ast->clone(); + if (is_time_column_func_now) + time_now_visitor.visit(node); + func_window_visitor.visit(node); + to_identifier_visitor.visit(node); + new_storage->set(field, node); + } + }; + + visit(storage->partition_by, new_storage->partition_by); + visit(storage->primary_key, new_storage->primary_key); + visit(storage->order_by, new_storage->order_by); + visit(storage->sample_by, new_storage->sample_by); + if (storage->settings) new_storage->set(new_storage->settings, storage->settings->clone()); } From 628251dce3bd2ff8f2678ea904c241e16b533ac7 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 20 Jul 2020 21:32:34 +0800 Subject: [PATCH 067/609] add check before static_cast --- src/Storages/WindowView/StorageWindowView.cpp | 312 +++++++++--------- src/Storages/WindowView/StorageWindowView.h | 5 +- 2 files changed, 161 insertions(+), 156 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index ddd49360ba0..1e777963df2 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -62,64 +62,75 @@ namespace { const auto RESCHEDULE_MS = 500; - struct MergeableQueryVisitorData + struct MergeableQueryMatcher { + using Visitor = InDepthNodeVisitor; using TypeToVisit = ASTFunction; - ASTPtr window_function; - String window_id_name; - String window_id_alias; - String serialized_window_function; - String timestamp_column_name; - bool is_tumble = false; - bool is_hop = false; - - void visit(const ASTFunction & node, ASTPtr & node_ptr) + struct Data { - if (node.name == "TUMBLE" || node.name == "HOP") + ASTPtr window_function; + String window_id_name; + String window_id_alias; + String serialized_window_function; + String timestamp_column_name; + bool is_tumble = false; + bool is_hop = false; + }; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data & data) + { + if (auto * t = ast->as()) { - is_tumble = node.name == "TUMBLE"; - is_hop = node.name == "HOP"; - if (!window_function) + if (t->name == "TUMBLE" || t->name == "HOP") { - std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; - window_id_name = node.getColumnName(); - window_id_alias = node.alias; - window_function = node.clone(); - window_function->setAlias(""); - serialized_window_function = serializeAST(*window_function); - timestamp_column_name = node.arguments->children[0]->getColumnName(); - } - else - { - auto temp_node = node.clone(); - temp_node->setAlias(""); - if (serializeAST(*temp_node) != serialized_window_function) - throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + data.is_tumble = t->name == "TUMBLE"; + data.is_hop = t->name == "HOP"; + if (!data.window_function) + { + t->name = "WINDOW_ID"; + data.window_id_name = t->getColumnName(); + data.window_id_alias = t->alias; + data.window_function = t->clone(); + data.window_function->setAlias(""); + data.serialized_window_function = serializeAST(*data.window_function); + data.timestamp_column_name = t->arguments->children[0]->getColumnName(); + } + else + { + auto temp_node = t->clone(); + temp_node->setAlias(""); + if (serializeAST(*temp_node) != data.serialized_window_function) + throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + } } } } }; - struct ReplaceWindowIdData + struct ReplaceWindowIdMatcher { - using TypeToVisit = ASTFunction; - bool is_tumble; - - void visit(const ASTFunction & node, ASTPtr & node_ptr) const + public: + using Visitor = InDepthNodeVisitor; + struct Data { - if (node.name == "WINDOW_ID") + String window_name; + }; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data & data) + { + if (auto * t = ast->as()) { - if (is_tumble) - std::static_pointer_cast(node_ptr)->name = "TUMBLE"; - else - std::static_pointer_cast(node_ptr)->name = "HOP"; + if (t->name == "WINDOW_ID") + t->name = data.window_name; } } }; - using ReplaceWindowIdVisitor = InDepthNodeVisitor, true>; - struct ReplaceFunctionNowData { using TypeToVisit = ASTFunction; @@ -143,19 +154,24 @@ namespace using ReplaceFunctionNowVisitor = InDepthNodeVisitor, true>; - struct ReplaceFunctionWindowData + struct ReplaceFunctionWindowMatcher { - using TypeToVisit = ASTFunction; + using Visitor = InDepthNodeVisitor; - static void visit(ASTFunction & node, ASTPtr & node_ptr) + struct Data{}; + + static bool needChildVisit(ASTPtr &, const ASTPtr &) { return true; } + + static void visit(ASTPtr & ast, Data &) { - if (node.name == "HOP" || node.name == "TUMBLE") - std::static_pointer_cast(node_ptr)->name = "WINDOW_ID"; + if (auto * t = ast->as()) + { + if (t->name == "HOP" || t->name == "TUMBLE") + t->name = "WINDOW_ID"; + } } }; - using ReplaceFunctionWindowVisitor = InDepthNodeVisitor, true>; - class ToIdentifierMatcher { public: @@ -414,6 +430,26 @@ inline void StorageWindowView::fire(UInt32 watermark) fire_condition.notify_all(); } +void StorageWindowView::extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg) +{ + const auto * arg = ast->as(); + if (!arg || !startsWith(arg->name, "toInterval")) + throw Exception(err_msg, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + kind = strToIntervalKind(arg->name.substr(10)); + const auto * interval_unit = arg->children.front()->children.front()->as(); + if (!interval_unit + || (interval_unit->value.getType() != Field::Types::String && interval_unit->value.getType() != Field::Types::UInt64)) + throw Exception("Interval argument must be integer", ErrorCodes::BAD_ARGUMENTS); + if (interval_unit->value.getType() == Field::Types::String) + num_units = std::stoi(interval_unit->value.safeGet()); + else + num_units = interval_unit->value.safeGet(); + + if (num_units <= 0) + throw Exception("Value for Interval argument must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); +} + std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name) { /// We will create a query to create an internal table. @@ -456,8 +492,8 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery ReplaceFunctionNowData time_now_data; ReplaceFunctionNowVisitor time_now_visitor(time_now_data); - ReplaceFunctionWindowData func_hop_data; - ReplaceFunctionWindowVisitor func_window_visitor(func_hop_data); + ReplaceFunctionWindowMatcher::Data func_hop_data; + ReplaceFunctionWindowMatcher::Visitor func_window_visitor(func_hop_data); auto new_storage = std::make_shared(); if (storage == nullptr) @@ -808,100 +844,48 @@ StorageWindowView::StorageWindowView( String select_database_name = global_context.getCurrentDatabase(); String select_table_name; extractDependentTable(select_query, select_database_name, select_table_name); - select_table_id = StorageID(select_database_name, select_table_name); - auto inner_query = innerQueryParser(select_query); - - mergeable_query = inner_query->clone(); - - ReplaceFunctionNowData func_now_data; - ReplaceFunctionNowVisitor(func_now_data).visit(mergeable_query); - is_time_column_func_now = func_now_data.is_time_column_func_now; - if (is_time_column_func_now) - window_id_name = func_now_data.window_id_name; - - final_query = mergeable_query->clone(); - - ReplaceWindowIdData final_query_data; - final_query_data.is_tumble = is_tumble; - ReplaceWindowIdVisitor(final_query_data).visit(final_query); - - is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; - is_watermark_ascending = query.is_watermark_ascending; - is_watermark_bounded = query.is_watermark_bounded; - /// If the table is not specified - use the table `system.one` if (select_table_name.empty()) { select_database_name = "system"; select_table_name = "one"; } - + select_table_id = StorageID(select_database_name, select_table_name); DatabaseCatalog::instance().addDependency(select_table_id, table_id_); + // Parser inner query + auto inner_query = innerQueryParser(select_query); + + // Parser mergeable query + mergeable_query = inner_query->clone(); + ReplaceFunctionNowData func_now_data; + ReplaceFunctionNowVisitor(func_now_data).visit(mergeable_query); + is_time_column_func_now = func_now_data.is_time_column_func_now; + if (is_time_column_func_now) + window_id_name = func_now_data.window_id_name; + + // Parser final query + final_query = mergeable_query->clone(); + ReplaceWindowIdMatcher::Data final_query_data; + if (is_tumble) + final_query_data.window_name = "TUMBLE"; + else + final_query_data.window_name = "HOP"; + ReplaceWindowIdMatcher::Visitor(final_query_data).visit(final_query); + + is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; + is_watermark_ascending = query.is_watermark_ascending; + is_watermark_bounded = query.is_watermark_bounded; target_table_id = query.to_table_id; - clean_interval = global_context.getSettingsRef().window_view_clean_interval.totalSeconds(); - next_fire_signal = getWindowUpperBound(std::time(nullptr)); + eventTimeParser(query); - if (query.is_watermark_strictly_ascending || query.is_watermark_ascending || query.is_watermark_bounded) - { - is_proctime = false; - if (is_time_column_func_now) - throw Exception("now() is not support for Event time processing.", ErrorCodes::INCORRECT_QUERY); - if (query.is_watermark_ascending) - { - is_watermark_bounded = true; - watermark_kind = IntervalKind::Second; - watermark_num_units = 1; - } - else if (query.is_watermark_bounded) - { - // parser watermark function - const auto & watermark_function = std::static_pointer_cast(query.watermark_function); - if (!startsWith(watermark_function->name, "toInterval")) - throw Exception("Illegal type of WATERMARK function, should be Interval", ErrorCodes::ILLEGAL_COLUMN); - - const auto & interval_units_p1 = std::static_pointer_cast(watermark_function->children.front()->children.front()); - watermark_kind = strToIntervalKind(watermark_function->name.substr(10)); - try - { - watermark_num_units = boost::lexical_cast(interval_units_p1->value.get()); - } - catch (const boost::bad_lexical_cast &) - { - throw Exception("Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); - } - if (watermark_num_units <= 0) - throw Exception("Value for WATERMARK function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } - } - - if (query.allowed_lateness) - { - allowed_lateness = true; - - // parser lateness function - const auto & lateness_function = std::static_pointer_cast(query.lateness_function); - if (!startsWith(lateness_function->name, "toInterval")) - throw Exception("Illegal type of ALLOWED_LATENESS function, should be Interval", ErrorCodes::ILLEGAL_COLUMN); - - const auto & interval_units_p1 = std::static_pointer_cast(lateness_function->children.front()->children.front()); - lateness_kind = strToIntervalKind(lateness_function->name.substr(10)); - try - { - lateness_num_units = boost::lexical_cast(interval_units_p1->value.get()); - } - catch (const boost::bad_lexical_cast &) - { - throw Exception( - "Cannot parse string '" + interval_units_p1->value.get() + "' as Integer.", ErrorCodes::CANNOT_PARSE_TEXT); - } - if (lateness_num_units <= 0) - throw Exception("Value for ALLOWED_LATENESS function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } + if (is_tumble) + window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "TUMBLE"); + else + window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); auto generateInnerTableName = [](const String & table_name) { return ".inner." + table_name; }; - if (attach_) { inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); @@ -918,10 +902,8 @@ StorageWindowView::StorageWindowView( inner_table_id = inner_storage->getStorageID(); } - if (is_tumble) - window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "TUMBLE"); - else - window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); + clean_interval = global_context.getSettingsRef().window_view_clean_interval.totalSeconds(); + next_fire_signal = getWindowUpperBound(std::time(nullptr)); clean_cache_task = wv_context->getSchedulePool().createTask(getStorageID().getFullTableName(), [this] { threadFuncCleanup(); }); if (is_proctime) @@ -941,8 +923,8 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parse stage mergeable ASTPtr result = query.clone(); ASTPtr expr_list = result; - MergeableQueryVisitorData stage_mergeable_data; - InDepthNodeVisitor, true>(stage_mergeable_data).visit(expr_list); + MergeableQueryMatcher::Data stage_mergeable_data; + MergeableQueryMatcher::Visitor(stage_mergeable_data).visit(expr_list); if (!stage_mergeable_data.is_tumble && !stage_mergeable_data.is_hop) throw Exception("WINDOW FUNCTION is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); window_id_name = stage_mergeable_data.window_id_name; @@ -953,33 +935,54 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) // parser window function ASTFunction & window_function = typeid_cast(*stage_mergeable_data.window_function); const auto & arguments = window_function.arguments->children; - const auto & arg1 = std::static_pointer_cast(arguments.at(1)); - if (!arg1 || !startsWith(arg1->name, "toInterval")) - throw Exception("Illegal type of second argument of function " + arg1->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - window_kind = strToIntervalKind(arg1->name.substr(10)); - const auto & interval_units_p1 = std::static_pointer_cast(arg1->children.front()->children.front()); - window_num_units = stoi(interval_units_p1->value.get()); - if (window_num_units <= 0) - throw Exception("Interval value for WINDOW function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + extractWindowArgument( + arguments.at(1), + window_kind, + window_num_units, + "Illegal type of second argument of function " + window_function.name + " should be Interval"); if (!is_tumble) { hop_kind = window_kind; hop_num_units = window_num_units; - const auto & arg2 = std::static_pointer_cast(arguments.at(2)); - if (!arg2 || !startsWith(arg2->name, "toInterval")) - throw Exception("Illegal type of last argument of function " + arg2->name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - window_kind = strToIntervalKind(arg2->name.substr(10)); - const auto & interval_units_p2 = std::static_pointer_cast(arg2->children.front()->children.front()); - window_num_units = stoi(interval_units_p2->value.get()); - if (window_num_units <= 0) - throw Exception("Interval value for WINDOW function must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - + extractWindowArgument( + arguments.at(2), + window_kind, + window_num_units, + "Illegal type of third argument of function " + window_function.name + " should be Interval"); slice_num_units= std::gcd(hop_num_units, window_num_units); } return result; } +void StorageWindowView::eventTimeParser(const ASTCreateQuery & query) +{ + if (query.is_watermark_strictly_ascending || query.is_watermark_ascending || query.is_watermark_bounded) + { + is_proctime = false; + if (is_time_column_func_now) + throw Exception("now() is not support for Event time processing.", ErrorCodes::INCORRECT_QUERY); + if (query.is_watermark_ascending) + { + is_watermark_bounded = true; + watermark_kind = IntervalKind::Second; + watermark_num_units = 1; + } + else if (query.is_watermark_bounded) + { + extractWindowArgument( + query.watermark_function, watermark_kind, watermark_num_units, "Illegal type WATERMARK function should be Interval"); + } + } + + if (query.allowed_lateness) + { + allowed_lateness = true; + extractWindowArgument( + query.lateness_function, lateness_kind, lateness_num_units, "Illegal type ALLOWED_LATENESS function should be Interval"); + } +} + void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, const Block & block, const Context & context) { Pipe pipe(std::make_shared(block.cloneEmpty(), Chunk(block.getColumns(), block.rows()))); @@ -1057,8 +1060,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con UInt32 block_max_timestamp = 0; if (window_view.is_watermark_bounded || window_view.allowed_lateness) { - const auto & column_timestamp = block.getByName(window_view.timestamp_column_name).column; - const ColumnUInt32::Container & timestamp_data = static_cast(*column_timestamp).getData(); + const auto & timestamp_data = typeid_cast(*block.getByName(window_view.timestamp_column_name).column).getData(); for (const auto & timestamp : timestamp_data) { if (timestamp > block_max_timestamp) diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 0713bfffa79..7bf20c2a143 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -116,8 +116,11 @@ private: BackgroundSchedulePool::TaskHolder fire_task; ASTPtr innerQueryParser(ASTSelectQuery & inner_query); + void eventTimeParser(const ASTCreateQuery & query); - std::shared_ptr generateInnerTableCreateQuery(const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name); + void extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg); + std::shared_ptr generateInnerTableCreateQuery( + const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name); UInt32 getCleanupBound(); ASTPtr generateCleanupQuery(); From fa267aff04538f1a7aef80272065e0b592d60aa3 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 20 Jul 2020 21:39:41 +0800 Subject: [PATCH 068/609] add test for integer argument of function interval --- .../0_stateless/01048_window_view_sql_parser.reference | 4 ++-- tests/queries/0_stateless/01048_window_view_sql_parser.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.reference b/tests/queries/0_stateless/01048_window_view_sql_parser.reference index 31bec6c3bb5..afe0148e48d 100644 --- a/tests/queries/0_stateless/01048_window_view_sql_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.reference @@ -1,6 +1,6 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- @@ -11,7 +11,7 @@ CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(tim CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.sql b/tests/queries/0_stateless/01048_window_view_sql_parser.sql index 7c66fa21a10..578e2add8a8 100644 --- a/tests/queries/0_stateless/01048_window_view_sql_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.sql @@ -9,7 +9,7 @@ SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS wv; DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM mt GROUP BY TUMBLE(timestamp, INTERVAL 1 SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; @@ -45,7 +45,7 @@ SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS wv; DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND) as wid; SHOW CREATE TABLE `.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; From 60e50a17a7e9b0dabb5ad3412e7416550de91fb4 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 21 Jul 2020 17:43:42 +0800 Subject: [PATCH 069/609] update code style --- src/Storages/WindowView/StorageWindowView.cpp | 54 +++++++++---------- src/Storages/WindowView/StorageWindowView.h | 1 - 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 1e777963df2..739383b3225 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -49,8 +49,8 @@ namespace DB namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; - extern const int CANNOT_PARSE_TEXT; - extern const int ILLEGAL_COLUMN; + extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; extern const int QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW; @@ -210,7 +210,7 @@ namespace } }; - IntervalKind strToIntervalKind(const String& interval_str) + static IntervalKind strToIntervalKind(const String& interval_str) { if (interval_str == "Second") return IntervalKind::Second; @@ -230,6 +230,26 @@ namespace return IntervalKind::Year; __builtin_unreachable(); } + + static void extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg) + { + const auto * arg = ast->as(); + if (!arg || !startsWith(arg->name, "toInterval")) + throw Exception(err_msg, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + kind = strToIntervalKind(arg->name.substr(10)); + const auto * interval_unit = arg->children.front()->children.front()->as(); + if (!interval_unit + || (interval_unit->value.getType() != Field::Types::String && interval_unit->value.getType() != Field::Types::UInt64)) + throw Exception("Interval argument must be integer", ErrorCodes::BAD_ARGUMENTS); + if (interval_unit->value.getType() == Field::Types::String) + num_units = std::stoi(interval_unit->value.safeGet()); + else + num_units = interval_unit->value.safeGet(); + + if (num_units <= 0) + throw Exception("Value for Interval argument must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } } static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) @@ -430,26 +450,6 @@ inline void StorageWindowView::fire(UInt32 watermark) fire_condition.notify_all(); } -void StorageWindowView::extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg) -{ - const auto * arg = ast->as(); - if (!arg || !startsWith(arg->name, "toInterval")) - throw Exception(err_msg, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - kind = strToIntervalKind(arg->name.substr(10)); - const auto * interval_unit = arg->children.front()->children.front()->as(); - if (!interval_unit - || (interval_unit->value.getType() != Field::Types::String && interval_unit->value.getType() != Field::Types::UInt64)) - throw Exception("Interval argument must be integer", ErrorCodes::BAD_ARGUMENTS); - if (interval_unit->value.getType() == Field::Types::String) - num_units = std::stoi(interval_unit->value.safeGet()); - else - num_units = interval_unit->value.safeGet(); - - if (num_units <= 0) - throw Exception("Value for Interval argument must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); -} - std::shared_ptr StorageWindowView::generateInnerTableCreateQuery(const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name) { /// We will create a query to create an internal table. @@ -538,7 +538,7 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery new_storage->set(new_storage->engine, storage->engine->clone()); - auto visit = [&](const IAST * ast, IAST * & field) { + auto visit = [&](const IAST * ast, IAST *& field) { if (ast) { auto node = ast->clone(); @@ -885,15 +885,15 @@ StorageWindowView::StorageWindowView( else window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); - auto generateInnerTableName = [](const String & table_name) { return ".inner." + table_name; }; + auto generate_inner_table_name = [](const String & table_name) { return ".inner." + table_name; }; if (attach_) { - inner_table_id = StorageID(table_id_.database_name, generateInnerTableName(table_id_.table_name)); + inner_table_id = StorageID(table_id_.database_name, generate_inner_table_name(table_id_.table_name)); } else { auto inner_create_query - = generateInnerTableCreateQuery(inner_query, query.storage, table_id_.database_name, generateInnerTableName(table_id_.table_name)); + = generateInnerTableCreateQuery(inner_query, query.storage, table_id_.database_name, generate_inner_table_name(table_id_.table_name)); InterpreterCreateQuery create_interpreter(inner_create_query, *wv_context); create_interpreter.setInternal(true); diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 7bf20c2a143..cd0f864e8f0 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -118,7 +118,6 @@ private: ASTPtr innerQueryParser(ASTSelectQuery & inner_query); void eventTimeParser(const ASTCreateQuery & query); - void extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg); std::shared_ptr generateInnerTableCreateQuery( const ASTPtr & inner_query, ASTStorage * storage, const String & database_name, const String & table_name); UInt32 getCleanupBound(); From ecd1e7265695ba5457c7d81cea6b6318206437cd Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 21 Jul 2020 18:10:35 +0800 Subject: [PATCH 070/609] update code style --- src/Storages/WindowView/StorageWindowView.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 739383b3225..35f57501803 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -538,7 +538,8 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery new_storage->set(new_storage->engine, storage->engine->clone()); - auto visit = [&](const IAST * ast, IAST *& field) { + auto visit = [&](const IAST * ast, IAST * & field) + { if (ast) { auto node = ast->clone(); From 686232f385cefefe9c0ca62b83f9a1a0e78655ad Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 22 Jul 2020 01:41:03 +0800 Subject: [PATCH 071/609] remove redundant static --- src/Storages/WindowView/StorageWindowView.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 35f57501803..8b814063a7f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -210,7 +210,7 @@ namespace } }; - static IntervalKind strToIntervalKind(const String& interval_str) + IntervalKind strToIntervalKind(const String& interval_str) { if (interval_str == "Second") return IntervalKind::Second; @@ -231,7 +231,7 @@ namespace __builtin_unreachable(); } - static void extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg) + void extractWindowArgument(const ASTPtr & ast, IntervalKind::Kind & kind, Int64 & num_units, String err_msg) { const auto * arg = ast->as(); if (!arg || !startsWith(arg->name, "toInterval")) From c8a98ab59ba597d41be0de58f31a38db3dc6b84e Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 22 Jul 2020 01:53:20 +0800 Subject: [PATCH 072/609] remove unnecessary filter when using now as timestamp --- src/Storages/WindowView/StorageWindowView.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8b814063a7f..69964f406d4 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -1014,7 +1014,7 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con lateness_bound = watermark_lower_bound; } } - else + else if (! window_view.is_time_column_func_now) { lateness_bound = t_max_fired_watermark; } From d935b075af8da45a5a46b4fc0494531c02888316 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 22 Jul 2020 11:54:34 +0800 Subject: [PATCH 073/609] add window keyword fuzzer dict --- tests/fuzz/ast.dict | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/fuzz/ast.dict b/tests/fuzz/ast.dict index af759b6e805..c69a41df610 100644 --- a/tests/fuzz/ast.dict +++ b/tests/fuzz/ast.dict @@ -394,6 +394,7 @@ "WATCH" "WEEK" "WHERE" +"WINDOW" "windowFunnel" "WITH" "WITH FILL" From e2967ef1aea9ebd54f917b8e589c7b7a887d96fc Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 25 Jul 2020 20:05:56 +0800 Subject: [PATCH 074/609] update test --- .../0_stateless/01062_window_view_event_asc_hop_watch.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py b/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py index fceb1f3dc4b..7744c2dd268 100755 --- a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py +++ b/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py @@ -47,8 +47,8 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.expect(r'Progress') client2.send("INSERT INTO test.mt VALUES (1, '1990/01/01 12:00:10');") client2.expect(prompt) - client1.expect('1*' + end_of_block) - client1.expect('2*' + end_of_block) + client1.expect(r'1*' + end_of_block) + client1.expect(r'2*' + end_of_block) client1.expect(r'Progress') # send Ctrl-C From 8c79a34a6c9bed92b3a860752485234e5a41030e Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 02:50:18 +0800 Subject: [PATCH 075/609] delete redundent period --- src/Functions/FunctionsWindow.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 4fb444f30fa..ea0183a2d72 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -134,7 +134,7 @@ namespace Int64 num_units = interval_column_const_int64->getValue(); if (num_units <= 0) throw Exception( - "Value for column " + interval_column.name + " of function " + function_name + " must be positive.", + "Value for column " + interval_column.name + " of function " + function_name + " must be positive", ErrorCodes::ARGUMENT_OUT_OF_BOUND); return {interval_type->getKind(), num_units}; @@ -484,7 +484,7 @@ namespace if (arguments.size() > 4) throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should not larger than 4.", + + ", should be 2, 3 or 4", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); return std::make_shared(); } From 513b76ebf76cfbb80b35b846dff05510918aae8f Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 02:54:09 +0800 Subject: [PATCH 076/609] use DayNum when interval is larger than Day --- src/Functions/FunctionsWindow.h | 134 ++++++++++++++++++-------------- 1 file changed, 76 insertions(+), 58 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index ea0183a2d72..07eba7ae520 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -1,7 +1,9 @@ #pragma once +#include #include #include +#include #include #include #include @@ -10,7 +12,6 @@ #include #include #include -#include #include "IFunctionImpl.h" @@ -65,7 +66,7 @@ struct ToStartOfTransform; template <> \ struct ToStartOfTransform \ { \ - static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ + static DayNum execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) \ { \ return time_zone.toStartOf##INTERVAL_KIND##Interval(time_zone.toDayNum(t), delta); \ } \ @@ -74,9 +75,17 @@ struct ToStartOfTransform; TRANSFORM_DATE(Quarter) TRANSFORM_DATE(Month) TRANSFORM_DATE(Week) - TRANSFORM_DATE(Day) #undef TRANSFORM_DATE + template <> + struct ToStartOfTransform + { + static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) + { + return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), delta); + } + }; + #define TRANSFORM_TIME(INTERVAL_KIND) \ template <> \ struct ToStartOfTransform \ @@ -98,21 +107,29 @@ struct ToStartOfTransform; template <> \ struct AddTime \ { \ - static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl & time_zone) { return time_zone.add##INTERVAL_KIND##s(t, delta); } \ + static DayNum execute(UInt16 d, UInt64 delta, const DateLUTImpl & time_zone) \ + { \ + return time_zone.add##INTERVAL_KIND##s(DayNum(d), delta); \ + } \ }; ADD_DATE(Year) ADD_DATE(Quarter) ADD_DATE(Month) - ADD_DATE(Week) - ADD_DATE(Day) #undef ADD_DATE + template <> + struct AddTime + { + static DayNum execute(UInt16 d, UInt64 delta, const DateLUTImpl &) { return DayNum(d + 7 * delta);} + }; + #define ADD_TIME(INTERVAL_KIND, INTERVAL) \ template <> \ struct AddTime \ { \ static UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) { return t + INTERVAL * delta; } \ }; + ADD_TIME(Day, 86400) ADD_TIME(Hour, 3600) ADD_TIME(Minute, 60) ADD_TIME(Second, 1) @@ -211,39 +228,40 @@ namespace switch (std::get<0>(interval)) { case IntervalKind::Second: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Minute: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Hour: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Day: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Week: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Month: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Quarter: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); case IntervalKind::Year: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); } __builtin_unreachable(); } - template + template static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) { const auto & time_data = time_column.getData(); size_t size = time_column.size(); - auto start = ColumnUInt32::create(size); - auto end = ColumnUInt32::create(size); - ColumnUInt32::Container & start_data = start->getData(); - ColumnUInt32::Container & end_data = end->getData(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); for (size_t i = 0; i != size; ++i) { - UInt32 wid = static_cast(ToStartOfTransform::execute(time_data[i], num_units, time_zone)); - start_data[i] = wid; - end_data[i] = AddTime::execute(wid, num_units, time_zone); + start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); + end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); } MutableColumns result; result.emplace_back(std::move(start)); @@ -390,52 +408,54 @@ namespace switch (std::get<0>(window_interval)) { case IntervalKind::Second: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Minute: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Hour: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Day: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Week: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Month: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Quarter: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Year: - return execute_hop( + return execute_hop( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); } __builtin_unreachable(); } - template + template static ColumnPtr execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) { const auto & time_data = time_column.getData(); size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); - auto start = ColumnUInt32::create(size); - auto end = ColumnUInt32::create(size); - ColumnUInt32::Container & start_data = start->getData(); - ColumnUInt32::Container & end_data = end->getData(); for (size_t i = 0; i < size; ++i) { - UInt32 wstart = static_cast(ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone)); - UInt32 wend = AddTime::execute(wstart, hop_num_units, time_zone); + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); - UInt32 wend_ = wend; - UInt32 wend_latest; + ToType wend_ = wend; + ToType wend_latest; do { @@ -507,45 +527,42 @@ namespace auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - if (std::get<0>(hop_interval) != std::get<0>(window_interval)) - throw Exception( - "Interval type of window and hop column of function " + function_name + ", must be same.", ErrorCodes::ILLEGAL_COLUMN); if (std::get<1>(hop_interval) > std::get<1>(window_interval)) throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval.", + "Value for hop interval of function " + function_name + " must not larger than window interval", ErrorCodes::ARGUMENT_OUT_OF_BOUND); switch (std::get<0>(window_interval)) { case IntervalKind::Second: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Minute: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Hour: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Day: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Week: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Month: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Quarter: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); case IntervalKind::Year: - return execute_hop_slice( + return execute_hop_slice( *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); } __builtin_unreachable(); } - template + template static ColumnPtr execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) { @@ -554,15 +571,16 @@ namespace const auto & time_data = time_column.getData(); size_t size = time_column.size(); - auto end = ColumnUInt32::create(size); - ColumnUInt32::Container & end_data = end->getData(); + auto end = ColumnVector::create(); + auto & end_data = end->getData(); + end_data.resize(size); for (size_t i = 0; i < size; ++i) { - UInt32 wstart = static_cast(ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone)); - UInt32 wend = AddTime::execute(wstart, hop_num_units, time_zone); + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - UInt32 wend_ = wend; - UInt32 wend_latest; + ToType wend_ = wend; + ToType wend_latest; do { From 1d5303c142031efdc90245db2dc0802d452b8bd9 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 02:59:01 +0800 Subject: [PATCH 077/609] update tests for DayNum type --- ...1049_window_view_window_functions.reference | 16 ++++++++++++++++ .../01049_window_view_window_functions.sql | 18 ++++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index 93fa40adf9c..9258c406f8f 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -1,4 +1,12 @@ --TUMBLE-- +('2020-01-09 12:00:01','2020-01-09 12:00:02') +('2020-01-09 12:00:00','2020-01-09 12:01:00') +('2020-01-09 12:00:00','2020-01-09 13:00:00') +('2020-01-09 00:00:00','2020-01-10 00:00:00') +('2020-01-06','2020-01-13') +('2020-01-01','2020-02-01') +('2020-01-01','2020-04-01') +('2020-01-01','2021-01-01') ('2020-01-09 00:00:00','2020-01-10 00:00:00') 2020-01-09 00:00:00 2020-01-08 00:00:00 @@ -9,6 +17,14 @@ 2020-01-10 00:00:00 2020-01-10 00:00:00 --HOP-- +('2020-01-09 11:59:59','2020-01-09 12:00:02') +('2020-01-09 11:58:00','2020-01-09 12:01:00') +('2020-01-09 10:00:00','2020-01-09 13:00:00') +('2020-01-07 00:00:00','2020-01-10 00:00:00') +('2019-12-23','2020-01-13') +('2019-11-01','2020-02-01') +('2019-07-01','2020-04-01') +('2018-01-01','2021-01-01') ('2020-01-07 00:00:00','2020-01-10 00:00:00') 2020-01-07 00:00:00 2020-01-06 00:00:00 diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.sql b/tests/queries/0_stateless/01049_window_view_window_functions.sql index 09a47d41a96..77041710c27 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.sql +++ b/tests/queries/0_stateless/01049_window_view_window_functions.sql @@ -1,4 +1,13 @@ SELECT '--TUMBLE--'; +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 SECOND); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 MINUTE); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' HOUR); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 WEEK); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' MONTH); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' QUARTER); +SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' YEAR); + SELECT TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); SELECT TUMBLE_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY); SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); @@ -10,6 +19,15 @@ SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTE SELECT TUMBLE_END(TUMBLE(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY)); SELECT '--HOP--'; +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 SECOND, INTERVAL 3 SECOND); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 MINUTE, INTERVAL 3 MINUTE); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 HOUR, INTERVAL 3 HOUR); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 DAY, INTERVAL 3 DAY); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 WEEK, INTERVAL 3 WEEK); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 MONTH, INTERVAL 3 MONTH); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 QUARTER, INTERVAL 3 QUARTER); +SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL 1 YEAR, INTERVAL 3 YEAR); + SELECT HOP(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); SELECT HOP_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY); SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); From 15912c55be772e415773fc0df39c42026b09e516 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 03:07:33 +0800 Subject: [PATCH 078/609] simplify arguments checking --- src/Functions/FunctionsWindow.h | 243 +++++++++++++++++++------------- 1 file changed, 143 insertions(+), 100 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 07eba7ae520..d092df22266 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -175,6 +175,57 @@ namespace } } + void checkFirstArgument(const ColumnWithTypeAndName & argument, const String & function_name) + { + if (!isDateTime(argument.type)) + throw Exception( + "Illegal type " + argument.type->getName() + " of argument of function " + function_name + + ". Should be a date with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) + { + auto interval_type = checkAndGetDataType(argument.type.get()); + if (!interval_type) + throw Exception( + "Illegal type " + argument.type->getName() + " of argument of function " + function_name + + ". Should be an interval of time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + interval_kind = interval_type->getKind(); + result_type_is_date = (interval_type->getKind() == IntervalKind::Year) || (interval_type->getKind() == IntervalKind::Quarter) + || (interval_type->getKind() == IntervalKind::Month) || (interval_type->getKind() == IntervalKind::Week); + } + + void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, bool & result_type_is_date) + { + IntervalKind interval_kind; + checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); + } + + void checkTimeZoneArgument( + const ColumnWithTypeAndName & argument, + const String & function_name) + { + if (!WhichDataType(argument.type).isString()) + throw Exception( + "Illegal type " + argument.type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + bool checkIntervalOrTimeZoneArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) + { + if (WhichDataType(argument.type).isString()) + { + checkTimeZoneArgument(argument, function_name); + return false; + } + checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); + return true; + } + + template struct WindowImpl { @@ -188,25 +239,34 @@ namespace [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - if (arguments.size() != 2 && arguments.size() != 3) + bool result_type_is_date; + + if (arguments.size() == 2) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + checkTimeZoneArgument(arguments.at(2), function_name); + } + else { throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2.", + + ", should be 2 or 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - if (!WhichDataType(arguments[0].type).isDateTime()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[1].type).isInterval()) - throw Exception( - "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString()) - throw Exception( - "Illegal type " + arguments[2].type->getName() + " of argument of function " + function_name - + ". This argument is optional and must be a constant string with timezone name", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); + + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + + return std::make_shared(DataTypes{dataType, dataType}); } [[maybe_unused]] static ColumnPtr @@ -282,31 +342,13 @@ namespace auto type_ = WhichDataType(arguments[0].type); if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Illegal type of first argument of function " + function_name + " should be tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); - return std::make_shared(); - } - else if (arguments.size() == 2 || arguments.size() == 3) - { - if (!WhichDataType(arguments[0].type).isDateTime()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[1].type).isInterval()) - throw Exception( - "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString()) - throw Exception( - "Illegal type " + arguments[2].type->getName() + " of argument of function " + function_name - + ". This argument is optional and must be a constant string with timezone name", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(); + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); } else { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should not larger than 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + WindowImpl::getReturnType(arguments, function_name); } + return std::make_shared(); } [[maybe_unused]] static ColumnPtr @@ -355,28 +397,41 @@ namespace [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - if (arguments.size() != 3 && arguments.size() != 4) + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + } + else if (arguments.size() == 4) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else { throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 3.", + + ", should be 3 or 4", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - if (!WhichDataType(arguments[0].type).isDateTime()) + + if (interval_kind_1 != interval_kind_2) throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[1].type).isInterval()) - throw Exception( - "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[2].type).isInterval()) - throw Exception( - "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) - throw Exception( - "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name - + ". This argument is optional and must be a constant string with timezone name", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(DataTypes{std::make_shared(), std::make_shared()}); + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); + return std::make_shared(DataTypes{dataType, dataType}); } static ColumnPtr @@ -397,12 +452,9 @@ namespace auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - if (std::get<0>(hop_interval) != std::get<0>(window_interval)) - throw Exception( - "Interval type of window and hop column of function " + function_name + ", must be same.", ErrorCodes::ILLEGAL_COLUMN); if (std::get<1>(hop_interval) > std::get<1>(window_interval)) throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval.", + "Value for hop interval of function " + function_name + " must not larger than window interval", ErrorCodes::ARGUMENT_OUT_OF_BOUND); switch (std::get<0>(window_interval)) @@ -480,33 +532,45 @@ namespace [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - if (arguments.size() < 2) + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 2) { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should not smaller than 2.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); } - if (!WhichDataType(arguments[0].type).isDateTime()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[1].type).isInterval()) - throw Exception( - "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() >= 3 && !WhichDataType(arguments[2].type).isInterval()) - throw Exception( - "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) - throw Exception( - "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name - + ". This argument is optional and must be a constant string with timezone name", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (arguments.size() > 4) + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) + { + if (interval_kind_1 != interval_kind_2) + throw Exception( + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + } + } + else if (arguments.size() == 4) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { throw Exception( "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + ", should be 2, 3 or 4", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - return std::make_shared(); + } + + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(); } [[maybe_unused]] static ColumnPtr @@ -628,35 +692,14 @@ namespace auto type_ = WhichDataType(arguments[0].type); if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Illegal type of first argument of function " + function_name + " should be Tuple or UInt32", + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); - return std::make_shared(); - } - else if (arguments.size() == 3 || arguments.size() == 4) - { - if (!WhichDataType(arguments[0].type).isDateTime()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[1].type).isInterval()) - throw Exception( - "Illegal type of second argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (!WhichDataType(arguments[2].type).isInterval()) - throw Exception( - "Illegal type of third argument of function " + function_name + " should be Interval", ErrorCodes::ILLEGAL_COLUMN); - if (arguments.size() == 4 && !WhichDataType(arguments[3].type).isString()) - throw Exception( - "Illegal type " + arguments[3].type->getName() + " of argument of function " + function_name - + ". This argument is optional and must be a constant string with timezone name", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return std::make_shared(); } else { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 1, 3 or 4.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + WindowImpl::getReturnType(arguments, function_name); } + return std::make_shared(); } [[maybe_unused]] static ColumnPtr From 2b179bf84beab92903ff277729555ad0dade9898 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 03:11:27 +0800 Subject: [PATCH 079/609] remote addTime from member function --- src/Storages/WindowView/StorageWindowView.cpp | 68 +++++++++---------- src/Storages/WindowView/StorageWindowView.h | 1 - 2 files changed, 34 insertions(+), 35 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 69964f406d4..04d83aab822 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -250,6 +250,27 @@ namespace if (num_units <= 0) throw Exception("Value for Interval argument must be positive.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } + + UInt32 addTime(UInt32 time_sec, IntervalKind::Kind kind, Int64 num_units, const DateLUTImpl & time_zone) + { + switch (kind) + { +#define CASE_WINDOW_KIND(KIND) \ + case IntervalKind::KIND: { \ + return AddTime::execute(time_sec, num_units, time_zone); \ + } + CASE_WINDOW_KIND(Second) + CASE_WINDOW_KIND(Minute) + CASE_WINDOW_KIND(Hour) + CASE_WINDOW_KIND(Day) + CASE_WINDOW_KIND(Week) + CASE_WINDOW_KIND(Month) + CASE_WINDOW_KIND(Quarter) + CASE_WINDOW_KIND(Year) +#undef CASE_WINDOW_KIND + } + __builtin_unreachable(); + } } static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) @@ -304,7 +325,7 @@ UInt32 StorageWindowView::getCleanupBound() return 0; if (allowed_lateness) { - UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units); + UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, time_zone); lateness_bound = getWindowLowerBound(lateness_bound); if (lateness_bound < w_bound) w_bound = lateness_bound; @@ -568,27 +589,6 @@ std::shared_ptr StorageWindowView::generateInnerTableCreateQuery return inner_create_query; } -inline UInt32 StorageWindowView::addTime(UInt32 time_sec, IntervalKind::Kind kind, Int64 num_units) const -{ - switch (kind) - { -#define CASE_WINDOW_KIND(KIND) \ - case IntervalKind::KIND: { \ - return AddTime::execute(time_sec, num_units, time_zone); \ - } - CASE_WINDOW_KIND(Second) - CASE_WINDOW_KIND(Minute) - CASE_WINDOW_KIND(Hour) - CASE_WINDOW_KIND(Day) - CASE_WINDOW_KIND(Week) - CASE_WINDOW_KIND(Month) - CASE_WINDOW_KIND(Quarter) - CASE_WINDOW_KIND(Year) -#undef CASE_WINDOW_KIND - } - __builtin_unreachable(); -} - inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) { IntervalKind window_interval_kind; @@ -694,12 +694,12 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; max_watermark - = is_tumble ? addTime(max_watermark, window_kind, window_num_units) : addTime(max_watermark, hop_kind, hop_num_units); + = is_tumble ? addTime(max_watermark, window_kind, window_num_units, time_zone) : addTime(max_watermark, hop_kind, hop_num_units, time_zone); } } else // strictly || bounded { - UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units); + UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units, time_zone); updated = max_watermark_bias <= watermark; while (max_watermark_bias <= max_timestamp) { @@ -707,13 +707,13 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) max_fired_watermark = max_watermark; if (is_tumble) { - max_watermark = addTime(max_watermark, window_kind, window_num_units); - max_watermark_bias = addTime(max_watermark, window_kind, window_num_units); + max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); + max_watermark_bias = addTime(max_watermark, window_kind, window_num_units, time_zone); } else { - max_watermark = addTime(max_watermark, hop_kind, hop_num_units); - max_watermark_bias = addTime(max_watermark, hop_kind, hop_num_units); + max_watermark = addTime(max_watermark, hop_kind, hop_num_units, time_zone); + max_watermark_bias = addTime(max_watermark, hop_kind, hop_num_units, time_zone); } } } @@ -757,7 +757,7 @@ void StorageWindowView::threadFuncFireProc() tryLogCurrentException(__PRETTY_FUNCTION__); } max_fired_watermark = next_fire_signal; - next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units); + next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, time_zone); } next_fire_signal = getWindowUpperBound(timestamp_now); @@ -1004,12 +1004,12 @@ void StorageWindowView::writeIntoWindowView(StorageWindowView & window_view, con if (window_view.allowed_lateness && t_max_timestamp != 0) { lateness_bound - = window_view.addTime(t_max_timestamp, window_view.lateness_kind, -1 * window_view.lateness_num_units); + = addTime(t_max_timestamp, window_view.lateness_kind, -1 * window_view.lateness_num_units, *window_view.time_zone); if (window_view.is_watermark_bounded) { UInt32 watermark_lower_bound = window_view.is_tumble - ? window_view.addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units) - : window_view.addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units); + ? addTime(t_max_watermark, window_view.window_kind, -1 * window_view.window_num_units, *window_view.time_zone) + : addTime(t_max_watermark, window_view.hop_kind, -1 * window_view.hop_num_units, *window_view.time_zone); if (watermark_lower_bound < lateness_bound) lateness_bound = watermark_lower_bound; } @@ -1157,7 +1157,7 @@ ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 w_start, UInt32 w_end) cons while (w_end > w_start) { func_array ->arguments->children.push_back(std::make_shared(w_end)); - w_end = addTime(w_end, window_kind, -1 * slice_num_units); + w_end = addTime(w_end, window_kind, -1 * slice_num_units, time_zone); } auto func_has = makeASTFunction("has", func_array, std::make_shared(window_id_name)); res_query->setExpression(ASTSelectQuery::Expression::PREWHERE, func_has); @@ -1175,7 +1175,7 @@ StoragePtr & StorageWindowView::getTargetStorage() const BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 watermark) { - UInt32 w_start = addTime(watermark, window_kind, -1 * window_num_units); + UInt32 w_start = addTime(watermark, window_kind, -1 * window_num_units, time_zone); InterpreterSelectQuery fetch( getFetchColumnQuery(w_start, watermark), diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index cd0f864e8f0..b1ce90fb5e5 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -123,7 +123,6 @@ private: UInt32 getCleanupBound(); ASTPtr generateCleanupQuery(); - UInt32 addTime(UInt32 time_sec, IntervalKind::Kind kind, Int64 num_units) const; UInt32 getWindowLowerBound(UInt32 time_sec); UInt32 getWindowUpperBound(UInt32 time_sec); From 8f7637c6d06c3c728d36d08e60d440366e2718aa Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 12:10:52 +0800 Subject: [PATCH 080/609] time zone support --- src/Storages/WindowView/StorageWindowView.cpp | 63 ++++++++++++------- src/Storages/WindowView/StorageWindowView.h | 2 +- .../01048_window_view_sql_parser.reference | 4 ++ .../01048_window_view_sql_parser.sql | 13 ++++ 4 files changed, 57 insertions(+), 25 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 04d83aab822..8d624d91ecb 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -325,7 +325,7 @@ UInt32 StorageWindowView::getCleanupBound() return 0; if (allowed_lateness) { - UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, time_zone); + UInt32 lateness_bound = addTime(max_timestamp, lateness_kind, -1 * lateness_num_units, *time_zone); lateness_bound = getWindowLowerBound(lateness_bound); if (lateness_bound < w_bound) w_bound = lateness_bound; @@ -603,12 +603,12 @@ inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) case IntervalKind::KIND: \ { \ if (is_tumble) \ - return ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ + return ToStartOfTransform::execute(time_sec, window_num_units, *time_zone); \ else \ {\ - UInt32 w_start = ToStartOfTransform::execute(time_sec, hop_num_units, time_zone); \ - UInt32 w_end = AddTime::execute(w_start, hop_num_units, time_zone);\ - return AddTime::execute(w_end, -1 * window_num_units, time_zone);\ + UInt32 w_start = ToStartOfTransform::execute(time_sec, hop_num_units, *time_zone); \ + UInt32 w_end = AddTime::execute(w_start, hop_num_units, *time_zone);\ + return AddTime::execute(w_end, -1 * window_num_units, *time_zone);\ }\ } CASE_WINDOW_KIND(Second) @@ -639,13 +639,13 @@ inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) { \ if (is_tumble) \ {\ - UInt32 w_start = ToStartOfTransform::execute(time_sec, window_num_units, time_zone); \ - return AddTime::execute(w_start, window_num_units, time_zone); \ + UInt32 w_start = ToStartOfTransform::execute(time_sec, window_num_units, *time_zone); \ + return AddTime::execute(w_start, window_num_units, *time_zone); \ }\ else \ {\ - UInt32 w_start = ToStartOfTransform::execute(time_sec, hop_num_units, time_zone); \ - return AddTime::execute(w_start, hop_num_units, time_zone);\ + UInt32 w_start = ToStartOfTransform::execute(time_sec, hop_num_units, *time_zone); \ + return AddTime::execute(w_start, hop_num_units, *time_zone);\ }\ } CASE_WINDOW_KIND(Second) @@ -694,12 +694,12 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) fire_signal.push_back(max_watermark); max_fired_watermark = max_watermark; max_watermark - = is_tumble ? addTime(max_watermark, window_kind, window_num_units, time_zone) : addTime(max_watermark, hop_kind, hop_num_units, time_zone); + = is_tumble ? addTime(max_watermark, window_kind, window_num_units, *time_zone) : addTime(max_watermark, hop_kind, hop_num_units, *time_zone); } } else // strictly || bounded { - UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units, time_zone); + UInt32 max_watermark_bias = addTime(max_watermark, watermark_kind, watermark_num_units, *time_zone); updated = max_watermark_bias <= watermark; while (max_watermark_bias <= max_timestamp) { @@ -707,13 +707,13 @@ inline void StorageWindowView::updateMaxWatermark(UInt32 watermark) max_fired_watermark = max_watermark; if (is_tumble) { - max_watermark = addTime(max_watermark, window_kind, window_num_units, time_zone); - max_watermark_bias = addTime(max_watermark, window_kind, window_num_units, time_zone); + max_watermark = addTime(max_watermark, window_kind, window_num_units, *time_zone); + max_watermark_bias = addTime(max_watermark, window_kind, window_num_units, *time_zone); } else { - max_watermark = addTime(max_watermark, hop_kind, hop_num_units, time_zone); - max_watermark_bias = addTime(max_watermark, hop_kind, hop_num_units, time_zone); + max_watermark = addTime(max_watermark, hop_kind, hop_num_units, *time_zone); + max_watermark_bias = addTime(max_watermark, hop_kind, hop_num_units, *time_zone); } } } @@ -757,7 +757,7 @@ void StorageWindowView::threadFuncFireProc() tryLogCurrentException(__PRETTY_FUNCTION__); } max_fired_watermark = next_fire_signal; - next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, time_zone); + next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, *time_zone); } next_fire_signal = getWindowUpperBound(timestamp_now); @@ -826,7 +826,6 @@ StorageWindowView::StorageWindowView( bool attach_) : IStorage(table_id_) , global_context(local_context.getGlobalContext()) - , time_zone(DateLUT::instance()) { wv_context = std::make_unique(global_context); wv_context->makeQueryContext(); @@ -854,10 +853,10 @@ StorageWindowView::StorageWindowView( select_table_id = StorageID(select_database_name, select_table_name); DatabaseCatalog::instance().addDependency(select_table_id, table_id_); - // Parser inner query + // Parse inner query auto inner_query = innerQueryParser(select_query); - // Parser mergeable query + // Parse mergeable query mergeable_query = inner_query->clone(); ReplaceFunctionNowData func_now_data; ReplaceFunctionNowVisitor(func_now_data).visit(mergeable_query); @@ -865,7 +864,7 @@ StorageWindowView::StorageWindowView( if (is_time_column_func_now) window_id_name = func_now_data.window_id_name; - // Parser final query + // Parse final query final_query = mergeable_query->clone(); ReplaceWindowIdMatcher::Data final_query_data; if (is_tumble) @@ -921,7 +920,7 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) if (!query.groupBy()) throw Exception("GROUP BY query is required for " + getName(), ErrorCodes::INCORRECT_QUERY); - // parse stage mergeable + // Parse stage mergeable ASTPtr result = query.clone(); ASTPtr expr_list = result; MergeableQueryMatcher::Data stage_mergeable_data; @@ -933,7 +932,7 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) timestamp_column_name = stage_mergeable_data.timestamp_column_name; is_tumble = stage_mergeable_data.is_tumble; - // parser window function + // Parse window function ASTFunction & window_function = typeid_cast(*stage_mergeable_data.window_function); const auto & arguments = window_function.arguments->children; extractWindowArgument( @@ -953,6 +952,22 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) "Illegal type of third argument of function " + window_function.name + " should be Interval"); slice_num_units= std::gcd(hop_num_units, window_num_units); } + + // parse time zone + size_t time_zone_arg_num = is_tumble ? 2 : 3; + if (arguments.size() > time_zone_arg_num) + { + const auto & ast = arguments.at(time_zone_arg_num); + const auto * time_zone_ast = ast->as(); + if (!time_zone_ast || time_zone_ast->value.getType() != Field::Types::String) + throw Exception( + "Illegal column #" + std::to_string(time_zone_arg_num) + " of time zone argument of function, must be constant string", + ErrorCodes::ILLEGAL_COLUMN); + time_zone = &DateLUT::instance(time_zone_ast->value.safeGet()); + } + else + time_zone = &DateLUT::instance(); + return result; } @@ -1157,7 +1172,7 @@ ASTPtr StorageWindowView::getFetchColumnQuery(UInt32 w_start, UInt32 w_end) cons while (w_end > w_start) { func_array ->arguments->children.push_back(std::make_shared(w_end)); - w_end = addTime(w_end, window_kind, -1 * slice_num_units, time_zone); + w_end = addTime(w_end, window_kind, -1 * slice_num_units, *time_zone); } auto func_has = makeASTFunction("has", func_array, std::make_shared(window_id_name)); res_query->setExpression(ASTSelectQuery::Expression::PREWHERE, func_has); @@ -1175,7 +1190,7 @@ StoragePtr & StorageWindowView::getTargetStorage() const BlockInputStreamPtr StorageWindowView::getNewBlocksInputStreamPtr(UInt32 watermark) { - UInt32 w_start = addTime(watermark, window_kind, -1 * window_num_units, time_zone); + UInt32 w_start = addTime(watermark, window_kind, -1 * window_num_units, *time_zone); InterpreterSelectQuery fetch( getFetchColumnQuery(w_start, watermark), diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index b1ce90fb5e5..7fdb0f86f0b 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -71,7 +71,7 @@ private: mutable Block sample_block; mutable Block mergeable_header; UInt64 clean_interval; - const DateLUTImpl & time_zone; + const DateLUTImpl * time_zone = nullptr; UInt32 max_timestamp = 0; UInt32 max_watermark = 0; // next watermark to fire UInt32 max_fired_watermark = 0; diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.reference b/tests/queries/0_stateless/01048_window_view_sql_parser.reference index afe0148e48d..b6b1d386e55 100644 --- a/tests/queries/0_stateless/01048_window_view_sql_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.reference @@ -9,6 +9,8 @@ CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond( ||---FUNCTION--- CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +||---TimeZone--- +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 @@ -19,4 +21,6 @@ CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, t CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +||---TimeZone--- +CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree()\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.sql b/tests/queries/0_stateless/01048_window_view_sql_parser.sql index 578e2add8a8..eebb36aefdf 100644 --- a/tests/queries/0_stateless/01048_window_view_sql_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_sql_parser.sql @@ -40,6 +40,12 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); SHOW CREATE TABLE `.inner.wv`; +SELECT '||---TimeZone---'; +DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM mt GROUP BY wid; +SHOW CREATE TABLE `.inner.wv`; + SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; @@ -71,6 +77,13 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE `.inner.wv`; +SELECT '||---TimeZone---'; +DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS `.inner.wv`; +CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'Asia/Shanghai') as wid; +SHOW CREATE TABLE `.inner.wv`; + + DROP TABLE IF EXISTS wv; DROP TABLE IF EXISTS `.inner.wv`; CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); From 401544c34cc51c504aea19ae68458af22cbe4926 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 27 Jul 2020 17:32:15 +0800 Subject: [PATCH 081/609] fix code style --- src/Functions/FunctionsWindow.h | 14 +++++++------- src/Storages/WindowView/StorageWindowView.cpp | 3 ++- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index d092df22266..01b6f60da1e 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -77,11 +77,11 @@ struct ToStartOfTransform; TRANSFORM_DATE(Week) #undef TRANSFORM_DATE - template <> - struct ToStartOfTransform - { - static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) - { + template <> + struct ToStartOfTransform + { + static UInt32 execute(UInt32 t, UInt64 delta, const DateLUTImpl & time_zone) + { return time_zone.toStartOfDayInterval(time_zone.toDayNum(t), delta); } }; @@ -117,8 +117,8 @@ struct ToStartOfTransform; ADD_DATE(Month) #undef ADD_DATE - template <> - struct AddTime + template <> + struct AddTime { static DayNum execute(UInt16 d, UInt64 delta, const DateLUTImpl &) { return DayNum(d + 7 * delta);} }; diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 8d624d91ecb..9bdcd1fd630 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -50,6 +50,7 @@ namespace ErrorCodes { extern const int ARGUMENT_OUT_OF_BOUND; extern const int BAD_ARGUMENTS; + extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int INCORRECT_QUERY; extern const int LOGICAL_ERROR; @@ -953,7 +954,7 @@ ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) slice_num_units= std::gcd(hop_num_units, window_num_units); } - // parse time zone + // Parse time zone size_t time_zone_arg_num = is_tumble ? 2 : 3; if (arguments.size() > time_zone_arg_num) { From 768eb4ecbf1e66ae9cee276201a342e57b39ca67 Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 28 Jul 2020 02:38:27 +0800 Subject: [PATCH 082/609] remove unnecesary calculation --- src/Storages/WindowView/StorageWindowView.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 9bdcd1fd630..3f7f2df63a5 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -761,7 +761,6 @@ void StorageWindowView::threadFuncFireProc() next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, *time_zone); } - next_fire_signal = getWindowUpperBound(timestamp_now); UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); fire_signal_condition.wait_for(lock, std::chrono::microseconds(static_cast(next_fire_signal) * 1000000 - timestamp_usec)); } From d93e2940d87a2cd96731b9c73c2cb217fdb8d1c8 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sun, 2 Aug 2020 18:40:50 +0800 Subject: [PATCH 083/609] fix drop query freezes when using large window interval --- src/Storages/WindowView/StorageWindowView.cpp | 38 ++++++++----------- 1 file changed, 15 insertions(+), 23 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 3f7f2df63a5..d12a86b3894 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -61,8 +61,6 @@ namespace ErrorCodes namespace { - const auto RESCHEDULE_MS = 500; - struct MergeableQueryMatcher { using Visitor = InDepthNodeVisitor; @@ -737,35 +735,31 @@ void StorageWindowView::threadFuncCleanup() break; } } - if (!shutdown_called) - clean_cache_task->scheduleAfter(RESCHEDULE_MS); } void StorageWindowView::threadFuncFireProc() { std::unique_lock lock(fire_signal_mutex); - while (!shutdown_called) + UInt32 timestamp_now = std::time(nullptr); + while (next_fire_signal <= timestamp_now) { - UInt32 timestamp_now = std::time(nullptr); - while (next_fire_signal <= timestamp_now) + try { - try - { - fire(next_fire_signal); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - max_fired_watermark = next_fire_signal; - next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, *time_zone); + fire(next_fire_signal); } - - UInt64 timestamp_usec = static_cast(Poco::Timestamp().epochMicroseconds()); - fire_signal_condition.wait_for(lock, std::chrono::microseconds(static_cast(next_fire_signal) * 1000000 - timestamp_usec)); + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + max_fired_watermark = next_fire_signal; + next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, *time_zone); } + + UInt64 timestamp_ms = static_cast(Poco::Timestamp().epochMicroseconds()); if (!shutdown_called) - fire_task->scheduleAfter(RESCHEDULE_MS); + { + fire_task->scheduleAfter(std::max(UInt64(0), static_cast(next_fire_signal) * 1000 - timestamp_ms)); + } } void StorageWindowView::threadFuncFireEvent() @@ -783,8 +777,6 @@ void StorageWindowView::threadFuncFireEvent() fire_signal.pop_front(); } } - if (!shutdown_called) - fire_task->scheduleAfter(RESCHEDULE_MS); } BlockInputStreams StorageWindowView::watch( From fc8b5db4868580d5149c088dc63b12b5c4e1d1c5 Mon Sep 17 00:00:00 2001 From: Vxider Date: Mon, 3 Aug 2020 02:07:01 +0800 Subject: [PATCH 084/609] fix timestamp calculation --- src/Storages/WindowView/StorageWindowView.cpp | 4 +- .../01069_window_view_proc_tumble_watch.py | 55 +++++++++++++++++++ ...69_window_view_proc_tumble_watch.reference | 0 3 files changed, 56 insertions(+), 3 deletions(-) create mode 100755 tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py create mode 100644 tests/queries/0_stateless/01069_window_view_proc_tumble_watch.reference diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index d12a86b3894..0d5663f0435 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -755,11 +755,9 @@ void StorageWindowView::threadFuncFireProc() next_fire_signal = addTime(next_fire_signal, window_kind, window_num_units, *time_zone); } - UInt64 timestamp_ms = static_cast(Poco::Timestamp().epochMicroseconds()); + UInt64 timestamp_ms = static_cast(Poco::Timestamp().epochMicroseconds()) / 1000; if (!shutdown_called) - { fire_task->scheduleAfter(std::max(UInt64(0), static_cast(next_fire_signal) * 1000 - timestamp_ms)); - } } void StorageWindowView::threadFuncFireEvent() diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py new file mode 100755 index 00000000000..b27f700131f --- /dev/null +++ b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py @@ -0,0 +1,55 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +# log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('SET allow_experimental_window_view = 1') + client1.expect(prompt) + client1.send('SET window_view_heartbeat_interval = 1') + client1.expect(prompt) + client2.send('SET allow_experimental_window_view = 1') + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.wv') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS `.inner.wv`') + client1.expect(prompt) + + client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') + client1.expect(prompt) + client1.send("CREATE WINDOW VIEW test.wv AS SELECT count(a) AS count FROM test.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid;") + client1.expect(prompt) + + client1.send('WATCH test.wv') + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('Progress: 0.00 rows.*\)') + client1.expect('1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1, now())') + client1.expect('Progress: 0.00 rows.*\)') + client1.expect('1' + end_of_block) + + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.wv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.reference b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.reference new file mode 100644 index 00000000000..e69de29bb2d From e53909cafc8f2c24f2220d3dbd9a487218a97745 Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 7 Aug 2020 17:33:53 +0800 Subject: [PATCH 085/609] rename tests --- ...r.reference => 01047_window_view_parser_inner_table.reference} | 0 ..._table_parser.sql => 01047_window_view_parser_inner_table.sql} | 0 ...ew_sql_parser.reference => 01048_window_view_parser.reference} | 0 ...48_window_view_sql_parser.sql => 01048_window_view_parser.sql} | 0 ...tumble.reference => 01050_window_view_parser_tumble.reference} | 0 ..._sql_parser_tumble.sql => 01050_window_view_parser_tumble.sql} | 0 ...arser_hop.reference => 01051_window_view_parser_hop.reference} | 0 ...w_view_sql_parser_hop.sql => 01051_window_view_parser_hop.sql} | 0 ...o.reference => 01052_window_view_proc_tumble_to_now.reference} | 0 ...now_tumble_to.sql => 01052_window_view_proc_tumble_to_now.sql} | 0 ...p_to.reference => 01053_window_view_proc_hop_to_now.reference} | 0 ..._proc_now_hop_to.sql => 01053_window_view_proc_hop_to_now.sql} | 0 ...nce => 01057_window_view_event_tumble_to_strict_asc.reference} | 0 ...le_to.sql => 01057_window_view_event_tumble_to_strict_asc.sql} | 0 ...erence => 01058_window_view_event_hop_to_strict_asc.reference} | 0 ...c_hop_to.sql => 01058_window_view_event_hop_to_strict_asc.sql} | 0 ...p_watch.py => 01059_window_view_event_hop_watch_strict_asc.py} | 0 ...nce => 01059_window_view_event_hop_watch_strict_asc.reference} | 0 ....reference => 01060_window_view_event_tumble_to_asc.reference} | 0 ...sc_tumble_to.sql => 01060_window_view_event_tumble_to_asc.sql} | 0 ..._to.reference => 01061_window_view_event_hop_to_asc.reference} | 0 ...vent_asc_hop_to.sql => 01061_window_view_event_hop_to_asc.sql} | 0 ..._asc_hop_watch.py => 01062_window_view_event_hop_watch_asc.py} | 0 ....reference => 01062_window_view_event_hop_watch_asc.reference} | 0 ...erence => 01063_window_view_event_tumble_to_bounded.reference} | 0 ...umble_to.sql => 01063_window_view_event_tumble_to_bounded.sql} | 0 ...reference => 01064_window_view_event_hop_to_bounded.reference} | 0 ...nded_hop_to.sql => 01064_window_view_event_hop_to_bounded.sql} | 0 ..._hop_watch.py => 01065_window_view_event_hop_watch_bounded.py} | 0 ...erence => 01065_window_view_event_hop_watch_bounded.reference} | 0 ...066_window_view_event_tumble_to_strict_asc_lateness.reference} | 0 ... => 01066_window_view_event_tumble_to_strict_asc_lateness.sql} | 0 ...e => 01067_window_view_event_tumble_to_asc_lateness.reference} | 0 ...ess.sql => 01067_window_view_event_tumble_to_asc_lateness.sql} | 0 ... 01068_window_view_event_tumble_to_bounded_lateness.reference} | 0 ...sql => 01068_window_view_event_tumble_to_bounded_lateness.sql} | 0 36 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01047_window_view_inner_table_parser.reference => 01047_window_view_parser_inner_table.reference} (100%) rename tests/queries/0_stateless/{01047_window_view_inner_table_parser.sql => 01047_window_view_parser_inner_table.sql} (100%) rename tests/queries/0_stateless/{01048_window_view_sql_parser.reference => 01048_window_view_parser.reference} (100%) rename tests/queries/0_stateless/{01048_window_view_sql_parser.sql => 01048_window_view_parser.sql} (100%) rename tests/queries/0_stateless/{01050_window_view_sql_parser_tumble.reference => 01050_window_view_parser_tumble.reference} (100%) rename tests/queries/0_stateless/{01050_window_view_sql_parser_tumble.sql => 01050_window_view_parser_tumble.sql} (100%) rename tests/queries/0_stateless/{01051_window_view_sql_parser_hop.reference => 01051_window_view_parser_hop.reference} (100%) rename tests/queries/0_stateless/{01051_window_view_sql_parser_hop.sql => 01051_window_view_parser_hop.sql} (100%) rename tests/queries/0_stateless/{01052_window_view_proc_now_tumble_to.reference => 01052_window_view_proc_tumble_to_now.reference} (100%) rename tests/queries/0_stateless/{01052_window_view_proc_now_tumble_to.sql => 01052_window_view_proc_tumble_to_now.sql} (100%) rename tests/queries/0_stateless/{01053_window_view_proc_now_hop_to.reference => 01053_window_view_proc_hop_to_now.reference} (100%) rename tests/queries/0_stateless/{01053_window_view_proc_now_hop_to.sql => 01053_window_view_proc_hop_to_now.sql} (100%) rename tests/queries/0_stateless/{01057_window_view_event_strict_asc_tumble_to.reference => 01057_window_view_event_tumble_to_strict_asc.reference} (100%) rename tests/queries/0_stateless/{01057_window_view_event_strict_asc_tumble_to.sql => 01057_window_view_event_tumble_to_strict_asc.sql} (100%) rename tests/queries/0_stateless/{01058_window_view_event_strict_asc_hop_to.reference => 01058_window_view_event_hop_to_strict_asc.reference} (100%) rename tests/queries/0_stateless/{01058_window_view_event_strict_asc_hop_to.sql => 01058_window_view_event_hop_to_strict_asc.sql} (100%) rename tests/queries/0_stateless/{01059_window_view_event_strict_asc_hop_watch.py => 01059_window_view_event_hop_watch_strict_asc.py} (100%) rename tests/queries/0_stateless/{01059_window_view_event_strict_asc_hop_watch.reference => 01059_window_view_event_hop_watch_strict_asc.reference} (100%) rename tests/queries/0_stateless/{01060_window_view_event_asc_tumble_to.reference => 01060_window_view_event_tumble_to_asc.reference} (100%) rename tests/queries/0_stateless/{01060_window_view_event_asc_tumble_to.sql => 01060_window_view_event_tumble_to_asc.sql} (100%) rename tests/queries/0_stateless/{01061_window_view_event_asc_hop_to.reference => 01061_window_view_event_hop_to_asc.reference} (100%) rename tests/queries/0_stateless/{01061_window_view_event_asc_hop_to.sql => 01061_window_view_event_hop_to_asc.sql} (100%) rename tests/queries/0_stateless/{01062_window_view_event_asc_hop_watch.py => 01062_window_view_event_hop_watch_asc.py} (100%) rename tests/queries/0_stateless/{01062_window_view_event_asc_hop_watch.reference => 01062_window_view_event_hop_watch_asc.reference} (100%) rename tests/queries/0_stateless/{01063_window_view_event_bounded_tumble_to.reference => 01063_window_view_event_tumble_to_bounded.reference} (100%) rename tests/queries/0_stateless/{01063_window_view_event_bounded_tumble_to.sql => 01063_window_view_event_tumble_to_bounded.sql} (100%) rename tests/queries/0_stateless/{01064_window_view_event_bounded_hop_to.reference => 01064_window_view_event_hop_to_bounded.reference} (100%) rename tests/queries/0_stateless/{01064_window_view_event_bounded_hop_to.sql => 01064_window_view_event_hop_to_bounded.sql} (100%) rename tests/queries/0_stateless/{01065_window_view_event_bounded_hop_watch.py => 01065_window_view_event_hop_watch_bounded.py} (100%) rename tests/queries/0_stateless/{01065_window_view_event_bounded_hop_watch.reference => 01065_window_view_event_hop_watch_bounded.reference} (100%) rename tests/queries/0_stateless/{01066_window_view_event_strict_asc_tumble_to_lateness.reference => 01066_window_view_event_tumble_to_strict_asc_lateness.reference} (100%) rename tests/queries/0_stateless/{01066_window_view_event_strict_asc_tumble_to_lateness.sql => 01066_window_view_event_tumble_to_strict_asc_lateness.sql} (100%) rename tests/queries/0_stateless/{01067_window_view_event_asc_tumble_to_lateness.reference => 01067_window_view_event_tumble_to_asc_lateness.reference} (100%) rename tests/queries/0_stateless/{01067_window_view_event_asc_tumble_to_lateness.sql => 01067_window_view_event_tumble_to_asc_lateness.sql} (100%) rename tests/queries/0_stateless/{01068_window_view_event_bounded_tumble_to_lateness.reference => 01068_window_view_event_tumble_to_bounded_lateness.reference} (100%) rename tests/queries/0_stateless/{01068_window_view_event_bounded_tumble_to_lateness.sql => 01068_window_view_event_tumble_to_bounded_lateness.sql} (100%) diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference similarity index 100% rename from tests/queries/0_stateless/01047_window_view_inner_table_parser.reference rename to tests/queries/0_stateless/01047_window_view_parser_inner_table.reference diff --git a/tests/queries/0_stateless/01047_window_view_inner_table_parser.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql similarity index 100% rename from tests/queries/0_stateless/01047_window_view_inner_table_parser.sql rename to tests/queries/0_stateless/01047_window_view_parser_inner_table.sql diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.reference b/tests/queries/0_stateless/01048_window_view_parser.reference similarity index 100% rename from tests/queries/0_stateless/01048_window_view_sql_parser.reference rename to tests/queries/0_stateless/01048_window_view_parser.reference diff --git a/tests/queries/0_stateless/01048_window_view_sql_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql similarity index 100% rename from tests/queries/0_stateless/01048_window_view_sql_parser.sql rename to tests/queries/0_stateless/01048_window_view_parser.sql diff --git a/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference b/tests/queries/0_stateless/01050_window_view_parser_tumble.reference similarity index 100% rename from tests/queries/0_stateless/01050_window_view_sql_parser_tumble.reference rename to tests/queries/0_stateless/01050_window_view_parser_tumble.reference diff --git a/tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql similarity index 100% rename from tests/queries/0_stateless/01050_window_view_sql_parser_tumble.sql rename to tests/queries/0_stateless/01050_window_view_parser_tumble.sql diff --git a/tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference b/tests/queries/0_stateless/01051_window_view_parser_hop.reference similarity index 100% rename from tests/queries/0_stateless/01051_window_view_sql_parser_hop.reference rename to tests/queries/0_stateless/01051_window_view_parser_hop.reference diff --git a/tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql similarity index 100% rename from tests/queries/0_stateless/01051_window_view_sql_parser_hop.sql rename to tests/queries/0_stateless/01051_window_view_parser_hop.sql diff --git a/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.reference b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.reference similarity index 100% rename from tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.reference rename to tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.reference diff --git a/tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql similarity index 100% rename from tests/queries/0_stateless/01052_window_view_proc_now_tumble_to.sql rename to tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql diff --git a/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.reference b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.reference similarity index 100% rename from tests/queries/0_stateless/01053_window_view_proc_now_hop_to.reference rename to tests/queries/0_stateless/01053_window_view_proc_hop_to_now.reference diff --git a/tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql similarity index 100% rename from tests/queries/0_stateless/01053_window_view_proc_now_hop_to.sql rename to tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql diff --git a/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.reference b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.reference similarity index 100% rename from tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.reference rename to tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.reference diff --git a/tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql similarity index 100% rename from tests/queries/0_stateless/01057_window_view_event_strict_asc_tumble_to.sql rename to tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql diff --git a/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.reference b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.reference similarity index 100% rename from tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.reference rename to tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.reference diff --git a/tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql similarity index 100% rename from tests/queries/0_stateless/01058_window_view_event_strict_asc_hop_to.sql rename to tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql diff --git a/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py similarity index 100% rename from tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.py rename to tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py diff --git a/tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.reference b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.reference similarity index 100% rename from tests/queries/0_stateless/01059_window_view_event_strict_asc_hop_watch.reference rename to tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.reference diff --git a/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.reference b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.reference similarity index 100% rename from tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.reference rename to tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.reference diff --git a/tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql similarity index 100% rename from tests/queries/0_stateless/01060_window_view_event_asc_tumble_to.sql rename to tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql diff --git a/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.reference b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.reference similarity index 100% rename from tests/queries/0_stateless/01061_window_view_event_asc_hop_to.reference rename to tests/queries/0_stateless/01061_window_view_event_hop_to_asc.reference diff --git a/tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql similarity index 100% rename from tests/queries/0_stateless/01061_window_view_event_asc_hop_to.sql rename to tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql diff --git a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py similarity index 100% rename from tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.py rename to tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py diff --git a/tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.reference b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.reference similarity index 100% rename from tests/queries/0_stateless/01062_window_view_event_asc_hop_watch.reference rename to tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.reference diff --git a/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.reference b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.reference similarity index 100% rename from tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.reference rename to tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.reference diff --git a/tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql similarity index 100% rename from tests/queries/0_stateless/01063_window_view_event_bounded_tumble_to.sql rename to tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql diff --git a/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.reference b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.reference similarity index 100% rename from tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.reference rename to tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.reference diff --git a/tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql similarity index 100% rename from tests/queries/0_stateless/01064_window_view_event_bounded_hop_to.sql rename to tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql diff --git a/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py similarity index 100% rename from tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.py rename to tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py diff --git a/tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.reference b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.reference similarity index 100% rename from tests/queries/0_stateless/01065_window_view_event_bounded_hop_watch.reference rename to tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.reference diff --git a/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.reference b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.reference similarity index 100% rename from tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.reference rename to tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.reference diff --git a/tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql similarity index 100% rename from tests/queries/0_stateless/01066_window_view_event_strict_asc_tumble_to_lateness.sql rename to tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql diff --git a/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.reference b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.reference similarity index 100% rename from tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.reference rename to tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.reference diff --git a/tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql similarity index 100% rename from tests/queries/0_stateless/01067_window_view_event_asc_tumble_to_lateness.sql rename to tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql diff --git a/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.reference b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.reference similarity index 100% rename from tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.reference rename to tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.reference diff --git a/tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql similarity index 100% rename from tests/queries/0_stateless/01068_window_view_event_bounded_tumble_to_lateness.sql rename to tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql From 635870f50c28212c62f88060eea80afd308ef11a Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 29 May 2021 07:37:53 +0000 Subject: [PATCH 086/609] fix typos --- src/Functions/FunctionsWindow.h | 12 ++++++------ src/Storages/WindowView/StorageWindowView.h | 2 +- src/Storages/WindowView/WindowViewBlockInputStream.h | 2 +- 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 8fbe8dcfa9c..3d7be1e9159 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -269,7 +269,7 @@ namespace return std::make_shared(DataTypes{dataType, dataType}); } - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto & interval_column = arguments[1]; @@ -350,7 +350,7 @@ namespace return std::make_shared(); } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto which_type = WhichDataType(arguments[0].type); ColumnPtr result_column_; @@ -429,7 +429,7 @@ namespace return std::make_shared(DataTypes{dataType, dataType}); } - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto & hop_interval_column = arguments[1]; @@ -658,7 +658,7 @@ namespace return executeWindowBound(column, 1, function_name); } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { if (arguments.size() == 2) return dispatchForTumbleColumns(arguments, function_name); @@ -695,7 +695,7 @@ namespace return std::make_shared(); } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto which_type = WhichDataType(time_column.type); @@ -723,7 +723,7 @@ namespace return WindowImpl::getReturnType(arguments, function_name); } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { const auto & time_column = arguments[0]; const auto which_type = WhichDataType(time_column.type); diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index e765f8a4d19..99915a5995e 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -25,7 +25,7 @@ class StorageWindowView final : public ext::shared_ptr_helper public: ~StorageWindowView() override; String getName() const override { return "WindowView"; } - + bool isView() const override { return true; } bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } diff --git a/src/Storages/WindowView/WindowViewBlockInputStream.h b/src/Storages/WindowView/WindowViewBlockInputStream.h index 01bbfe54ab2..2598292641a 100644 --- a/src/Storages/WindowView/WindowViewBlockInputStream.h +++ b/src/Storages/WindowView/WindowViewBlockInputStream.h @@ -7,7 +7,7 @@ namespace DB { /** Implements WINDOW VIEW table WATCH input stream. - * Keeps stream alive by outputing blocks with no rows + * Keeps stream alive by outputting blocks with no rows * based on window interval. */ class WindowViewBlockInputStream : public IBlockInputStream From b56d471540771677b8644b68a6780a1aa62a7dd1 Mon Sep 17 00:00:00 2001 From: Vxider Date: Sat, 29 May 2021 10:29:38 +0000 Subject: [PATCH 087/609] update code style --- src/Functions/FunctionsWindow.h | 948 +++++++++--------- src/Storages/WindowView/StorageWindowView.cpp | 9 +- .../01052_window_view_proc_tumble_to_now.sql | 0 .../01053_window_view_proc_hop_to_now.sql | 0 .../01054_window_view_proc_tumble_to.sql | 0 .../01055_window_view_proc_hop_to.sql | 0 6 files changed, 479 insertions(+), 478 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql mode change 100755 => 100644 tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql mode change 100755 => 100644 tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql mode change 100755 => 100644 tests/queries/0_stateless/01055_window_view_proc_hop_to.sql diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 3d7be1e9159..5e5ec9db825 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -21,25 +21,25 @@ namespace DB /** Window functions: * * TUMBLE(time_attr, interval [, timezone]) - * + * * TUMBLE_START(window_id) - * + * * TUMBLE_START(time_attr, interval [, timezone]) - * + * * TUMBLE_END(window_id) - * + * * TUMBLE_END(time_attr, interval [, timezone]) - * + * * HOP(time_attr, hop_interval, window_interval [, timezone]) - * + * * HOP_START(window_id) - * + * * HOP_START(time_attr, hop_interval, window_interval [, timezone]) - * + * * HOP_END(window_id) - * + * * HOP_END(time_attr, hop_interval, window_interval [, timezone]) - * + * */ enum WindowFunctionName { @@ -137,7 +137,7 @@ struct ToStartOfTransform; namespace { - static std::tuple + std::tuple dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) { const auto * interval_type = checkAndGetDataType(interval_column.type.get()); @@ -157,7 +157,7 @@ namespace return {interval_type->getKind(), num_units}; } - static ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) + ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) { if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) { @@ -224,523 +224,521 @@ namespace checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); return true; } +} +template +struct WindowImpl +{ + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); +}; - template - struct WindowImpl +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); - }; + bool result_type_is_date; - template <> - struct WindowImpl - { - static constexpr auto name = "TUMBLE"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + if (arguments.size() == 2) { - bool result_type_is_date; + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + checkTimeZoneArgument(arguments.at(2), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } - if (arguments.size() == 2) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); - } - else if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); - checkTimeZoneArgument(arguments.at(2), function_name); - } - else - { + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + + return std::make_shared(DataTypes{dataType, dataType}); + } + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & interval_column = arguments[1]; + const auto & from_datatype = *time_column.type.get(); + const auto which_type = WhichDataType(from_datatype); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); + if (!which_type.isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto interval = dispatchForIntervalColumns(interval_column, function_name); + + switch (std::get<0>(interval)) + { + case IntervalKind::Second: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Minute: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Hour: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Day: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Week: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Month: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Quarter: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Year: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); + for (size_t i = 0; i != size; ++i) + { + start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); + end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE_START"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 1) + { + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - DataTypePtr dataType = nullptr; - if (result_type_is_date) - dataType = std::make_shared(); - else - dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); - - return std::make_shared(DataTypes{dataType, dataType}); + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); } - - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + else { - const auto & time_column = arguments[0]; - const auto & interval_column = arguments[1]; - const auto & from_datatype = *time_column.type.get(); - const auto which_type = WhichDataType(from_datatype); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - if (!which_type.isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto interval = dispatchForIntervalColumns(interval_column, function_name); - - switch (std::get<0>(interval)) - { - case IntervalKind::Second: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Minute: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Hour: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Day: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Week: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Month: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Quarter: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Year: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - } - __builtin_unreachable(); + WindowImpl::getReturnType(arguments, function_name); } + return std::make_shared(); + } - template - static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) - { - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - auto start = ColumnVector::create(); - auto end = ColumnVector::create(); - auto & start_data = start->getData(); - auto & end_data = end->getData(); - start_data.resize(size); - end_data.resize(size); - for (size_t i = 0; i != size; ++i) - { - start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); - end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); - } - MutableColumns result; - result.emplace_back(std::move(start)); - result.emplace_back(std::move(end)); - return ColumnTuple::create(std::move(result)); - } - }; - - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "TUMBLE_START"; + const auto which_type = WhichDataType(arguments[0].type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + else + result_column_ = arguments[0].column; + return executeWindowBound(result_column_, 0, function_name); + } +}; - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE_END"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return WindowImpl::getReturnType(arguments, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + { + const auto which_type = WhichDataType(arguments[0].type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + else + result_column_ = arguments[0].column; + return executeWindowBound(result_column_, 1, function_name); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 3) { - if (arguments.size() == 1) + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + } + else if (arguments.size() == 4) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 3 or 4", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (interval_kind_1 != interval_kind_2) + throw Exception( + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); + return std::make_shared(DataTypes{dataType, dataType}); + } + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & hop_interval_column = arguments[1]; + const auto & window_interval_column = arguments[2]; + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) + { + case IntervalKind::Second: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr + execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); + + for (size_t i = 0; i < size; ++i) + { + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); + wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); + + ToType wend_ = wend; + ToType wend_latest; + + do { - auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isUInt32()) + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); + } while (wend_ > time_data[i]); + + end_data[i] = wend_latest; + start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "WINDOW_ID"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 2) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) + { + if (interval_kind_1 != interval_kind_2) throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); } - else - { - WindowImpl::getReturnType(arguments, function_name); - } - return std::make_shared(); } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + else if (arguments.size() == 4) { - const auto which_type = WhichDataType(arguments[0].type); - ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - else - result_column_ = arguments[0].column; - return executeWindowBound(result_column_, 0, function_name); + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2, 3 or 4", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } - }; + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(); + } - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr + dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "TUMBLE_END"; + const auto & time_column = arguments[0]; + const auto & hop_interval_column = arguments[1]; + const auto & window_interval_column = arguments[2]; + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) { - return WindowImpl::getReturnType(arguments, function_name); + case IntervalKind::Second: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); } + __builtin_unreachable(); + } - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) - { - const auto which_type = WhichDataType(arguments[0].type); - ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - else - result_column_ = arguments[0].column; - return executeWindowBound(result_column_, 1, function_name); - } - }; - - template <> - struct WindowImpl + template + static ColumnPtr + execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) { - static constexpr auto name = "HOP"; + Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + + auto end = ColumnVector::create(); + auto & end_data = end->getData(); + end_data.resize(size); + for (size_t i = 0; i < size; ++i) { - bool result_type_is_date; - IntervalKind interval_kind_1; - IntervalKind interval_kind_2; + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - } - else if (arguments.size() == 4) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - checkTimeZoneArgument(arguments.at(3), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 3 or 4", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } + ToType wend_ = wend; + ToType wend_latest; - if (interval_kind_1 != interval_kind_2) - throw Exception( - "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + do + { + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); + } while (wend_ > time_data[i]); - DataTypePtr dataType = nullptr; - if (result_type_is_date) - dataType = std::make_shared(); - else - dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); - return std::make_shared(DataTypes{dataType, dataType}); + end_data[i] = wend_latest; } + return end; + } - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & hop_interval_column = arguments[1]; - const auto & window_interval_column = arguments[2]; - const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); - if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " argument of function " + function_name - + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); - auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - - if (std::get<1>(hop_interval) > std::get<1>(window_interval)) - throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - switch (std::get<0>(window_interval)) - { - case IntervalKind::Second: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Minute: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Hour: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Day: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Week: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Month: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Quarter: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Year: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr - execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) - { - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - auto start = ColumnVector::create(); - auto end = ColumnVector::create(); - auto & start_data = start->getData(); - auto & end_data = end->getData(); - start_data.resize(size); - end_data.resize(size); - - for (size_t i = 0; i < size; ++i) - { - ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); - ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); - - ToType wend_ = wend; - ToType wend_latest; - - do - { - wend_latest = wend_; - wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); - } while (wend_ > time_data[i]); - - end_data[i] = wend_latest; - start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); - } - MutableColumns result; - result.emplace_back(std::move(start)); - result.emplace_back(std::move(end)); - return ColumnTuple::create(std::move(result)); - } - }; - - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr + dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "WINDOW_ID"; + ColumnPtr column = WindowImpl::dispatchForColumns(arguments, function_name); + return executeWindowBound(column, 1, function_name); + } - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 2) + return dispatchForTumbleColumns(arguments, function_name); + else { - bool result_type_is_date; - IntervalKind interval_kind_1; - IntervalKind interval_kind_2; - - if (arguments.size() == 2) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - } - else if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) - { - if (interval_kind_1 != interval_kind_2) - throw Exception( - "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); - } - } - else if (arguments.size() == 4) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - checkTimeZoneArgument(arguments.at(3), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2, 3 or 4", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - if (result_type_is_date) - return std::make_shared(); - else - return std::make_shared(); - } - - [[maybe_unused]] static ColumnPtr - dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & hop_interval_column = arguments[1]; - const auto & window_interval_column = arguments[2]; - const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); - if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " argument of function " + function_name - + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); - auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - - if (std::get<1>(hop_interval) > std::get<1>(window_interval)) - throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - switch (std::get<0>(window_interval)) - { - case IntervalKind::Second: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Minute: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Hour: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Day: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Week: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Month: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Quarter: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Year: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr - execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) - { - Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); - - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - - auto end = ColumnVector::create(); - auto & end_data = end->getData(); - end_data.resize(size); - for (size_t i = 0; i < size; ++i) - { - ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); - ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - - ToType wend_ = wend; - ToType wend_latest; - - do - { - wend_latest = wend_; - wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); - } while (wend_ > time_data[i]); - - end_data[i] = wend_latest; - } - return end; - } - - [[maybe_unused]] static ColumnPtr - dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - ColumnPtr column = WindowImpl::dispatchForColumns(arguments, function_name); - return executeWindowBound(column, 1, function_name); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - if (arguments.size() == 2) + const auto & third_column = arguments[2]; + if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) return dispatchForTumbleColumns(arguments, function_name); else - { - const auto & third_column = arguments[2]; - if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) - return dispatchForTumbleColumns(arguments, function_name); - else - return dispatchForHopColumns(arguments, function_name); - } + return dispatchForHopColumns(arguments, function_name); } - }; + } +}; - template <> - struct WindowImpl +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP_START"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "HOP_START"; - - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + if (arguments.size() == 1) { - if (arguments.size() == 1) - { - auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isUInt32()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", - ErrorCodes::ILLEGAL_COLUMN); - } - else - { - WindowImpl::getReturnType(arguments, function_name); - } - return std::make_shared(); + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", + ErrorCodes::ILLEGAL_COLUMN); } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + else { - const auto & time_column = arguments[0]; - const auto which_type = WhichDataType(time_column.type); - ColumnPtr result_column_; - if (arguments.size() == 1) - { - if (which_type.isUInt32()) - return time_column.column; - else //isTuple - result_column_ = time_column.column; - } - else - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - return executeWindowBound(result_column_, 0, function_name); + WindowImpl::getReturnType(arguments, function_name); } - }; + return std::make_shared(); + } - template <> - struct WindowImpl + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - static constexpr auto name = "HOP_END"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (arguments.size() == 1) { - return WindowImpl::getReturnType(arguments, function_name); + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; } + else + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + return executeWindowBound(result_column_, 0, function_name); + } +}; - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP_END"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return WindowImpl::getReturnType(arguments, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (arguments.size() == 1) { - const auto & time_column = arguments[0]; - const auto which_type = WhichDataType(time_column.type); - ColumnPtr result_column_; - if (arguments.size() == 1) - { - if (which_type.isUInt32()) - return time_column.column; - else //isTuple - result_column_ = time_column.column; - } - else - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - - return executeWindowBound(result_column_, 1, function_name); + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; } - }; + else + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + + return executeWindowBound(result_column_, 1, function_name); + } }; template diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index e88150ea515..d7fdf759b3b 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -165,8 +165,8 @@ namespace { if (auto * t = ast->as()) { - if (t->name == "HOP" || t->name == "TUMBLE") - t->name = "WINDOW_ID"; + if (t->name == "HOP" || t->name == "TUMBLE") + t->name = "WINDOW_ID"; } } }; @@ -205,7 +205,10 @@ namespace static void visit(const ASTIdentifier & node, ASTPtr & node_ptr, Data & data) { if (node.getColumnName() == data.window_id_alias) - dynamic_cast(node_ptr.get())->setShortName(data.window_id_name); + { + if (auto identifier = std::dynamic_pointer_cast(node_ptr)) + identifier->setShortName(data.window_id_name); + } } }; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql old mode 100755 new mode 100644 diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql old mode 100755 new mode 100644 From d18609467b53f8f50fcee677cc273f4f23cde53f Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 May 2021 13:57:30 +0000 Subject: [PATCH 088/609] First version --- .../AggregateFunctionAggThrow.cpp | 5 +- .../AggregateFunctionArgMinMax.h | 4 +- .../AggregateFunctionArray.h | 18 +++- src/AggregateFunctions/AggregateFunctionAvg.h | 4 +- .../AggregateFunctionBitwise.h | 4 +- .../AggregateFunctionBoundingRatio.h | 4 +- ...egateFunctionCategoricalInformationValue.h | 29 +----- .../AggregateFunctionCount.h | 8 +- .../AggregateFunctionDeltaSum.h | 4 +- .../AggregateFunctionDeltaSumTimestamp.h | 4 +- .../AggregateFunctionDistinct.h | 4 +- .../AggregateFunctionEntropy.h | 4 +- .../AggregateFunctionForEach.h | 16 ++- .../AggregateFunctionGroupArray.h | 8 +- .../AggregateFunctionGroupArrayInsertAt.h | 4 +- .../AggregateFunctionGroupArrayMoving.h | 4 +- .../AggregateFunctionGroupBitmap.h | 8 +- .../AggregateFunctionGroupUniqArray.h | 8 +- .../AggregateFunctionHistogram.h | 4 +- src/AggregateFunctions/AggregateFunctionIf.h | 18 +++- .../AggregateFunctionMLMethod.h | 4 +- .../AggregateFunctionMannWhitney.h | 4 +- .../AggregateFunctionMaxIntersections.h | 4 +- .../AggregateFunctionMerge.h | 18 +++- .../AggregateFunctionMinMaxAny.h | 4 +- .../AggregateFunctionNothing.h | 4 +- .../AggregateFunctionNull.h | 8 +- .../AggregateFunctionOrFill.h | 23 +++-- .../AggregateFunctionQuantile.h | 4 +- .../AggregateFunctionRankCorrelation.h | 4 +- .../AggregateFunctionResample.h | 29 ++---- .../AggregateFunctionRetention.h | 4 +- .../AggregateFunctionSegmentLengthSum.h | 4 +- .../AggregateFunctionSequenceMatch.h | 4 +- .../AggregateFunctionSimpleLinearRegression.h | 15 +-- .../AggregateFunctionSimpleState.h | 28 ++++-- .../AggregateFunctionState.h | 18 +++- .../AggregateFunctionStatistics.h | 8 +- .../AggregateFunctionStatisticsSimple.h | 4 +- src/AggregateFunctions/AggregateFunctionSum.h | 4 +- .../AggregateFunctionSumMap.h | 60 ++++++++++- .../AggregateFunctionTTest.h | 4 +- .../AggregateFunctionTopK.h | 8 +- .../AggregateFunctionUniq.h | 8 +- .../AggregateFunctionUniqCombined.h | 8 +- .../AggregateFunctionUniqUpTo.h | 9 +- .../AggregateFunctionWindowFunnel.h | 4 +- src/AggregateFunctions/IAggregateFunction.h | 10 +- src/Columns/ColumnAggregateFunction.cpp | 4 +- src/Columns/ColumnAggregateFunction.h | 3 +- src/DataTypes/DataTypeAggregateFunction.cpp | 46 ++++++--- src/DataTypes/DataTypeAggregateFunction.h | 19 ++-- .../SerializationAggregateFunction.cpp | 38 +++---- .../SerializationAggregateFunction.h | 3 +- src/Formats/ProtobufSerializer.cpp | 2 +- src/Processors/Transforms/WindowTransform.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 15 +++ .../test_version_update/configs/log_conf.xml | 11 +++ tests/integration/test_version_update/test.py | 99 ++++++++++++++++--- ...1_aggregate_functions_versioning.reference | 1 + .../01881_aggregate_functions_versioning.sql | 10 ++ 61 files changed, 475 insertions(+), 248 deletions(-) create mode 100644 tests/integration/test_version_update/configs/log_conf.xml create mode 100644 tests/queries/0_stateless/01881_aggregate_functions_versioning.reference create mode 100644 tests/queries/0_stateless/01881_aggregate_functions_versioning.sql diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp index c9d292f1993..8eb25953dbd 100644 --- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp +++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp @@ -84,13 +84,13 @@ public: { } - void serialize(ConstAggregateDataPtr, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr, WriteBuffer & buf, std::optional /* version */) const override { char c = 0; buf.write(c); } - void deserialize(AggregateDataPtr, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr /* place */, ReadBuffer & buf, std::optional /* version */, Arena *) const override { char c = 0; buf.read(c); @@ -119,4 +119,3 @@ void registerAggregateFunctionAggThrow(AggregateFunctionFactory & factory) } } - diff --git a/src/AggregateFunctions/AggregateFunctionArgMinMax.h b/src/AggregateFunctions/AggregateFunctionArgMinMax.h index 335ee7c8ecb..66a9a19ce1c 100644 --- a/src/AggregateFunctions/AggregateFunctionArgMinMax.h +++ b/src/AggregateFunctions/AggregateFunctionArgMinMax.h @@ -79,13 +79,13 @@ public: this->data(place).result.change(this->data(rhs).result, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).result.write(buf, *serialization_res); this->data(place).value.write(buf, *serialization_val); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).result.read(buf, *serialization_res, arena); this->data(place).value.read(buf, *serialization_val, arena); diff --git a/src/AggregateFunctions/AggregateFunctionArray.h b/src/AggregateFunctions/AggregateFunctionArray.h index f1005e2e43a..c62a1fd7560 100644 --- a/src/AggregateFunctions/AggregateFunctionArray.h +++ b/src/AggregateFunctions/AggregateFunctionArray.h @@ -48,6 +48,16 @@ public: return nested_func->getReturnType(); } + bool isVersioned() const override + { + return nested_func->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_func->getDefaultVersion(); + } + void create(AggregateDataPtr __restrict place) const override { nested_func->create(place); @@ -110,14 +120,14 @@ public: nested_func->merge(place, rhs, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { - nested_func->serialize(place, buf); + nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - nested_func->deserialize(place, buf, arena); + nested_func->deserialize(place, buf, version, arena); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index 7cdef3bfe69..c593f03d4dd 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -107,7 +107,7 @@ public: this->data(place).denominator += this->data(rhs).denominator; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeBinary(this->data(place).numerator, buf); @@ -117,7 +117,7 @@ public: writeBinary(this->data(place).denominator, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readBinary(this->data(place).numerator, buf); diff --git a/src/AggregateFunctions/AggregateFunctionBitwise.h b/src/AggregateFunctions/AggregateFunctionBitwise.h index 5582a200921..36820fd8910 100644 --- a/src/AggregateFunctions/AggregateFunctionBitwise.h +++ b/src/AggregateFunctions/AggregateFunctionBitwise.h @@ -67,12 +67,12 @@ public: this->data(place).update(this->data(rhs).value); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeBinary(this->data(place).value, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readBinary(this->data(place).value, buf); } diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index e2073c37c63..9176b217625 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -142,12 +142,12 @@ public: data(place).merge(data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h index a32ea97ec0b..1c20e3c50d4 100644 --- a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h +++ b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.h @@ -61,12 +61,7 @@ public: return alignof(T); } - void add( - AggregateDataPtr place, - const IColumn ** columns, - size_t row_num, - Arena * - ) const override + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena *) const override { auto y_col = static_cast(columns[category_count]); bool y = y_col->getData()[row_num]; @@ -83,11 +78,7 @@ public: reinterpret_cast(place)[category_count * 2 + size_t(y)] += 1; } - void merge( - AggregateDataPtr place, - ConstAggregateDataPtr rhs, - Arena * - ) const override + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { for (size_t i : ext::range(0, category_count + 1)) { @@ -96,19 +87,12 @@ public: } } - void serialize( - ConstAggregateDataPtr place, - WriteBuffer & buf - ) const override + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf, std::optional /* version */) const override { buf.write(place, sizeOfData()); } - void deserialize( - AggregateDataPtr place, - ReadBuffer & buf, - Arena * - ) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { buf.read(place, sizeOfData()); } @@ -120,10 +104,7 @@ public: ); } - void insertResultInto( - AggregateDataPtr place, - IColumn & to, - Arena *) const override + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override { auto & col = static_cast(to); auto & data_col = static_cast(col.getData()); diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index 8d9b0a6894e..e64e9512914 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -84,12 +84,12 @@ public: data(place).count += data(rhs).count; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeVarUInt(data(place).count, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readVarUInt(data(place).count, buf); } @@ -141,12 +141,12 @@ public: data(place).count += data(rhs).count; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeVarUInt(data(place).count, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readVarUInt(data(place).count, buf); } diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.h b/src/AggregateFunctions/AggregateFunctionDeltaSum.h index 31d25c559dd..cce83423008 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSum.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.h @@ -102,7 +102,7 @@ public: // Otherwise lhs either has data or is uninitialized, so we don't need to modify its values. } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeIntBinary(this->data(place).sum, buf); writeIntBinary(this->data(place).first, buf); @@ -110,7 +110,7 @@ public: writePODBinary(this->data(place).seen, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readIntBinary(this->data(place).sum, buf); readIntBinary(this->data(place).first, buf); diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h index b7e91e9524a..f265c70f2db 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h @@ -143,7 +143,7 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeIntBinary(this->data(place).sum, buf); writeIntBinary(this->data(place).first, buf); @@ -153,7 +153,7 @@ public: writePODBinary(this->data(place).seen, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readIntBinary(this->data(place).sum, buf); readIntBinary(this->data(place).first, buf); diff --git a/src/AggregateFunctions/AggregateFunctionDistinct.h b/src/AggregateFunctions/AggregateFunctionDistinct.h index 9b7853f8665..3b031445f2c 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/AggregateFunctionDistinct.h @@ -182,12 +182,12 @@ public: this->data(place).merge(this->data(rhs), arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).deserialize(buf, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionEntropy.h b/src/AggregateFunctions/AggregateFunctionEntropy.h index 8e91dd87d77..ad2e43f665a 100644 --- a/src/AggregateFunctions/AggregateFunctionEntropy.h +++ b/src/AggregateFunctions/AggregateFunctionEntropy.h @@ -125,12 +125,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(const_cast(place)).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 66209d8c0f5..db71bc94f08 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -129,6 +129,16 @@ public: return std::make_shared(nested_func->getReturnType()); } + bool isVersioned() const override + { + return nested_func->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_func->getDefaultVersion(); + } + void destroy(AggregateDataPtr __restrict place) const noexcept override { AggregateFunctionForEachData & state = data(place); @@ -196,7 +206,7 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const AggregateFunctionForEachData & state = data(place); writeBinary(state.dynamic_array_size, buf); @@ -209,7 +219,7 @@ public: } } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override { AggregateFunctionForEachData & state = data(place); @@ -221,7 +231,7 @@ public: char * nested_state = state.array_of_aggregate_datas; for (size_t i = 0; i < new_size; ++i) { - nested_func->deserialize(nested_state, buf, arena); + nested_func->deserialize(nested_state, buf, version, arena); nested_state += nested_size_of_data; } } diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 06292992a2f..6f6883ae4b4 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -237,7 +237,7 @@ public: // if constexpr (Trait::sampler == Sampler::DETERMINATOR) } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const auto & value = this->data(place).value; size_t size = value.size(); @@ -256,7 +256,7 @@ public: // if constexpr (Trait::sampler == Sampler::DETERMINATOR) } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { size_t size = 0; readVarUInt(size, buf); @@ -550,7 +550,7 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { writeVarUInt(data(place).value.size(), buf); @@ -570,7 +570,7 @@ public: // if constexpr (Trait::sampler == Sampler::DETERMINATOR) } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { UInt64 elems; readVarUInt(elems, buf); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index b804e4465ac..302f3412b7c 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -144,7 +144,7 @@ public: arr_lhs[i] = arr_rhs[i]; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const Array & arr = data(place).value; size_t size = arr.size(); @@ -164,7 +164,7 @@ public: } } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { size_t size = 0; readVarUInt(size, buf); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index daaa7a69baf..3c2a83ec03b 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -139,7 +139,7 @@ public: cur_elems.sum += rhs_elems.sum; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const auto & value = this->data(place).value; size_t size = value.size(); @@ -147,7 +147,7 @@ public: buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { size_t size = 0; readVarUInt(size, buf); diff --git a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h index cbcac4f7200..05282bf4b4f 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupBitmap.h +++ b/src/AggregateFunctions/AggregateFunctionGroupBitmap.h @@ -35,9 +35,9 @@ public: this->data(place).rbs.merge(this->data(rhs).rbs); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { this->data(place).rbs.write(buf); } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).rbs.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override { this->data(place).rbs.read(buf); } + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).rbs.read(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { @@ -95,9 +95,9 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { this->data(place).rbs.write(buf); } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).rbs.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override { this->data(place).rbs.read(buf); } + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).rbs.read(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index ccba789483f..21dd623d1ce 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -87,7 +87,7 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { auto & set = this->data(place).value; size_t size = set.size(); @@ -96,7 +96,7 @@ public: writeIntBinary(elem, buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).value.read(buf); } @@ -169,7 +169,7 @@ public: return true; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { auto & set = this->data(place).value; writeVarUInt(set.size(), buf); @@ -180,7 +180,7 @@ public: } } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { auto & set = this->data(place).value; size_t size; diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 5268f84e7ac..fcbbe3d7f98 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -346,12 +346,12 @@ public: this->data(place).merge(this->data(rhs), max_bins); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf, max_bins); } diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index 5ef8e3bc75a..bba9ba2aed8 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -50,6 +50,16 @@ public: return nested_func->getReturnType(); } + bool isVersioned() const override + { + return nested_func->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_func->getDefaultVersion(); + } + void create(AggregateDataPtr __restrict place) const override { nested_func->create(place); @@ -124,14 +134,14 @@ public: nested_func->mergeBatch(batch_size, places, place_offset, rhs, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { - nested_func->serialize(place, buf); + nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - nested_func->deserialize(place, buf, arena); + nested_func->deserialize(place, buf, version, arena); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionMLMethod.h b/src/AggregateFunctions/AggregateFunctionMLMethod.h index a97b565eed3..952a4ee5ce0 100644 --- a/src/AggregateFunctions/AggregateFunctionMLMethod.h +++ b/src/AggregateFunctions/AggregateFunctionMLMethod.h @@ -357,9 +357,9 @@ public: void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { this->data(place).write(buf); } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override { this->data(place).read(buf); } + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf); } void predictValues( ConstAggregateDataPtr place, diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.h b/src/AggregateFunctions/AggregateFunctionMannWhitney.h index 365bc45f7b7..a6fb715f492 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.h +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.h @@ -215,12 +215,12 @@ public: a.merge(b, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).read(buf, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 2b7b28266c1..6f125610936 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -110,7 +110,7 @@ public: cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { const auto & value = this->data(place).value; size_t size = value.size(); @@ -118,7 +118,7 @@ public: buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { size_t size = 0; readVarUInt(size, buf); diff --git a/src/AggregateFunctions/AggregateFunctionMerge.h b/src/AggregateFunctions/AggregateFunctionMerge.h index d129b829f9c..5c4800b16b1 100644 --- a/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/src/AggregateFunctions/AggregateFunctionMerge.h @@ -50,6 +50,16 @@ public: return nested_func->getReturnType(); } + bool isVersioned() const override + { + return nested_func->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_func->getDefaultVersion(); + } + void create(AggregateDataPtr __restrict place) const override { nested_func->create(place); @@ -85,14 +95,14 @@ public: nested_func->merge(place, rhs, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { - nested_func->serialize(place, buf); + nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - nested_func->deserialize(place, buf, arena); + nested_func->deserialize(place, buf, version, arena); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index de1f4fad296..e24ca085c49 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -735,12 +735,12 @@ public: this->data(place).changeIfBetter(this->data(rhs), arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf, *serialization); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).read(buf, *serialization, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionNothing.h b/src/AggregateFunctions/AggregateFunctionNothing.h index 1f5b95c5cf3..4374ecf85c3 100644 --- a/src/AggregateFunctions/AggregateFunctionNothing.h +++ b/src/AggregateFunctions/AggregateFunctionNothing.h @@ -62,11 +62,11 @@ public: { } - void serialize(ConstAggregateDataPtr, WriteBuffer &) const override + void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { } - void deserialize(AggregateDataPtr, ReadBuffer &, Arena *) const override + void deserialize(AggregateDataPtr, ReadBuffer &, std::optional, Arena *) const override { } diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index a0e36d1bc3d..d97bae3ebcc 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -130,16 +130,16 @@ public: nested_function->merge(nestedPlace(place), nestedPlace(rhs), arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { bool flag = getFlag(place); if constexpr (serialize_flag) writeBinary(flag, buf); if (flag) - nested_function->serialize(nestedPlace(place), buf); + nested_function->serialize(nestedPlace(place), buf, version); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { bool flag = 1; if constexpr (serialize_flag) @@ -147,7 +147,7 @@ public: if (flag) { setFlag(place); - nested_function->deserialize(nestedPlace(place), buf, arena); + nested_function->deserialize(nestedPlace(place), buf, version, arena); } } diff --git a/src/AggregateFunctions/AggregateFunctionOrFill.h b/src/AggregateFunctions/AggregateFunctionOrFill.h index aaafb8cc2c0..7d33b56adad 100644 --- a/src/AggregateFunctions/AggregateFunctionOrFill.h +++ b/src/AggregateFunctions/AggregateFunctionOrFill.h @@ -52,6 +52,16 @@ public: return nested_function->getName() + "OrDefault"; } + bool isVersioned() const override + { + return nested_function->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_function->getDefaultVersion(); + } + bool isState() const override { return nested_function->isState(); @@ -209,21 +219,16 @@ public: (places[i] + place_offset)[size_of_data] |= rhs[i][size_of_data]; } - void serialize( - ConstAggregateDataPtr place, - WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf, std::optional version) const override { - nested_function->serialize(place, buf); + nested_function->serialize(place, buf, version); writeChar(place[size_of_data], buf); } - void deserialize( - AggregateDataPtr place, - ReadBuffer & buf, - Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - nested_function->deserialize(place, buf, arena); + nested_function->deserialize(place, buf, version, arena); readChar(place[size_of_data], buf); } diff --git a/src/AggregateFunctions/AggregateFunctionQuantile.h b/src/AggregateFunctions/AggregateFunctionQuantile.h index e7e60fe1def..b34f13da295 100644 --- a/src/AggregateFunctions/AggregateFunctionQuantile.h +++ b/src/AggregateFunctions/AggregateFunctionQuantile.h @@ -131,13 +131,13 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { /// const_cast is required because some data structures apply finalizaton (like compactization) before serializing. this->data(const_cast(place)).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h index b7da1f28fd3..9d181210f5a 100644 --- a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h +++ b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h @@ -82,12 +82,12 @@ public: a.merge(b, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).read(buf, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionResample.h b/src/AggregateFunctions/AggregateFunctionResample.h index 45aad8c1a56..deea9538daf 100644 --- a/src/AggregateFunctions/AggregateFunctionResample.h +++ b/src/AggregateFunctions/AggregateFunctionResample.h @@ -134,11 +134,7 @@ public: nested_function->destroy(place + i * size_of_data); } - void add( - AggregateDataPtr place, - const IColumn ** columns, - size_t row_num, - Arena * arena) const override + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override { Key key; @@ -155,30 +151,22 @@ public: nested_function->add(place + pos * size_of_data, columns, row_num, arena); } - void merge( - AggregateDataPtr place, - ConstAggregateDataPtr rhs, - Arena * arena) const override + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override { for (size_t i = 0; i < total; ++i) nested_function->merge(place + i * size_of_data, rhs + i * size_of_data, arena); } - void serialize( - ConstAggregateDataPtr place, - WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf, std::optional version) const override { for (size_t i = 0; i < total; ++i) - nested_function->serialize(place + i * size_of_data, buf); + nested_function->serialize(place + i * size_of_data, buf, version); } - void deserialize( - AggregateDataPtr place, - ReadBuffer & buf, - Arena * arena) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override { for (size_t i = 0; i < total; ++i) - nested_function->deserialize(place + i * size_of_data, buf, arena); + nested_function->deserialize(place + i * size_of_data, buf, version, arena); } DataTypePtr getReturnType() const override @@ -186,10 +174,7 @@ public: return std::make_shared(nested_function->getReturnType()); } - void insertResultInto( - AggregateDataPtr place, - IColumn & to, - Arena * arena) const override + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override { auto & col = assert_cast(to); auto & col_offsets = assert_cast(col.getOffsetsColumn()); diff --git a/src/AggregateFunctions/AggregateFunctionRetention.h b/src/AggregateFunctions/AggregateFunctionRetention.h index 1496c92681a..ed7dccc10f9 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/src/AggregateFunctions/AggregateFunctionRetention.h @@ -115,12 +115,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.h b/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.h index 0ca70cc7367..ed047be7d64 100644 --- a/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.h +++ b/src/AggregateFunctions/AggregateFunctionSegmentLengthSum.h @@ -177,12 +177,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index f0fd1e33dca..287182b3c43 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -169,12 +169,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h index e557ffe5d34..333b847ea67 100644 --- a/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h +++ b/src/AggregateFunctions/AggregateFunctionSimpleLinearRegression.h @@ -125,26 +125,17 @@ public: this->data(place).add(x, y); } - void merge( - AggregateDataPtr place, - ConstAggregateDataPtr rhs, Arena * - ) const override + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena *) const override { this->data(place).merge(this->data(rhs)); } - void serialize( - ConstAggregateDataPtr place, - WriteBuffer & buf - ) const override + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize( - AggregateDataPtr place, - ReadBuffer & buf, Arena * - ) const override + void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSimpleState.h b/src/AggregateFunctions/AggregateFunctionSimpleState.h index 9d34113b028..d32d9a4f806 100644 --- a/src/AggregateFunctions/AggregateFunctionSimpleState.h +++ b/src/AggregateFunctions/AggregateFunctionSimpleState.h @@ -49,6 +49,16 @@ public: return storage_type; } + bool isVersioned() const override + { + return nested_func->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_func->getDefaultVersion(); + } + void create(AggregateDataPtr __restrict place) const override { nested_func->create(place); } void destroy(AggregateDataPtr __restrict place) const noexcept override { nested_func->destroy(place); } @@ -64,13 +74,19 @@ public: nested_func->add(place, columns, row_num, arena); } - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { nested_func->merge(place, rhs, arena); } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { nested_func->serialize(place, buf); } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { - nested_func->deserialize(place, buf, arena); + nested_func->merge(place, rhs, arena); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override + { + nested_func->serialize(place, buf, version); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override + { + nested_func->deserialize(place, buf, version, arena); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionState.h b/src/AggregateFunctions/AggregateFunctionState.h index b33bb64c6dd..41fbd96a9b5 100644 --- a/src/AggregateFunctions/AggregateFunctionState.h +++ b/src/AggregateFunctions/AggregateFunctionState.h @@ -35,6 +35,16 @@ public: DataTypePtr getReturnType() const override; + bool isVersioned() const override + { + return nested_func->isVersioned(); + } + + size_t getDefaultVersion() const override + { + return nested_func->getDefaultVersion(); + } + void create(AggregateDataPtr __restrict place) const override { nested_func->create(place); @@ -70,14 +80,14 @@ public: nested_func->merge(place, rhs, arena); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { - nested_func->serialize(place, buf); + nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - nested_func->deserialize(place, buf, arena); + nested_func->deserialize(place, buf, version, arena); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionStatistics.h b/src/AggregateFunctions/AggregateFunctionStatistics.h index 97ea9ad8917..85178bc2413 100644 --- a/src/AggregateFunctions/AggregateFunctionStatistics.h +++ b/src/AggregateFunctions/AggregateFunctionStatistics.h @@ -136,12 +136,12 @@ public: this->data(place).mergeWith(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } @@ -390,12 +390,12 @@ public: this->data(place).mergeWith(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h index e57781fac33..b360aceb3dd 100644 --- a/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h +++ b/src/AggregateFunctions/AggregateFunctionStatisticsSimple.h @@ -148,12 +148,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSum.h b/src/AggregateFunctions/AggregateFunctionSum.h index 7261e56c044..d44d2ccedde 100644 --- a/src/AggregateFunctions/AggregateFunctionSum.h +++ b/src/AggregateFunctions/AggregateFunctionSum.h @@ -369,12 +369,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index ec2f24d12cb..e1bf8f355f4 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -17,6 +17,8 @@ #include #include #include +#include +#include namespace DB @@ -38,7 +40,7 @@ struct AggregateFunctionMapData std::map merged_maps; }; -/** Aggregate function, that takes at least two arguments: keys and values, and as a result, builds a tuple of of at least 2 arrays - +/** Aggregate function, that takes at least two arguments: keys and values, and as a result, builds a tuple of at least 2 arrays - * ordered keys and variable number of argument values aggregated by corresponding keys. * * sumMap function is the most useful when using SummingMergeTree to sum Nested columns, which name ends in "Map". @@ -85,6 +87,18 @@ public: values_serializations.emplace_back(type->getDefaultSerialization()); } + bool isVersioned() const override { return true; } + + size_t getDefaultVersion() const override { return 1; } + + size_t getVersionFromRevision(size_t revision) const override + { + if (revision >= 54448) + return 1; + else + return 0; + } + DataTypePtr getReturnType() const override { DataTypes types; @@ -250,8 +264,12 @@ public: } } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version) const override { + if (!version) + version = getDefaultVersion(); + + LOG_TRACE(&Poco::Logger::get("kssenii"), "version to serialize: {}, stack: {}", *version, StackTrace().toString()); const auto & merged_maps = this->data(place).merged_maps; size_t size = merged_maps.size(); writeVarUInt(size, buf); @@ -260,12 +278,30 @@ public: { keys_serialization->serializeBinary(elem.first, buf); for (size_t col = 0; col < values_types.size(); ++col) - values_serializations[col]->serializeBinary(elem.second[col], buf); + { + switch (*version) + { + case 0: + { + values_serializations[col]->serializeBinary(elem.second[col], buf); + break; + } + case 1: + { + SerializationNumber().serializeBinary(elem.second[col], buf); + break; + } + } + } } } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena *) const override { + if (!version) + version = getDefaultVersion(); + + LOG_TRACE(&Poco::Logger::get("kssenii"), "version to deserialize: {}, stack: {}", *version, StackTrace().toString()); auto & merged_maps = this->data(place).merged_maps; size_t size = 0; readVarUInt(size, buf); @@ -278,7 +314,21 @@ public: Array values; values.resize(values_types.size()); for (size_t col = 0; col < values_types.size(); ++col) - values_serializations[col]->deserializeBinary(values[col], buf); + { + switch (*version) + { + case 0: + { + values_serializations[col]->deserializeBinary(values[col], buf); + break; + } + case 1: + { + SerializationNumber().deserializeBinary(values[col], buf); + break; + } + } + } if constexpr (IsDecimalNumber) merged_maps[key.get>()] = values; diff --git a/src/AggregateFunctions/AggregateFunctionTTest.h b/src/AggregateFunctions/AggregateFunctionTTest.h index 001de95c547..a91ce16c3ff 100644 --- a/src/AggregateFunctions/AggregateFunctionTTest.h +++ b/src/AggregateFunctions/AggregateFunctionTTest.h @@ -128,12 +128,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionTopK.h b/src/AggregateFunctions/AggregateFunctionTopK.h index 30d69b8ca7b..56184efee3d 100644 --- a/src/AggregateFunctions/AggregateFunctionTopK.h +++ b/src/AggregateFunctions/AggregateFunctionTopK.h @@ -72,12 +72,12 @@ public: set.merge(this->data(rhs).value); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).value.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { auto & set = this->data(place).value; set.resize(reserved); @@ -147,12 +147,12 @@ public: return true; } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).value.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { auto & set = this->data(place).value; set.clear(); diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index efbb9758b03..537088cf2f9 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -242,12 +242,12 @@ public: this->data(place).set.merge(this->data(rhs).set); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).set.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).set.read(buf); } @@ -299,12 +299,12 @@ public: this->data(place).set.merge(this->data(rhs).set); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).set.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).set.read(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.h b/src/AggregateFunctions/AggregateFunctionUniqCombined.h index cb623bcb761..809253cd4ae 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.h +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.h @@ -162,12 +162,12 @@ public: this->data(place).set.merge(this->data(rhs).set); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).set.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).set.read(buf); } @@ -226,12 +226,12 @@ public: this->data(place).set.merge(this->data(rhs).set); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).set.write(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).set.read(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index a1599a0935b..2a8706e7132 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -209,12 +209,12 @@ public: this->data(place).merge(this->data(rhs), threshold); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf, threshold); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf, threshold); } @@ -273,12 +273,12 @@ public: this->data(place).merge(this->data(rhs), threshold); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).write(buf, threshold); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf, threshold); } @@ -295,4 +295,3 @@ public: #if !defined(__clang__) #pragma GCC diagnostic pop #endif - diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h index 591596461f8..acfc7795652 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.h +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.h @@ -281,12 +281,12 @@ public: this->data(place).merge(this->data(rhs)); } - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 2d47a1e7fe0..dfadd76766a 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -67,6 +67,12 @@ public: throw Exception("Prediction is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + virtual bool isVersioned() const { return false; } + + virtual size_t getVersionFromRevision(size_t /* revision */) const { return 0; } + + virtual size_t getDefaultVersion() const { return 0; } + virtual ~IAggregateFunction() = default; /** Data manipulating functions. */ @@ -99,10 +105,10 @@ public: virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0; /// Serializes state (to transmit it over the network, for example). - virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const = 0; + virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional verison = std::nullopt) const = 0; /// Deserializes state. This function is called only for empty (just created) states. - virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const = 0; + virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version = std::nullopt, Arena * arena = nullptr) const = 0; /// Returns true if a function requires Arena to handle own states (see add(), merge(), deserialize()). virtual bool allocatesMemoryInArena() const = 0; diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index a5bfcae05e4..49fde42b9b0 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -514,7 +514,7 @@ void ColumnAggregateFunction::insert(const Field & x) Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); ReadBufferFromString read_buffer(x.get().data); - func->deserialize(data.back(), read_buffer, &arena); + func->deserialize(data.back(), read_buffer, std::nullopt, &arena); } void ColumnAggregateFunction::insertDefault() @@ -549,7 +549,7 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * * Probably this will not work under UBSan. */ ReadBufferFromMemory read_buffer(src_arena, std::numeric_limits::max() - src_arena - 1); - func->deserialize(data.back(), read_buffer, &dst_arena); + func->deserialize(data.back(), read_buffer, std::nullopt, &dst_arena); return read_buffer.position(); } diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index e75af280ed5..059d7c330a1 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -94,8 +94,7 @@ private: ColumnAggregateFunction(const AggregateFunctionPtr & func_); - ColumnAggregateFunction(const AggregateFunctionPtr & func_, - const ConstArenas & arenas_); + ColumnAggregateFunction(const AggregateFunctionPtr & func_, const ConstArenas & arenas_); ColumnAggregateFunction(const ColumnAggregateFunction & src_); diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 7f7b01e031b..3f8beafcfd3 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB @@ -35,10 +36,14 @@ namespace ErrorCodes } -std::string DataTypeAggregateFunction::doGetName() const +String DataTypeAggregateFunction::doGetName() const { + LOG_TRACE(&Poco::Logger::get("kssenii"), "doGetName stack trace:{}", StackTrace().toString()); WriteBufferFromOwnString stream; - stream << "AggregateFunction(" << function->getName(); + stream << "AggregateFunction("; + if (version) + stream << version << ", "; + stream << function->getName(); if (!parameters.empty()) { @@ -61,10 +66,10 @@ std::string DataTypeAggregateFunction::doGetName() const MutableColumnPtr DataTypeAggregateFunction::createColumn() const { + /// FIXME: ColumnAggregateFunction also uses function->serialize methods return ColumnAggregateFunction::create(function); } - /// Create empty state Field DataTypeAggregateFunction::getDefault() const { @@ -79,7 +84,7 @@ Field DataTypeAggregateFunction::getDefault() const try { WriteBufferFromString buffer_from_field(field.get().data); - function->serialize(place, buffer_from_field); + function->serialize(place, buffer_from_field, version); } catch (...) { @@ -100,26 +105,46 @@ bool DataTypeAggregateFunction::equals(const IDataType & rhs) const SerializationPtr DataTypeAggregateFunction::doGetDefaultSerialization() const { - return std::make_shared(function); + LOG_TRACE(&Poco::Logger::get("kssenii"), "get serializaton version: {}, name: {}, stack: {}", version, getName(), StackTrace().toString()); + return std::make_shared(function, version); } static DataTypePtr create(const ASTPtr & arguments) { + LOG_TRACE(&Poco::Logger::get("kssenii"), "create data type: {}", StackTrace().toString()); + String function_name; AggregateFunctionPtr function; DataTypes argument_types; Array params_row; + std::optional version; if (!arguments || arguments->children.empty()) throw Exception("Data type AggregateFunction requires parameters: " "name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (const auto * parametric = arguments->children[0]->as()) + ASTPtr data_type_ast = arguments->children[0]; + size_t argument_types_start_idx = 1; + + /* If aggregate function definition doesn't have version, it will have in AST children args [ASTFunction, types...] - in case + * it is parametric, or [ASTIdentifier, types...] - otherwise. If aggregate function has version in AST, then it will be: + * [ASTLitearl, ASTFunction (or ASTIdentifier), types]. + */ + if (auto version_ast = arguments->children[0]->as()) + { + version = version_ast->value.safeGet(); + data_type_ast = arguments->children[1]; + argument_types_start_idx = 2; + } + + if (const auto * parametric = data_type_ast->as()) { if (parametric->parameters) throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR); + function_name = parametric->name; + LOG_TRACE(&Poco::Logger::get("kssenii"), "Paramtric function name: {}", function_name); if (parametric->arguments) { @@ -140,11 +165,11 @@ static DataTypePtr create(const ASTPtr & arguments) } } } - else if (auto opt_name = tryGetIdentifierName(arguments->children[0])) + else if (auto opt_name = tryGetIdentifierName(data_type_ast)) { function_name = *opt_name; } - else if (arguments->children[0]->as()) + else if (data_type_ast->as()) { throw Exception("Aggregate function name for data type AggregateFunction must be passed as identifier (without quotes) or function", ErrorCodes::BAD_ARGUMENTS); @@ -153,7 +178,7 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception("Unexpected AST element passed as aggregate function name for data type AggregateFunction. Must be identifier or function.", ErrorCodes::BAD_ARGUMENTS); - for (size_t i = 1; i < arguments->children.size(); ++i) + for (size_t i = argument_types_start_idx; i < arguments->children.size(); ++i) argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i])); if (function_name.empty()) @@ -161,7 +186,7 @@ static DataTypePtr create(const ASTPtr & arguments) AggregateFunctionProperties properties; function = AggregateFunctionFactory::instance().get(function_name, argument_types, params_row, properties); - return std::make_shared(function, argument_types, params_row); + return std::make_shared(function, argument_types, params_row, version); } void registerDataTypeAggregateFunction(DataTypeFactory & factory) @@ -169,5 +194,4 @@ void registerDataTypeAggregateFunction(DataTypeFactory & factory) factory.registerDataType("AggregateFunction", create); } - } diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index c3fea2ba727..ac486087205 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -3,6 +3,7 @@ #include #include +#include namespace DB @@ -17,19 +18,24 @@ private: AggregateFunctionPtr function; DataTypes argument_types; Array parameters; + mutable size_t version; public: static constexpr bool is_parametric = true; - DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_) - : function(function_), argument_types(argument_types_), parameters(parameters_) + DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, + const Array & parameters_, std::optional version_ = std::nullopt) + : function(function_) + , argument_types(argument_types_) + , parameters(parameters_) + , version(version_ ? *version_ : function_->getDefaultVersion()) { } - std::string getFunctionName() const { return function->getName(); } + String getFunctionName() const { return function->getName(); } AggregateFunctionPtr getFunction() const { return function; } - std::string doGetName() const override; + String doGetName() const override; const char * getFamilyName() const override { return "AggregateFunction"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } @@ -50,8 +56,9 @@ public: bool shouldAlignRightInPrettyFormats() const override { return false; } SerializationPtr doGetDefaultSerialization() const override; + + bool isVersioned() const { return function->isVersioned(); } + void setVersion(size_t version_) const { version = version_; } }; - } - diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index e0bcb65d895..d8130c92652 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -37,7 +37,7 @@ void SerializationAggregateFunction::deserializeBinary(Field & field, ReadBuffer void SerializationAggregateFunction::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const { - function->serialize(assert_cast(column).getData()[row_num], ostr); + function->serialize(assert_cast(column).getData()[row_num], ostr, version); } void SerializationAggregateFunction::deserializeBinary(IColumn & column, ReadBuffer & istr) const @@ -51,7 +51,7 @@ void SerializationAggregateFunction::deserializeBinary(IColumn & column, ReadBuf function->create(place); try { - function->deserialize(place, istr, &arena); + function->deserialize(place, istr, version, &arena); } catch (...) { @@ -74,7 +74,7 @@ void SerializationAggregateFunction::serializeBinaryBulk(const IColumn & column, end = vec.end(); for (; it != end; ++it) - function->serialize(*it, ostr); + function->serialize(*it, ostr, version); } void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const @@ -100,7 +100,7 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea try { - function->deserialize(place, istr, &arena); + function->deserialize(place, istr, version, &arena); } catch (...) { @@ -112,14 +112,14 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea } } -static String serializeToString(const AggregateFunctionPtr & function, const IColumn & column, size_t row_num) +static String serializeToString(const AggregateFunctionPtr & function, const IColumn & column, size_t row_num, size_t version) { WriteBufferFromOwnString buffer; - function->serialize(assert_cast(column).getData()[row_num], buffer); + function->serialize(assert_cast(column).getData()[row_num], buffer, version); return buffer.str(); } -static void deserializeFromString(const AggregateFunctionPtr & function, IColumn & column, const String & s) +static void deserializeFromString(const AggregateFunctionPtr & function, IColumn & column, const String & s, size_t version) { ColumnAggregateFunction & column_concrete = assert_cast(column); @@ -132,7 +132,7 @@ static void deserializeFromString(const AggregateFunctionPtr & function, IColumn try { ReadBufferFromString istr(s); - function->deserialize(place, istr, &arena); + function->deserialize(place, istr, version, &arena); } catch (...) { @@ -145,13 +145,13 @@ static void deserializeFromString(const AggregateFunctionPtr & function, IColumn void SerializationAggregateFunction::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeString(serializeToString(function, column, row_num), ostr); + writeString(serializeToString(function, column, row_num, version), ostr); } void SerializationAggregateFunction::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeEscapedString(serializeToString(function, column, row_num), ostr); + writeEscapedString(serializeToString(function, column, row_num, version), ostr); } @@ -159,13 +159,13 @@ void SerializationAggregateFunction::deserializeTextEscaped(IColumn & column, Re { String s; readEscapedString(s, istr); - deserializeFromString(function, column, s); + deserializeFromString(function, column, s, version); } void SerializationAggregateFunction::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeQuotedString(serializeToString(function, column, row_num), ostr); + writeQuotedString(serializeToString(function, column, row_num, version), ostr); } @@ -173,7 +173,7 @@ void SerializationAggregateFunction::deserializeTextQuoted(IColumn & column, Rea { String s; readQuotedStringWithSQLStyle(s, istr); - deserializeFromString(function, column, s); + deserializeFromString(function, column, s, version); } @@ -181,13 +181,13 @@ void SerializationAggregateFunction::deserializeWholeText(IColumn & column, Read { String s; readStringUntilEOF(s, istr); - deserializeFromString(function, column, s); + deserializeFromString(function, column, s, version); } void SerializationAggregateFunction::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { - writeJSONString(serializeToString(function, column, row_num), ostr, settings); + writeJSONString(serializeToString(function, column, row_num, version), ostr, settings); } @@ -195,19 +195,19 @@ void SerializationAggregateFunction::deserializeTextJSON(IColumn & column, ReadB { String s; readJSONString(s, istr); - deserializeFromString(function, column, s); + deserializeFromString(function, column, s, version); } void SerializationAggregateFunction::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeXMLStringForTextElement(serializeToString(function, column, row_num), ostr); + writeXMLStringForTextElement(serializeToString(function, column, row_num, version), ostr); } void SerializationAggregateFunction::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeCSV(serializeToString(function, column, row_num), ostr); + writeCSV(serializeToString(function, column, row_num, version), ostr); } @@ -215,7 +215,7 @@ void SerializationAggregateFunction::deserializeTextCSV(IColumn & column, ReadBu { String s; readCSV(s, istr, settings.csv); - deserializeFromString(function, column, s); + deserializeFromString(function, column, s, version); } } diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.h b/src/DataTypes/Serializations/SerializationAggregateFunction.h index 58a7d52ffe7..66497c34a67 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.h +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.h @@ -12,11 +12,12 @@ class SerializationAggregateFunction final : public ISerialization { private: AggregateFunctionPtr function; + size_t version; public: static constexpr bool is_parametric = true; - SerializationAggregateFunction(const AggregateFunctionPtr & function_): function(function_) {} + SerializationAggregateFunction(const AggregateFunctionPtr & function_, size_t version_): function(function_), version(version_) {} /// NOTE These two functions for serializing single values are incompatible with the functions below. void serializeBinary(const Field & field, WriteBuffer & ostr) const override; diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 962df507f82..413abaf075a 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -1652,7 +1652,7 @@ namespace { aggregate_function->create(data); ReadBufferFromMemory buf(str.data(), str.length()); - aggregate_function->deserialize(data, buf, &arena); + aggregate_function->deserialize(data, buf, std::nullopt, &arena); return data; } catch (...) diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 03e7d958edd..faf1c49e384 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1366,8 +1366,8 @@ struct WindowFunction size_t alignOfData() const override { return 1; } void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { fail(); } void merge(AggregateDataPtr __restrict, ConstAggregateDataPtr, Arena *) const override { fail(); } - void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &) const override { fail(); } - void deserialize(AggregateDataPtr __restrict, ReadBuffer &, Arena *) const override { fail(); } + void serialize(ConstAggregateDataPtr __restrict, WriteBuffer &, std::optional) const override { fail(); } + void deserialize(AggregateDataPtr __restrict, ReadBuffer &, std::optional, Arena *) const override { fail(); } void insertResultInto(AggregateDataPtr __restrict, IColumn &, Arena *) const override { fail(); } }; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 8a35ce3dcb8..94fddb08ccf 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -19,6 +19,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -1026,10 +1027,24 @@ void IMergeTreeDataPart::loadColumns(bool require) loaded_columns.writeText(*buf); } volume->getDisk()->moveFile(path + ".tmp", path); + LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded from metadata"); } else { + //LOG_TRACE(&Poco::Logger::get("kssenii"), "Loading columns stacktrace: {}", col.name, col.type->getName()); loaded_columns.readText(*volume->getDisk()->readFile(path)); + LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded from disk"); + for (auto & col : loaded_columns) + { + LOG_TRACE(&Poco::Logger::get("kssenii"), "Setting version for columns: {}, {}", col.name, col.type->getName()); + if (auto agg = typeid_cast(col.type.get())) + agg->setVersion(0); + } + } + + for (auto & col : loaded_columns) + { + LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded columns: {}, {}", col.name, col.type->getName()); } setColumns(loaded_columns); diff --git a/tests/integration/test_version_update/configs/log_conf.xml b/tests/integration/test_version_update/configs/log_conf.xml new file mode 100644 index 00000000000..f9d15e572aa --- /dev/null +++ b/tests/integration/test_version_update/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index ecf24a7b194..7dd88a8404a 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -1,10 +1,33 @@ import pytest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.2', with_installed_binary=True, stay_alive=True) + +node1 = cluster.add_instance('node1', main_configs=["configs/log_conf.xml"]) +node2 = cluster.add_instance('node2', main_configs=["configs/log_conf.xml"], + image='yandex/clickhouse-server', + tag='21.5', with_installed_binary=True, stay_alive=True) + +node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.2', with_installed_binary=True, stay_alive=True) + + +def create_and_fill_table(node): + node.query("DROP TABLE IF EXISTS test_table;") + node.query(""" + CREATE TABLE test_table + ( + `col1` DateTime, + `col2` Int64, + `col3` AggregateFunction(sumMap, Tuple(Array(UInt8), Array(UInt8))) + ) + ENGINE = AggregatingMergeTree() ORDER BY (col1, col2) """) + + node.query(""" INSERT INTO test_table + SELECT toDateTime('2020-10-01 19:20:30'), 1, + sumMapState((arrayMap(i -> 1, range(300)), arrayMap(i -> 1, range(300))));""") @pytest.fixture(scope="module") @@ -12,17 +35,71 @@ def start_cluster(): try: cluster.start() yield cluster - finally: cluster.shutdown() -def test_modulo_partition_key_after_update(start_cluster): - node1.query("CREATE TABLE test (id Int64, v UInt64, value String) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/table1', '1', v) PARTITION BY id % 20 ORDER BY (id, v)") - node1.query("INSERT INTO test SELECT number, number, toString(number) FROM numbers(10)") - expected = node1.query("SELECT number, number, toString(number) FROM numbers(10)") - partition_data = node1.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition") - assert(expected == node1.query("SELECT * FROM test ORDER BY id")) - node1.restart_with_latest_version(signal=9) - assert(expected == node1.query("SELECT * FROM test ORDER BY id")) - assert(partition_data == node1.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition")) +def test_modulo_partition_key_issue_23508(start_cluster): + node3.query("CREATE TABLE test (id Int64, v UInt64, value String) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/table1', '1', v) PARTITION BY id % 20 ORDER BY (id, v)") + node3.query("INSERT INTO test SELECT number, number, toString(number) FROM numbers(10)") + + expected = node3.query("SELECT number, number, toString(number) FROM numbers(10)") + partition_data = node3.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition") + assert(expected == node3.query("SELECT * FROM test ORDER BY id")) + + node3.restart_with_latest_version() + + assert(expected == node3.query("SELECT * FROM test ORDER BY id")) + assert(partition_data == node3.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition")) + + +def test_aggregate_function_versioning_issue_16587(start_cluster): + for node in [node1, node2]: + node.query("DROP TABLE IF EXISTS test_table;") + node.query(""" + CREATE TABLE test_table (`col1` DateTime, `col2` Int64) + ENGINE = MergeTree() ORDER BY col1""") + node.query("insert into test_table select '2020-10-26 00:00:00', 70724110 from numbers(300)") + + expected = "([1],[600])" + + # Incorrect result on old server + result_on_old_version = node2.query("select sumMap(sm) from (select sumMap([1],[1]) as sm from remote('127.0.0.{1,2}', default.test_table) group by col1, col2);") + assert(result_on_old_version.strip() != expected) + + # Correct result on new server + result_on_new_version = node1.query("select sumMap(sm) from (select sumMap([1],[1]) as sm from remote('127.0.0.{1,2}', default.test_table) group by col1, col2);") + assert(result_on_new_version.strip() == expected) + + +def test_aggregate_function_versioning_fetch_data_from_new_to_old_server(start_cluster): + for node in [node1, node2]: + create_and_fill_table(node) + + expected = "([1],[300])" + + new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(new_server_data == expected) + + old_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(old_server_data != expected) + + data_from_old_to_new_server = node1.query("select finalizeAggregation(col3) from remote('node2', default.test_table);").strip() + assert(data_from_old_to_new_server == old_server_data) + + +def test_aggregate_function_versioning_server_upgrade(start_cluster): + for node in [node1, node2]: + create_and_fill_table(node) + + expected = "([1],[300])" + + new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() + old_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(old_server_data != new_server_data) + + node2.restart_with_latest_version() + + upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(upgraded_server_data == old_server_data) + diff --git a/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference b/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference new file mode 100644 index 00000000000..c30c4ca7e74 --- /dev/null +++ b/tests/queries/0_stateless/01881_aggregate_functions_versioning.reference @@ -0,0 +1 @@ +CREATE TABLE default.test_table\n(\n `col1` DateTime,\n `col2` Int64,\n `col3` AggregateFunction(1, sumMap, Tuple(Array(UInt8), Array(UInt8)))\n)\nENGINE = AggregatingMergeTree\nORDER BY (col1, col2)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql b/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql new file mode 100644 index 00000000000..2f6a60409d5 --- /dev/null +++ b/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + `col1` DateTime, + `col2` Int64, + `col3` AggregateFunction(sumMap, Tuple(Array(UInt8), Array(UInt8))) +) +ENGINE = AggregatingMergeTree() ORDER BY (col1, col2); + +SHOW CREATE TABLE test_table; From c11ad44aad9db603c5645d57c685d8126a441c82 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 30 May 2021 22:54:42 +0000 Subject: [PATCH 089/609] More correct version --- src/DataTypes/DataTypeAggregateFunction.cpp | 43 ++++++++++--- src/DataTypes/DataTypeAggregateFunction.h | 26 ++++++-- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 19 +++--- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- tests/integration/test_version_update/test.py | 61 ++++++++++++++++--- 5 files changed, 120 insertions(+), 31 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 3f8beafcfd3..9f885e53e2f 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -38,11 +38,33 @@ namespace ErrorCodes String DataTypeAggregateFunction::doGetName() const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "doGetName stack trace:{}", StackTrace().toString()); + return getNameImpl(true); +} + + +String DataTypeAggregateFunction::getNameWithoutVersion() const +{ + return getNameImpl(false); +} + + +size_t DataTypeAggregateFunction::getVersion() const +{ + if (version) + return *version; + return function->getDefaultVersion(); +} + + +String DataTypeAggregateFunction::getNameImpl(bool with_version) const +{ WriteBufferFromOwnString stream; stream << "AggregateFunction("; - if (version) - stream << version << ", "; + + /// If aggregate function does not support versioning its version is 0 and is not printed. + auto data_type_version = getVersion(); + if (with_version && data_type_version) + stream << data_type_version << ", "; stream << function->getName(); if (!parameters.empty()) @@ -64,12 +86,16 @@ String DataTypeAggregateFunction::doGetName() const return stream.str(); } + MutableColumnPtr DataTypeAggregateFunction::createColumn() const { - /// FIXME: ColumnAggregateFunction also uses function->serialize methods + /// FIXME: There are a lot of function->serialize inside ColumnAggregateFunction. + /// Looks like it also needs version. + LOG_TRACE(&Poco::Logger::get("kssenii"), "KSSENII COLUMN"); return ColumnAggregateFunction::create(function); } + /// Create empty state Field DataTypeAggregateFunction::getDefault() const { @@ -100,20 +126,18 @@ Field DataTypeAggregateFunction::getDefault() const bool DataTypeAggregateFunction::equals(const IDataType & rhs) const { - return typeid(rhs) == typeid(*this) && getName() == rhs.getName(); + return typeid(rhs) == typeid(*this) && getNameWithoutVersion() == typeid_cast(rhs).getNameWithoutVersion(); } + SerializationPtr DataTypeAggregateFunction::doGetDefaultSerialization() const { - LOG_TRACE(&Poco::Logger::get("kssenii"), "get serializaton version: {}, name: {}, stack: {}", version, getName(), StackTrace().toString()); - return std::make_shared(function, version); + return std::make_shared(function, getVersion()); } static DataTypePtr create(const ASTPtr & arguments) { - LOG_TRACE(&Poco::Logger::get("kssenii"), "create data type: {}", StackTrace().toString()); - String function_name; AggregateFunctionPtr function; DataTypes argument_types; @@ -189,6 +213,7 @@ static DataTypePtr create(const ASTPtr & arguments) return std::make_shared(function, argument_types, params_row, version); } + void registerDataTypeAggregateFunction(DataTypeFactory & factory) { factory.registerDataType("AggregateFunction", create); diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index ac486087205..fa194fd7e9f 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -11,6 +11,11 @@ namespace DB /** Type - the state of the aggregate function. * Type parameters is an aggregate function, the types of its arguments, and its parameters (for parametric aggregate functions). + * + * Data type can support versioning for serialization of aggregate function state. + * Version 0 also means no versioning. When a table with versioned data type is attached, its version is parsed from AST. If + * there is no version in AST, then it is either attach with no version in metadata (then version is 0) or it + * is a new data type (then version is default). In distributed queries version of data type is known from data type name. */ class DataTypeAggregateFunction final : public IDataType { @@ -18,7 +23,9 @@ private: AggregateFunctionPtr function; DataTypes argument_types; Array parameters; - mutable size_t version; + mutable std::optional version; + + String getNameImpl(bool with_version) const; public: static constexpr bool is_parametric = true; @@ -28,7 +35,7 @@ public: : function(function_) , argument_types(argument_types_) , parameters(parameters_) - , version(version_ ? *version_ : function_->getDefaultVersion()) + , version(version_) { } @@ -36,6 +43,7 @@ public: AggregateFunctionPtr getFunction() const { return function; } String doGetName() const override; + String getNameWithoutVersion() const; const char * getFamilyName() const override { return "AggregateFunction"; } TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; } @@ -57,8 +65,18 @@ public: SerializationPtr doGetDefaultSerialization() const override; - bool isVersioned() const { return function->isVersioned(); } - void setVersion(size_t version_) const { version = version_; } + /// Version of aggregate function state serialization. + size_t getVersion() const; + + /// Version is not empty only if it was parsed from AST. + /// It is ok to have an empty version value here - then for serialization + /// a default (latest) version is used. This method is used to force some + /// zero version to be used instead of default - if there was no version in AST. + void setVersionIfEmpty(size_t version_) const + { + if (!version) + version = version_; + } }; } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 94fddb08ccf..0fabee27c32 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -389,7 +389,7 @@ std::pair IMergeTreeDataPart::getMinMaxTime() const } -void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) +void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, bool loaded_from_disk) { columns = new_columns; column_name_to_position.clear(); @@ -398,8 +398,15 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) for (const auto & column : columns) { column_name_to_position.emplace(column.name, pos); + + /// TODO: May be there is a better way or a better place for that. + const auto * aggregate_function_data_type = typeid_cast(column.type.get()); + if (loaded_from_disk && aggregate_function_data_type) + aggregate_function_data_type->setVersionIfEmpty(0); + for (const auto & subcolumn : column.type->getSubcolumnNames()) column_name_to_position.emplace(Nested::concatenateName(column.name, subcolumn), pos); + ++pos; } } @@ -1007,6 +1014,7 @@ void IMergeTreeDataPart::loadColumns(bool require) metadata_snapshot = metadata_snapshot->projections.get(name).metadata; NamesAndTypesList loaded_columns; + bool loaded_from_disk = false; if (!volume->getDisk()->exists(path)) { /// We can get list of columns only from columns.txt in compact parts. @@ -1027,18 +1035,15 @@ void IMergeTreeDataPart::loadColumns(bool require) loaded_columns.writeText(*buf); } volume->getDisk()->moveFile(path + ".tmp", path); - LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded from metadata"); } else { - //LOG_TRACE(&Poco::Logger::get("kssenii"), "Loading columns stacktrace: {}", col.name, col.type->getName()); loaded_columns.readText(*volume->getDisk()->readFile(path)); - LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded from disk"); + loaded_from_disk = true; + for (auto & col : loaded_columns) { LOG_TRACE(&Poco::Logger::get("kssenii"), "Setting version for columns: {}, {}", col.name, col.type->getName()); - if (auto agg = typeid_cast(col.type.get())) - agg->setVersion(0); } } @@ -1047,7 +1052,7 @@ void IMergeTreeDataPart::loadColumns(bool require) LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded columns: {}, {}", col.name, col.type->getName()); } - setColumns(loaded_columns); + setColumns(loaded_columns, loaded_from_disk); } bool IMergeTreeDataPart::shallParticipateInMerges(const StoragePolicyPtr & storage_policy) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index a8a49680dd7..cc2a742f194 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -121,7 +121,7 @@ public: String getTypeName() const { return getType().toString(); } - void setColumns(const NamesAndTypesList & new_columns); + void setColumns(const NamesAndTypesList & new_columns, bool loaded_from_disk = false); const NamesAndTypesList & getColumns() const { return columns; } diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index 7dd88a8404a..7485ce226ea 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -6,7 +6,7 @@ from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', main_configs=["configs/log_conf.xml"]) +node1 = cluster.add_instance('node1', main_configs=["configs/log_conf.xml"], stay_alive=True) node2 = cluster.add_instance('node2', main_configs=["configs/log_conf.xml"], image='yandex/clickhouse-server', tag='21.5', with_installed_binary=True, stay_alive=True) @@ -14,6 +14,12 @@ node2 = cluster.add_instance('node2', main_configs=["configs/log_conf.xml"], node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.2', with_installed_binary=True, stay_alive=True) +def insert_data(node): + node.query(""" INSERT INTO test_table + SELECT toDateTime('2020-10-01 19:20:30'), 1, + sumMapState(arrayMap(i -> 1, range(300)), arrayMap(i -> 1, range(300)));""") + + def create_and_fill_table(node): node.query("DROP TABLE IF EXISTS test_table;") node.query(""" @@ -21,13 +27,10 @@ def create_and_fill_table(node): ( `col1` DateTime, `col2` Int64, - `col3` AggregateFunction(sumMap, Tuple(Array(UInt8), Array(UInt8))) + `col3` AggregateFunction(sumMap, Array(UInt8), Array(UInt8)) ) ENGINE = AggregatingMergeTree() ORDER BY (col1, col2) """) - - node.query(""" INSERT INTO test_table - SELECT toDateTime('2020-10-01 19:20:30'), 1, - sumMapState((arrayMap(i -> 1, range(300)), arrayMap(i -> 1, range(300))));""") + insert_data(node) @pytest.fixture(scope="module") @@ -92,14 +95,52 @@ def test_aggregate_function_versioning_server_upgrade(start_cluster): for node in [node1, node2]: create_and_fill_table(node) - expected = "([1],[300])" - new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(new_server_data == "([1],[300])") old_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() - assert(old_server_data != new_server_data) + assert(old_server_data == "([1],[44])") node2.restart_with_latest_version() + # Check that after server upgrade aggregate function is serialized according to older version. upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() - assert(upgraded_server_data == old_server_data) + assert(upgraded_server_data == "([1],[44])") + + # Remote fetches are still with older version. + data_from_upgraded_to_new_server = node1.query("select finalizeAggregation(col3) from remote('node2', default.test_table);").strip() + assert(data_from_upgraded_to_new_server == upgraded_server_data == "([1],[44])") + + # Check it is ok to write into table with older version of aggregate function. + insert_data(node2) + + # Hm, should newly inserted data be serialized as old version? + upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(upgraded_server_data == "([1],[300])\n([1],[44])") + + +def test_aggregate_function_versioning_persisting_metadata(start_cluster): + for node in [node1, node2]: + create_and_fill_table(node) + node2.restart_with_latest_version() + + for node in [node1, node2]: + node.query("DETACH TABLE test_table") + node.query("ATTACH TABLE test_table") + + for node in [node1, node2]: + insert_data(node) + + new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(new_server_data == "([1],[300])\n([1],[300])") + upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(upgraded_server_data == "([1],[44])\n([1],[44])") + + for node in [node1, node2]: + node.restart_clickhouse() + insert_data(node) + + result = node1.query("select finalizeAggregation(col3) from remote('127.0.0.{1,2}', default.test_table);").strip() + assert(result == "([1],[300])\n([1],[300])\n([1],[300])\n([1],[300])") + result = node2.query("select finalizeAggregation(col3) from remote('127.0.0.{1,2}', default.test_table);").strip() + assert(result == "([1],[44])\n([1],[44])\n([1],[44])\n([1],[44])") From e510c3839eee54dec13c431def7f4e50e2c6d6a3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 31 May 2021 14:44:57 +0000 Subject: [PATCH 090/609] More correct --- .../AggregateFunctionArgMinMax.h | 2 +- .../AggregateFunctionArray.h | 2 +- src/AggregateFunctions/AggregateFunctionAvg.h | 2 +- .../AggregateFunctionBitwise.h | 2 +- .../AggregateFunctionBoundingRatio.h | 2 +- .../AggregateFunctionCount.h | 2 +- .../AggregateFunctionDeltaSum.h | 2 +- .../AggregateFunctionDeltaSumTimestamp.h | 2 +- .../AggregateFunctionDistinct.h | 2 +- .../AggregateFunctionEntropy.h | 2 +- .../AggregateFunctionForEach.h | 2 +- .../AggregateFunctionGroupArray.h | 2 +- .../AggregateFunctionGroupArrayInsertAt.h | 2 +- .../AggregateFunctionGroupArrayMoving.h | 2 +- .../AggregateFunctionGroupUniqArray.h | 2 +- .../AggregateFunctionHistogram.h | 2 +- src/AggregateFunctions/AggregateFunctionIf.h | 2 +- .../AggregateFunctionMannWhitney.h | 2 +- .../AggregateFunctionMaxIntersections.h | 2 +- .../AggregateFunctionMerge.h | 2 +- .../AggregateFunctionRankCorrelation.h | 2 +- .../AggregateFunctionRetention.h | 2 +- .../AggregateFunctionSequenceMatch.h | 2 +- .../AggregateFunctionSumMap.h | 79 +++++--- src/Columns/ColumnAggregateFunction.cpp | 22 +- src/Columns/ColumnAggregateFunction.h | 4 +- src/DataStreams/NativeBlockInputStream.cpp | 13 +- src/DataStreams/NativeBlockInputStream.h | 2 +- src/DataStreams/NativeBlockOutputStream.cpp | 12 +- src/DataTypes/DataTypeAggregateFunction.cpp | 9 +- src/DataTypes/DataTypeAggregateFunction.h | 18 +- src/Interpreters/InterpreterCreateQuery.cpp | 5 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 13 +- tests/integration/test_version_update/test.py | 189 +++++++++++------- 34 files changed, 245 insertions(+), 167 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionArgMinMax.h b/src/AggregateFunctions/AggregateFunctionArgMinMax.h index 66a9a19ce1c..b25c91241e0 100644 --- a/src/AggregateFunctions/AggregateFunctionArgMinMax.h +++ b/src/AggregateFunctions/AggregateFunctionArgMinMax.h @@ -85,7 +85,7 @@ public: this->data(place).value.write(buf, *serialization_val); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).result.read(buf, *serialization_res, arena); this->data(place).value.read(buf, *serialization_val, arena); diff --git a/src/AggregateFunctions/AggregateFunctionArray.h b/src/AggregateFunctions/AggregateFunctionArray.h index c62a1fd7560..4a3da6137b7 100644 --- a/src/AggregateFunctions/AggregateFunctionArray.h +++ b/src/AggregateFunctions/AggregateFunctionArray.h @@ -125,7 +125,7 @@ public: nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { nested_func->deserialize(place, buf, version, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index c593f03d4dd..297054139af 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -117,7 +117,7 @@ public: writeBinary(this->data(place).denominator, buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readBinary(this->data(place).numerator, buf); diff --git a/src/AggregateFunctions/AggregateFunctionBitwise.h b/src/AggregateFunctions/AggregateFunctionBitwise.h index 36820fd8910..9df03d7eff4 100644 --- a/src/AggregateFunctions/AggregateFunctionBitwise.h +++ b/src/AggregateFunctions/AggregateFunctionBitwise.h @@ -72,7 +72,7 @@ public: writeBinary(this->data(place).value, buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readBinary(this->data(place).value, buf); } diff --git a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h index 9176b217625..2b7540a9a55 100644 --- a/src/AggregateFunctions/AggregateFunctionBoundingRatio.h +++ b/src/AggregateFunctions/AggregateFunctionBoundingRatio.h @@ -147,7 +147,7 @@ public: data(place).serialize(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionCount.h b/src/AggregateFunctions/AggregateFunctionCount.h index e64e9512914..effea847d04 100644 --- a/src/AggregateFunctions/AggregateFunctionCount.h +++ b/src/AggregateFunctions/AggregateFunctionCount.h @@ -146,7 +146,7 @@ public: writeVarUInt(data(place).count, buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readVarUInt(data(place).count, buf); } diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSum.h b/src/AggregateFunctions/AggregateFunctionDeltaSum.h index cce83423008..dd72bf58e31 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSum.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSum.h @@ -110,7 +110,7 @@ public: writePODBinary(this->data(place).seen, buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readIntBinary(this->data(place).sum, buf); readIntBinary(this->data(place).first, buf); diff --git a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h index f265c70f2db..05bead987a4 100644 --- a/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h +++ b/src/AggregateFunctions/AggregateFunctionDeltaSumTimestamp.h @@ -153,7 +153,7 @@ public: writePODBinary(this->data(place).seen, buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { readIntBinary(this->data(place).sum, buf); readIntBinary(this->data(place).first, buf); diff --git a/src/AggregateFunctions/AggregateFunctionDistinct.h b/src/AggregateFunctions/AggregateFunctionDistinct.h index 3b031445f2c..e6df80ab2a3 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/AggregateFunctionDistinct.h @@ -187,7 +187,7 @@ public: this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).deserialize(buf, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionEntropy.h b/src/AggregateFunctions/AggregateFunctionEntropy.h index ad2e43f665a..0be6d46225f 100644 --- a/src/AggregateFunctions/AggregateFunctionEntropy.h +++ b/src/AggregateFunctions/AggregateFunctionEntropy.h @@ -130,7 +130,7 @@ public: this->data(const_cast(place)).serialize(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index db71bc94f08..8fe03d544ee 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -219,7 +219,7 @@ public: } } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { AggregateFunctionForEachData & state = data(place); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 6f6883ae4b4..25188afdf21 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -570,7 +570,7 @@ public: // if constexpr (Trait::sampler == Sampler::DETERMINATOR) } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { UInt64 elems; readVarUInt(elems, buf); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h index 302f3412b7c..57fcd40fb82 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayInsertAt.h @@ -164,7 +164,7 @@ public: } } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { size_t size = 0; readVarUInt(size, buf); diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index 3c2a83ec03b..f4aea9d6267 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -147,7 +147,7 @@ public: buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { size_t size = 0; readVarUInt(size, buf); diff --git a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h index 21dd623d1ce..35d8c7f78af 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupUniqArray.h @@ -96,7 +96,7 @@ public: writeIntBinary(elem, buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).value.read(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index fcbbe3d7f98..665e505aa4e 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -351,7 +351,7 @@ public: this->data(place).write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).read(buf, max_bins); } diff --git a/src/AggregateFunctions/AggregateFunctionIf.h b/src/AggregateFunctions/AggregateFunctionIf.h index bba9ba2aed8..d1a28e47005 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.h +++ b/src/AggregateFunctions/AggregateFunctionIf.h @@ -139,7 +139,7 @@ public: nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { nested_func->deserialize(place, buf, version, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionMannWhitney.h b/src/AggregateFunctions/AggregateFunctionMannWhitney.h index a6fb715f492..fe93ad15384 100644 --- a/src/AggregateFunctions/AggregateFunctionMannWhitney.h +++ b/src/AggregateFunctions/AggregateFunctionMannWhitney.h @@ -220,7 +220,7 @@ public: this->data(place).write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).read(buf, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 6f125610936..51fa79822c3 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -118,7 +118,7 @@ public: buf.write(reinterpret_cast(value.data()), size * sizeof(value[0])); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { size_t size = 0; readVarUInt(size, buf); diff --git a/src/AggregateFunctions/AggregateFunctionMerge.h b/src/AggregateFunctions/AggregateFunctionMerge.h index 5c4800b16b1..c79c3d1f38d 100644 --- a/src/AggregateFunctions/AggregateFunctionMerge.h +++ b/src/AggregateFunctions/AggregateFunctionMerge.h @@ -100,7 +100,7 @@ public: nested_func->serialize(place, buf, version); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional version, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { nested_func->deserialize(place, buf, version, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h index 9d181210f5a..dec27222939 100644 --- a/src/AggregateFunctions/AggregateFunctionRankCorrelation.h +++ b/src/AggregateFunctions/AggregateFunctionRankCorrelation.h @@ -87,7 +87,7 @@ public: this->data(place).write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena * arena) const override { this->data(place).read(buf, arena); } diff --git a/src/AggregateFunctions/AggregateFunctionRetention.h b/src/AggregateFunctions/AggregateFunctionRetention.h index ed7dccc10f9..ff52c8c3276 100644 --- a/src/AggregateFunctions/AggregateFunctionRetention.h +++ b/src/AggregateFunctions/AggregateFunctionRetention.h @@ -120,7 +120,7 @@ public: this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 287182b3c43..0148ac23401 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -174,7 +174,7 @@ public: this->data(place).serialize(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { this->data(place).deserialize(buf); } diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index e1bf8f355f4..549cd5b3137 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -269,30 +269,37 @@ public: if (!version) version = getDefaultVersion(); - LOG_TRACE(&Poco::Logger::get("kssenii"), "version to serialize: {}, stack: {}", *version, StackTrace().toString()); const auto & merged_maps = this->data(place).merged_maps; size_t size = merged_maps.size(); writeVarUInt(size, buf); + std::function serialize; + switch (*version) + { + case 0: + { + serialize = [&](size_t col_idx, const Array & values){ values_serializations[col_idx]->serializeBinary(values[col_idx], buf); }; + break; + } + case 1: + { + serialize = [&](size_t col_idx, const Array & values) + { + const auto & type = values_types[col_idx]; + if (isInteger(type)) + SerializationNumber().serializeBinary(values[col_idx], buf); + else + values_serializations[col_idx]->serializeBinary(values[col_idx], buf); + }; + break; + } + } + for (const auto & elem : merged_maps) { keys_serialization->serializeBinary(elem.first, buf); for (size_t col = 0; col < values_types.size(); ++col) - { - switch (*version) - { - case 0: - { - values_serializations[col]->serializeBinary(elem.second[col], buf); - break; - } - case 1: - { - SerializationNumber().serializeBinary(elem.second[col], buf); - break; - } - } - } + serialize(col, elem.second); } } @@ -301,11 +308,32 @@ public: if (!version) version = getDefaultVersion(); - LOG_TRACE(&Poco::Logger::get("kssenii"), "version to deserialize: {}, stack: {}", *version, StackTrace().toString()); auto & merged_maps = this->data(place).merged_maps; size_t size = 0; readVarUInt(size, buf); + std::function deserialize; + switch (*version) + { + case 0: + { + deserialize = [&](size_t col_idx, Array & values){ values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); }; + break; + } + case 1: + { + deserialize = [&](size_t col_idx, Array & values) + { + const auto & type = values_types[col_idx]; + if (isInteger(type)) + SerializationNumber().deserializeBinary(values[col_idx], buf); + else + values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); + }; + break; + } + } + for (size_t i = 0; i < size; ++i) { Field key; @@ -313,22 +341,9 @@ public: Array values; values.resize(values_types.size()); + for (size_t col = 0; col < values_types.size(); ++col) - { - switch (*version) - { - case 0: - { - values_serializations[col]->deserializeBinary(values[col], buf); - break; - } - case 1: - { - SerializationNumber().deserializeBinary(values[col], buf); - break; - } - } - } + deserialize(col, values); if constexpr (IsDecimalNumber) merged_maps[key.get>()] = values; diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 49fde42b9b0..8fb3d524df0 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -28,7 +28,7 @@ namespace ErrorCodes } -static std::string getTypeString(const AggregateFunctionPtr & func) +static String getTypeString(const AggregateFunctionPtr & func) { WriteBufferFromOwnString stream; stream << "AggregateFunction(" << func->getName(); @@ -55,8 +55,8 @@ static std::string getTypeString(const AggregateFunctionPtr & func) } -ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & func_) - : func(func_), type_string(getTypeString(func)) +ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional version_) + : func(func_), type_string(getTypeString(func)), version(version_) { } @@ -354,7 +354,7 @@ INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction) void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const { WriteBufferFromOwnString wbuf; - func->serialize(data[n], wbuf); + func->serialize(data[n], wbuf, version); hash.update(wbuf.str().c_str(), wbuf.str().size()); } @@ -371,7 +371,7 @@ void ColumnAggregateFunction::updateWeakHash32(WeakHash32 & hash) const for (size_t i = 0; i < s; ++i) { WriteBufferFromVector> wbuf(v); - func->serialize(data[i], wbuf); + func->serialize(data[i], wbuf, version); wbuf.finalize(); hash_data[i] = ::updateWeakHash32(v.data(), v.size(), hash_data[i]); } @@ -423,7 +423,7 @@ Field ColumnAggregateFunction::operator[](size_t n) const field.get().name = type_string; { WriteBufferFromString buffer(field.get().data); - func->serialize(data[n], buffer); + func->serialize(data[n], buffer, version); } return field; } @@ -434,7 +434,7 @@ void ColumnAggregateFunction::get(size_t n, Field & res) const res.get().name = type_string; { WriteBufferFromString buffer(res.get().data); - func->serialize(data[n], buffer); + func->serialize(data[n], buffer, version); } } @@ -514,7 +514,7 @@ void ColumnAggregateFunction::insert(const Field & x) Arena & arena = createOrGetArena(); pushBackAndCreateState(data, arena, func.get()); ReadBufferFromString read_buffer(x.get().data); - func->deserialize(data.back(), read_buffer, std::nullopt, &arena); + func->deserialize(data.back(), read_buffer, version, &arena); } void ColumnAggregateFunction::insertDefault() @@ -527,7 +527,7 @@ void ColumnAggregateFunction::insertDefault() StringRef ColumnAggregateFunction::serializeValueIntoArena(size_t n, Arena & arena, const char *& begin) const { WriteBufferFromArena out(arena, begin); - func->serialize(data[n], out); + func->serialize(data[n], out, version); return out.finish(); } @@ -549,7 +549,7 @@ const char * ColumnAggregateFunction::deserializeAndInsertFromArena(const char * * Probably this will not work under UBSan. */ ReadBufferFromMemory read_buffer(src_arena, std::numeric_limits::max() - src_arena - 1); - func->deserialize(data.back(), read_buffer, std::nullopt, &dst_arena); + func->deserialize(data.back(), read_buffer, version, &dst_arena); return read_buffer.position(); } @@ -649,7 +649,7 @@ void ColumnAggregateFunction::getExtremes(Field & min, Field & max) const try { WriteBufferFromString buffer(serialized.data); - func->serialize(place, buffer); + func->serialize(place, buffer, version); } catch (...) { diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 059d7c330a1..a654316732d 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -82,6 +82,8 @@ private: /// Name of the type to distinguish different aggregation states. String type_string; + std::optional version; + ColumnAggregateFunction() = default; /// Create a new column that has another column as a source. @@ -92,7 +94,7 @@ private: /// but ownership of different elements cannot be mixed by different columns. void ensureOwnership(); - ColumnAggregateFunction(const AggregateFunctionPtr & func_); + ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional version_ = std::nullopt); ColumnAggregateFunction(const AggregateFunctionPtr & func_, const ConstArenas & arenas_); diff --git a/src/DataStreams/NativeBlockInputStream.cpp b/src/DataStreams/NativeBlockInputStream.cpp index 2f376f5230d..27b20725704 100644 --- a/src/DataStreams/NativeBlockInputStream.cpp +++ b/src/DataStreams/NativeBlockInputStream.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB @@ -71,7 +72,7 @@ void NativeBlockInputStream::resetParser() is_killed.store(false); } -void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) +void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint, size_t revision) { ISerialization::DeserializeBinaryBulkSettings settings; settings.getter = [&](ISerialization::SubstreamPath) -> ReadBuffer * { return &istr; }; @@ -79,6 +80,14 @@ void NativeBlockInputStream::readData(const IDataType & type, ColumnPtr & column settings.position_independent_encoding = false; ISerialization::DeserializeBinaryBulkStatePtr state; + + const auto * aggregate_function_data_type = typeid_cast(&type); + if (aggregate_function_data_type && aggregate_function_data_type->isVersioned()) + { + auto version = aggregate_function_data_type->getVersionFromRevision(revision); + aggregate_function_data_type->setVersionIfEmpty(version); + } + auto serialization = type.getDefaultSerialization(); serialization->deserializeBinaryBulkStatePrefix(settings, state); @@ -164,7 +173,7 @@ Block NativeBlockInputStream::readImpl() double avg_value_size_hint = avg_value_size_hints.empty() ? 0 : avg_value_size_hints[i]; if (rows) /// If no rows, nothing to read. - readData(*column.type, read_column, istr, rows, avg_value_size_hint); + readData(*column.type, read_column, istr, rows, avg_value_size_hint, server_revision); column.column = std::move(read_column); diff --git a/src/DataStreams/NativeBlockInputStream.h b/src/DataStreams/NativeBlockInputStream.h index 8f3d2843e0f..1e3577bcc40 100644 --- a/src/DataStreams/NativeBlockInputStream.h +++ b/src/DataStreams/NativeBlockInputStream.h @@ -74,7 +74,7 @@ public: String getName() const override { return "Native"; } - static void readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint); + static void readData(const IDataType & type, ColumnPtr & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint, size_t revision); Block getHeader() const override; diff --git a/src/DataStreams/NativeBlockOutputStream.cpp b/src/DataStreams/NativeBlockOutputStream.cpp index a5d8c15986b..ca030aca2d9 100644 --- a/src/DataStreams/NativeBlockOutputStream.cpp +++ b/src/DataStreams/NativeBlockOutputStream.cpp @@ -10,6 +10,7 @@ #include #include +#include namespace DB { @@ -41,7 +42,7 @@ void NativeBlockOutputStream::flush() } -static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit) +static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, UInt64 offset, UInt64 limit, size_t revision) { /** If there are columns-constants - then we materialize them. * (Since the data type does not know how to serialize / deserialize constants.) @@ -53,6 +54,13 @@ static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuf settings.position_independent_encoding = false; settings.low_cardinality_max_dictionary_size = 0; //-V1048 + const auto * aggregate_function_data_type = typeid_cast(&type); + if (aggregate_function_data_type && aggregate_function_data_type->isVersioned()) + { + auto version = aggregate_function_data_type->getVersionFromRevision(revision); + aggregate_function_data_type->setVersionIfEmpty(version); + } + auto serialization = type.getDefaultSerialization(); ISerialization::SerializeBinaryBulkStatePtr state; @@ -123,7 +131,7 @@ void NativeBlockOutputStream::write(const Block & block) /// Data if (rows) /// Zero items of data is always represented as zero number of bytes. - writeData(*column.type, column.column, ostr, 0, 0); + writeData(*column.type, column.column, ostr, 0, 0, client_revision); if (index_ostr) { diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index 9f885e53e2f..ffdd5b6cb94 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -20,7 +20,6 @@ #include #include #include -#include namespace DB @@ -89,10 +88,7 @@ String DataTypeAggregateFunction::getNameImpl(bool with_version) const MutableColumnPtr DataTypeAggregateFunction::createColumn() const { - /// FIXME: There are a lot of function->serialize inside ColumnAggregateFunction. - /// Looks like it also needs version. - LOG_TRACE(&Poco::Logger::get("kssenii"), "KSSENII COLUMN"); - return ColumnAggregateFunction::create(function); + return ColumnAggregateFunction::create(function, version); } @@ -153,7 +149,7 @@ static DataTypePtr create(const ASTPtr & arguments) /* If aggregate function definition doesn't have version, it will have in AST children args [ASTFunction, types...] - in case * it is parametric, or [ASTIdentifier, types...] - otherwise. If aggregate function has version in AST, then it will be: - * [ASTLitearl, ASTFunction (or ASTIdentifier), types]. + * [ASTLitearl, ASTFunction (or ASTIdentifier), types...]. */ if (auto version_ast = arguments->children[0]->as()) { @@ -168,7 +164,6 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR); function_name = parametric->name; - LOG_TRACE(&Poco::Logger::get("kssenii"), "Paramtric function name: {}", function_name); if (parametric->arguments) { diff --git a/src/DataTypes/DataTypeAggregateFunction.h b/src/DataTypes/DataTypeAggregateFunction.h index fa194fd7e9f..7a6a69b1f61 100644 --- a/src/DataTypes/DataTypeAggregateFunction.h +++ b/src/DataTypes/DataTypeAggregateFunction.h @@ -3,7 +3,6 @@ #include #include -#include namespace DB @@ -15,7 +14,7 @@ namespace DB * Data type can support versioning for serialization of aggregate function state. * Version 0 also means no versioning. When a table with versioned data type is attached, its version is parsed from AST. If * there is no version in AST, then it is either attach with no version in metadata (then version is 0) or it - * is a new data type (then version is default). In distributed queries version of data type is known from data type name. + * is a new data type (then version is default - latest). */ class DataTypeAggregateFunction final : public IDataType { @@ -26,6 +25,7 @@ private: mutable std::optional version; String getNameImpl(bool with_version) const; + size_t getVersion() const; public: static constexpr bool is_parametric = true; @@ -65,13 +65,15 @@ public: SerializationPtr doGetDefaultSerialization() const override; - /// Version of aggregate function state serialization. - size_t getVersion() const; + bool isVersioned() const { return function->isVersioned(); } - /// Version is not empty only if it was parsed from AST. - /// It is ok to have an empty version value here - then for serialization - /// a default (latest) version is used. This method is used to force some - /// zero version to be used instead of default - if there was no version in AST. + size_t getVersionFromRevision(size_t revision) const { return function->getVersionFromRevision(revision); } + + /// Version is not empty only if it was parsed from AST or implicitly cast to 0 or version according + /// to server revision. + /// It is ok to have an empty version value here - then for serialization a default (latest) + /// version is used. This method is used to force some zero version to be used instead of + /// default, or to set version for serialization in distributed queries. void setVersionIfEmpty(size_t version_) const { if (!version) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 6cb823aae3c..a3758f60b23 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -45,6 +45,7 @@ #include #include #include +#include #include #include @@ -394,6 +395,10 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( { column_type = DataTypeFactory::instance().get(col_decl.type); + auto aggregate_function_type = typeid_cast(column_type.get()); + if (attach && aggregate_function_type && aggregate_function_type->isVersioned()) + aggregate_function_type->setVersionIfEmpty(0); + if (col_decl.null_modifier) { if (column_type->isNullable()) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 0fabee27c32..00774030f10 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -399,9 +399,8 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, bool { column_name_to_position.emplace(column.name, pos); - /// TODO: May be there is a better way or a better place for that. const auto * aggregate_function_data_type = typeid_cast(column.type.get()); - if (loaded_from_disk && aggregate_function_data_type) + if (loaded_from_disk && aggregate_function_data_type && aggregate_function_data_type->isVersioned()) aggregate_function_data_type->setVersionIfEmpty(0); for (const auto & subcolumn : column.type->getSubcolumnNames()) @@ -1040,16 +1039,6 @@ void IMergeTreeDataPart::loadColumns(bool require) { loaded_columns.readText(*volume->getDisk()->readFile(path)); loaded_from_disk = true; - - for (auto & col : loaded_columns) - { - LOG_TRACE(&Poco::Logger::get("kssenii"), "Setting version for columns: {}, {}", col.name, col.type->getName()); - } - } - - for (auto & col : loaded_columns) - { - LOG_TRACE(&Poco::Logger::get("kssenii"), "Loaded columns: {}, {}", col.name, col.type->getName()); } setColumns(loaded_columns, loaded_from_disk); diff --git a/tests/integration/test_version_update/test.py b/tests/integration/test_version_update/test.py index 7485ce226ea..030d0b89cb1 100644 --- a/tests/integration/test_version_update/test.py +++ b/tests/integration/test_version_update/test.py @@ -2,35 +2,53 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry, exec_query_with_retry - cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=["configs/log_conf.xml"], stay_alive=True) -node2 = cluster.add_instance('node2', main_configs=["configs/log_conf.xml"], - image='yandex/clickhouse-server', - tag='21.5', with_installed_binary=True, stay_alive=True) -node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.2', with_installed_binary=True, stay_alive=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.2', with_installed_binary=True, stay_alive=True) + +# Use differents nodes because if there is node.restart_from_latest_version(), then in later tests +# it will be with latest version, but shouldn't, order of tests in CI is shuffled. +node3 = cluster.add_instance('node3', main_configs=["configs/log_conf.xml"], + image='yandex/clickhouse-server', tag='21.5', with_installed_binary=True, stay_alive=True) +node4 = cluster.add_instance('node4', main_configs=["configs/log_conf.xml"], + image='yandex/clickhouse-server', tag='21.5', with_installed_binary=True, stay_alive=True) +node5 = cluster.add_instance('node5', main_configs=["configs/log_conf.xml"], + image='yandex/clickhouse-server', tag='21.5', with_installed_binary=True, stay_alive=True) +node6 = cluster.add_instance('node6', main_configs=["configs/log_conf.xml"], + image='yandex/clickhouse-server', tag='21.5', with_installed_binary=True, stay_alive=True) -def insert_data(node): - node.query(""" INSERT INTO test_table - SELECT toDateTime('2020-10-01 19:20:30'), 1, - sumMapState(arrayMap(i -> 1, range(300)), arrayMap(i -> 1, range(300)));""") + +def insert_data(node, table_name='test_table', n=1, col2=1): + node.query(""" INSERT INTO {} + SELECT toDateTime(NOW()), {}, + sumMapState(arrayMap(i -> 1, range(300)), arrayMap(i -> 1, range(300))) + FROM numbers({});""".format(table_name, col2, n)) -def create_and_fill_table(node): - node.query("DROP TABLE IF EXISTS test_table;") - node.query(""" - CREATE TABLE test_table - ( - `col1` DateTime, - `col2` Int64, - `col3` AggregateFunction(sumMap, Array(UInt8), Array(UInt8)) - ) - ENGINE = AggregatingMergeTree() ORDER BY (col1, col2) """) - insert_data(node) +def create_table(node, name='test_table', version=None): + node.query("DROP TABLE IF EXISTS {};".format(name)) + if version is None: + node.query(""" + CREATE TABLE {} + ( + `col1` DateTime, + `col2` Int64, + `col3` AggregateFunction(sumMap, Array(UInt8), Array(UInt8)) + ) + ENGINE = AggregatingMergeTree() ORDER BY (col1, col2) """.format(name)) + else: + node.query(""" + CREATE TABLE {} + ( + `col1` DateTime, + `col2` Int64, + `col3` AggregateFunction({}, sumMap, Array(UInt8), Array(UInt8)) + ) + ENGINE = AggregatingMergeTree() ORDER BY (col1, col2) """.format(name, version)) @pytest.fixture(scope="module") @@ -43,104 +61,139 @@ def start_cluster(): def test_modulo_partition_key_issue_23508(start_cluster): - node3.query("CREATE TABLE test (id Int64, v UInt64, value String) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/table1', '1', v) PARTITION BY id % 20 ORDER BY (id, v)") - node3.query("INSERT INTO test SELECT number, number, toString(number) FROM numbers(10)") + node2.query("CREATE TABLE test (id Int64, v UInt64, value String) ENGINE = ReplicatedReplacingMergeTree('/clickhouse/tables/table1', '1', v) PARTITION BY id % 20 ORDER BY (id, v)") + node2.query("INSERT INTO test SELECT number, number, toString(number) FROM numbers(10)") - expected = node3.query("SELECT number, number, toString(number) FROM numbers(10)") - partition_data = node3.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition") - assert(expected == node3.query("SELECT * FROM test ORDER BY id")) + expected = node2.query("SELECT number, number, toString(number) FROM numbers(10)") + partition_data = node2.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition") + assert(expected == node2.query("SELECT * FROM test ORDER BY id")) - node3.restart_with_latest_version() + node2.restart_with_latest_version() - assert(expected == node3.query("SELECT * FROM test ORDER BY id")) - assert(partition_data == node3.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition")) + assert(expected == node2.query("SELECT * FROM test ORDER BY id")) + assert(partition_data == node2.query("SELECT partition, name FROM system.parts WHERE table='test' ORDER BY partition")) +# Test from issue 16587 def test_aggregate_function_versioning_issue_16587(start_cluster): - for node in [node1, node2]: + for node in [node1, node3]: node.query("DROP TABLE IF EXISTS test_table;") node.query(""" CREATE TABLE test_table (`col1` DateTime, `col2` Int64) ENGINE = MergeTree() ORDER BY col1""") - node.query("insert into test_table select '2020-10-26 00:00:00', 70724110 from numbers(300)") + node.query("insert into test_table select '2020-10-26 00:00:00', 1929292 from numbers(300)") expected = "([1],[600])" - # Incorrect result on old server - result_on_old_version = node2.query("select sumMap(sm) from (select sumMap([1],[1]) as sm from remote('127.0.0.{1,2}', default.test_table) group by col1, col2);") - assert(result_on_old_version.strip() != expected) + result_on_old_version = node3.query("select sumMap(sm) from (select sumMap([1],[1]) as sm from remote('127.0.0.{1,2}', default.test_table) group by col1, col2);").strip() + assert(result_on_old_version != expected) - # Correct result on new server - result_on_new_version = node1.query("select sumMap(sm) from (select sumMap([1],[1]) as sm from remote('127.0.0.{1,2}', default.test_table) group by col1, col2);") - assert(result_on_new_version.strip() == expected) + result_on_new_version = node1.query("select sumMap(sm) from (select sumMap([1],[1]) as sm from remote('127.0.0.{1,2}', default.test_table) group by col1, col2);").strip() + assert(result_on_new_version == expected) -def test_aggregate_function_versioning_fetch_data_from_new_to_old_server(start_cluster): - for node in [node1, node2]: - create_and_fill_table(node) +def test_aggregate_function_versioning_fetch_data_from_old_to_new_server(start_cluster): + for node in [node1, node4]: + create_table(node) + insert_data(node) expected = "([1],[300])" new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() assert(new_server_data == expected) - old_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + old_server_data = node4.query("select finalizeAggregation(col3) from default.test_table;").strip() assert(old_server_data != expected) - data_from_old_to_new_server = node1.query("select finalizeAggregation(col3) from remote('node2', default.test_table);").strip() + data_from_old_to_new_server = node1.query("select finalizeAggregation(col3) from remote('node4', default.test_table);").strip() assert(data_from_old_to_new_server == old_server_data) def test_aggregate_function_versioning_server_upgrade(start_cluster): - for node in [node1, node2]: - create_and_fill_table(node) + for node in [node1, node5]: + create_table(node) + insert_data(node1, col2=5) + insert_data(node5, col2=1) + + # Serialization with version 0. + old_server_data = node5.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(old_server_data == "([1],[44])") + + # Upgrade server. + node5.restart_with_latest_version() + + # Deserialized with version 0. + upgraded_server_data = node5.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(upgraded_server_data == "([1],[44])") + + # Data from upgraded server to new server. Deserialize with version 0. + data_from_upgraded_to_new_server = node1.query("select finalizeAggregation(col3) from remote('node5', default.test_table);").strip() + assert(data_from_upgraded_to_new_server == upgraded_server_data == "([1],[44])") + + upgraded_server_data = node5.query("select finalizeAggregation(col3) from remote('127.0.0.{1,2}', default.test_table);").strip() + assert(upgraded_server_data == "([1],[44])\n([1],[44])") + + # Check insertion after server upgarde. + insert_data(node5, col2=2) + + upgraded_server_data = node5.query("select finalizeAggregation(col3) from default.test_table order by col2;").strip() + assert(upgraded_server_data == "([1],[44])\n([1],[44])") new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() assert(new_server_data == "([1],[300])") - old_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() - assert(old_server_data == "([1],[44])") - node2.restart_with_latest_version() + # Insert from new server to upgraded server, data version 1. + node1.query("insert into table function remote('node5', default.test_table) select * from default.test_table;").strip() + upgraded_server_data = node5.query("select finalizeAggregation(col3) from default.test_table order by col2;").strip() + assert(upgraded_server_data == "([1],[44])\n([1],[44])\n([1],[44])") - # Check that after server upgrade aggregate function is serialized according to older version. - upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() - assert(upgraded_server_data == "([1],[44])") + insert_data(node1) + new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(new_server_data == "([1],[300])\n([1],[300])") - # Remote fetches are still with older version. - data_from_upgraded_to_new_server = node1.query("select finalizeAggregation(col3) from remote('node2', default.test_table);").strip() - assert(data_from_upgraded_to_new_server == upgraded_server_data == "([1],[44])") + # Create table with column with version 0 serialiazation to be used for futher check. + create_table(node1, name='test_table_0', version=0) + insert_data(node1, table_name='test_table_0', col2=3) + data = node1.query("select finalizeAggregation(col3) from default.test_table_0;").strip() + assert(data == "([1],[44])") - # Check it is ok to write into table with older version of aggregate function. - insert_data(node2) - - # Hm, should newly inserted data be serialized as old version? - upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() - assert(upgraded_server_data == "([1],[300])\n([1],[44])") + # Insert from new server to upgraded server, data version 0. + node1.query("insert into table function remote('node5', default.test_table) select * from default.test_table_0;").strip() + upgraded_server_data = node5.query("select finalizeAggregation(col3) from default.test_table order by col2;").strip() + assert(upgraded_server_data == "([1],[44])\n([1],[44])\n([1],[44])\n([1],[44])") def test_aggregate_function_versioning_persisting_metadata(start_cluster): - for node in [node1, node2]: - create_and_fill_table(node) - node2.restart_with_latest_version() + for node in [node1, node6]: + create_table(node) + insert_data(node) + data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(data == "([1],[300])") + data = node6.query("select finalizeAggregation(col3) from default.test_table;").strip() + assert(data == "([1],[44])") - for node in [node1, node2]: + node6.restart_with_latest_version() + + for node in [node1, node6]: node.query("DETACH TABLE test_table") node.query("ATTACH TABLE test_table") - for node in [node1, node2]: + for node in [node1, node6]: insert_data(node) new_server_data = node1.query("select finalizeAggregation(col3) from default.test_table;").strip() assert(new_server_data == "([1],[300])\n([1],[300])") - upgraded_server_data = node2.query("select finalizeAggregation(col3) from default.test_table;").strip() + + upgraded_server_data = node6.query("select finalizeAggregation(col3) from default.test_table;").strip() assert(upgraded_server_data == "([1],[44])\n([1],[44])") - for node in [node1, node2]: + for node in [node1, node6]: node.restart_clickhouse() insert_data(node) result = node1.query("select finalizeAggregation(col3) from remote('127.0.0.{1,2}', default.test_table);").strip() - assert(result == "([1],[300])\n([1],[300])\n([1],[300])\n([1],[300])") - result = node2.query("select finalizeAggregation(col3) from remote('127.0.0.{1,2}', default.test_table);").strip() - assert(result == "([1],[44])\n([1],[44])\n([1],[44])\n([1],[44])") + assert(result == "([1],[300])\n([1],[300])\n([1],[300])\n([1],[300])\n([1],[300])\n([1],[300])") + + result = node6.query("select finalizeAggregation(col3) from remote('127.0.0.{1,2}', default.test_table);").strip() + assert(result == "([1],[44])\n([1],[44])\n([1],[44])\n([1],[44])\n([1],[44])\n([1],[44])") From 5e3202e4c5f9d82bc054f2e4123be520aef39ffe Mon Sep 17 00:00:00 2001 From: Vxider Date: Tue, 1 Jun 2021 03:01:35 +0000 Subject: [PATCH 091/609] update code style --- src/Functions/FunctionsWindow.cpp | 625 +++++++++++++++++- src/Functions/FunctionsWindow.h | 624 +---------------- src/Storages/WindowView/StorageWindowView.cpp | 8 +- 3 files changed, 633 insertions(+), 624 deletions(-) diff --git a/src/Functions/FunctionsWindow.cpp b/src/Functions/FunctionsWindow.cpp index 11dc5981a5b..b73bd151852 100644 --- a/src/Functions/FunctionsWindow.cpp +++ b/src/Functions/FunctionsWindow.cpp @@ -1,8 +1,631 @@ -#include +#include + +#include +#include +#include +#include +#include +#include +#include #include +#include +#include namespace DB { + +namespace +{ + std::tuple + dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) + { + const auto * interval_type = checkAndGetDataType(interval_column.type.get()); + if (!interval_type) + throw Exception( + "Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN); + const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); + if (!interval_column_const_int64) + throw Exception( + "Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN); + Int64 num_units = interval_column_const_int64->getValue(); + if (num_units <= 0) + throw Exception( + "Value for column " + interval_column.name + " of function " + function_name + " must be positive", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + return {interval_type->getKind(), num_units}; + } + + ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) + { + if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) + { + if (!checkColumn>(*col_tuple->getColumnPtr(index))) + throw Exception( + "Illegal column for first argument of function " + function_name + ". Must be a Tuple(DataTime, DataTime)", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + return col_tuple->getColumnPtr(index); + } + else + { + throw Exception( + "Illegal column for first argument of function " + function_name + ". Must be Tuple", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + + void checkFirstArgument(const ColumnWithTypeAndName & argument, const String & function_name) + { + if (!isDateTime(argument.type)) + throw Exception( + "Illegal type " + argument.type->getName() + " of argument of function " + function_name + + ". Should be a date with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) + { + auto interval_type = checkAndGetDataType(argument.type.get()); + if (!interval_type) + throw Exception( + "Illegal type " + argument.type->getName() + " of argument of function " + function_name + + ". Should be an interval of time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + interval_kind = interval_type->getKind(); + result_type_is_date = (interval_type->getKind() == IntervalKind::Year) || (interval_type->getKind() == IntervalKind::Quarter) + || (interval_type->getKind() == IntervalKind::Month) || (interval_type->getKind() == IntervalKind::Week); + } + + void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, bool & result_type_is_date) + { + IntervalKind interval_kind; + checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); + } + + void checkTimeZoneArgument( + const ColumnWithTypeAndName & argument, + const String & function_name) + { + if (!WhichDataType(argument.type).isString()) + throw Exception( + "Illegal type " + argument.type->getName() + " of argument of function " + function_name + + ". This argument is optional and must be a constant string with timezone name", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + bool checkIntervalOrTimeZoneArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) + { + if (WhichDataType(argument.type).isString()) + { + checkTimeZoneArgument(argument, function_name); + return false; + } + checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); + return true; + } +} + +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + + if (arguments.size() == 2) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); + checkTimeZoneArgument(arguments.at(2), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2 or 3", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); + + return std::make_shared(DataTypes{dataType, dataType}); + } + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & interval_column = arguments[1]; + const auto & from_datatype = *time_column.type.get(); + const auto which_type = WhichDataType(from_datatype); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); + if (!which_type.isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto interval = dispatchForIntervalColumns(interval_column, function_name); + + switch (std::get<0>(interval)) + { + case IntervalKind::Second: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Minute: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Hour: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Day: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Week: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Month: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Quarter: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + case IntervalKind::Year: + return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); + for (size_t i = 0; i != size; ++i) + { + start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); + end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE_START"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 1) + { + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); + } + else + { + WindowImpl::getReturnType(arguments, function_name); + } + return std::make_shared(); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto which_type = WhichDataType(arguments[0].type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + else + result_column_ = arguments[0].column; + return executeWindowBound(result_column_, 0, function_name); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "TUMBLE_END"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return WindowImpl::getReturnType(arguments, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) + { + const auto which_type = WhichDataType(arguments[0].type); + ColumnPtr result_column_; + if (which_type.isDateTime()) + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + else + result_column_ = arguments[0].column; + return executeWindowBound(result_column_, 1, function_name); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + } + else if (arguments.size() == 4) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 3 or 4", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (interval_kind_1 != interval_kind_2) + throw Exception( + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + + DataTypePtr dataType = nullptr; + if (result_type_is_date) + dataType = std::make_shared(); + else + dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); + return std::make_shared(DataTypes{dataType, dataType}); + } + + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & hop_interval_column = arguments[1]; + const auto & window_interval_column = arguments[2]; + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) + { + case IntervalKind::Second: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr + execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + { + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + auto start = ColumnVector::create(); + auto end = ColumnVector::create(); + auto & start_data = start->getData(); + auto & end_data = end->getData(); + start_data.resize(size); + end_data.resize(size); + + for (size_t i = 0; i < size; ++i) + { + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); + wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); + + ToType wend_ = wend; + ToType wend_latest; + + do + { + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); + } while (wend_ > time_data[i]); + + end_data[i] = wend_latest; + start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); + } + MutableColumns result; + result.emplace_back(std::move(start)); + result.emplace_back(std::move(end)); + return ColumnTuple::create(std::move(result)); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "WINDOW_ID"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + bool result_type_is_date; + IntervalKind interval_kind_1; + IntervalKind interval_kind_2; + + if (arguments.size() == 2) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + } + else if (arguments.size() == 3) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) + { + if (interval_kind_1 != interval_kind_2) + throw Exception( + "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); + } + } + else if (arguments.size() == 4) + { + checkFirstArgument(arguments.at(0), function_name); + checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); + checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); + checkTimeZoneArgument(arguments.at(3), function_name); + } + else + { + throw Exception( + "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) + + ", should be 2, 3 or 4", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + if (result_type_is_date) + return std::make_shared(); + else + return std::make_shared(); + } + + [[maybe_unused]] static ColumnPtr + dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto & hop_interval_column = arguments[1]; + const auto & window_interval_column = arguments[2]; + const auto & from_datatype = *time_column.type.get(); + const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); + const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); + if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) + throw Exception( + "Illegal column " + time_column.name + " argument of function " + function_name + + ". Must contain dates or dates with time", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); + auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); + + if (std::get<1>(hop_interval) > std::get<1>(window_interval)) + throw Exception( + "Value for hop interval of function " + function_name + " must not larger than window interval", + ErrorCodes::ARGUMENT_OUT_OF_BOUND); + + switch (std::get<0>(window_interval)) + { + case IntervalKind::Second: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Minute: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Hour: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Day: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Week: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Month: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Quarter: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + case IntervalKind::Year: + return execute_hop_slice( + *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); + } + __builtin_unreachable(); + } + + template + static ColumnPtr + execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) + { + Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); + + const auto & time_data = time_column.getData(); + size_t size = time_column.size(); + + auto end = ColumnVector::create(); + auto & end_data = end->getData(); + end_data.resize(size); + for (size_t i = 0; i < size; ++i) + { + ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); + ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); + + ToType wend_ = wend; + ToType wend_latest; + + do + { + wend_latest = wend_; + wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); + } while (wend_ > time_data[i]); + + end_data[i] = wend_latest; + } + return end; + } + + [[maybe_unused]] static ColumnPtr + dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + ColumnPtr column = WindowImpl::dispatchForColumns(arguments, function_name); + return executeWindowBound(column, 1, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 2) + return dispatchForTumbleColumns(arguments, function_name); + else + { + const auto & third_column = arguments[2]; + if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) + return dispatchForTumbleColumns(arguments, function_name); + else + return dispatchForHopColumns(arguments, function_name); + } + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP_START"; + + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + if (arguments.size() == 1) + { + auto type_ = WhichDataType(arguments[0].type); + if (!type_.isTuple() && !type_.isUInt32()) + throw Exception( + "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", + ErrorCodes::ILLEGAL_COLUMN); + } + else + { + WindowImpl::getReturnType(arguments, function_name); + } + return std::make_shared(); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (arguments.size() == 1) + { + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; + } + else + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + return executeWindowBound(result_column_, 0, function_name); + } +}; + +template <> +struct WindowImpl +{ + static constexpr auto name = "HOP_END"; + + [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + return WindowImpl::getReturnType(arguments, function_name); + } + + [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) + { + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); + ColumnPtr result_column_; + if (arguments.size() == 1) + { + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column_ = time_column.column; + } + else + result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); + + return executeWindowBound(result_column_, 1, function_name); + } +}; + +template +DataTypePtr FunctionWindow::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + return WindowImpl::getReturnType(arguments, name); +} + +template +ColumnPtr FunctionWindow::executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const +{ + return WindowImpl::dispatchForColumns(arguments, name); +} + void registerFunctionsWindow(FunctionFactory& factory) { factory.registerFunction(); diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 9e0957c7cdc..0bbd6bb31aa 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -1,19 +1,8 @@ #pragma once -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include #include -#include +#include +#include namespace DB { @@ -135,610 +124,14 @@ struct ToStartOfTransform; ADD_TIME(Second, 1) #undef ADD_TIME -namespace -{ - std::tuple - dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const String & function_name) - { - const auto * interval_type = checkAndGetDataType(interval_column.type.get()); - if (!interval_type) - throw Exception( - "Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN); - const auto * interval_column_const_int64 = checkAndGetColumnConst(interval_column.column.get()); - if (!interval_column_const_int64) - throw Exception( - "Illegal column " + interval_column.name + " of argument of function " + function_name, ErrorCodes::ILLEGAL_COLUMN); - Int64 num_units = interval_column_const_int64->getValue(); - if (num_units <= 0) - throw Exception( - "Value for column " + interval_column.name + " of function " + function_name + " must be positive", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - return {interval_type->getKind(), num_units}; - } - - ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name) - { - if (const ColumnTuple * col_tuple = checkAndGetColumn(column.get()); col_tuple) - { - if (!checkColumn>(*col_tuple->getColumnPtr(index))) - throw Exception( - "Illegal column for first argument of function " + function_name + ". Must be a Tuple(DataTime, DataTime)", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - return col_tuple->getColumnPtr(index); - } - else - { - throw Exception( - "Illegal column for first argument of function " + function_name + ". Must be Tuple", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - } - - void checkFirstArgument(const ColumnWithTypeAndName & argument, const String & function_name) - { - if (!isDateTime(argument.type)) - throw Exception( - "Illegal type " + argument.type->getName() + " of argument of function " + function_name - + ". Should be a date with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) - { - auto interval_type = checkAndGetDataType(argument.type.get()); - if (!interval_type) - throw Exception( - "Illegal type " + argument.type->getName() + " of argument of function " + function_name - + ". Should be an interval of time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - interval_kind = interval_type->getKind(); - result_type_is_date = (interval_type->getKind() == IntervalKind::Year) || (interval_type->getKind() == IntervalKind::Quarter) - || (interval_type->getKind() == IntervalKind::Month) || (interval_type->getKind() == IntervalKind::Week); - } - - void checkIntervalArgument(const ColumnWithTypeAndName & argument, const String & function_name, bool & result_type_is_date) - { - IntervalKind interval_kind; - checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); - } - - void checkTimeZoneArgument( - const ColumnWithTypeAndName & argument, - const String & function_name) - { - if (!WhichDataType(argument.type).isString()) - throw Exception( - "Illegal type " + argument.type->getName() + " of argument of function " + function_name - + ". This argument is optional and must be a constant string with timezone name", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - - bool checkIntervalOrTimeZoneArgument(const ColumnWithTypeAndName & argument, const String & function_name, IntervalKind & interval_kind, bool & result_type_is_date) - { - if (WhichDataType(argument.type).isString()) - { - checkTimeZoneArgument(argument, function_name); - return false; - } - checkIntervalArgument(argument, function_name, interval_kind, result_type_is_date); - return true; - } -} - template struct WindowImpl { + static constexpr auto name = "UNKNOWN"; + static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name); -}; -template <> -struct WindowImpl -{ - static constexpr auto name = "TUMBLE"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - bool result_type_is_date; - - if (arguments.size() == 2) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); - } - else if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, result_type_is_date); - checkTimeZoneArgument(arguments.at(2), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - DataTypePtr dataType = nullptr; - if (result_type_is_date) - dataType = std::make_shared(); - else - dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0)); - - return std::make_shared(DataTypes{dataType, dataType}); - } - - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & interval_column = arguments[1]; - const auto & from_datatype = *time_column.type.get(); - const auto which_type = WhichDataType(from_datatype); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 2, 0); - if (!which_type.isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " of function " + function_name + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto interval = dispatchForIntervalColumns(interval_column, function_name); - - switch (std::get<0>(interval)) - { - case IntervalKind::Second: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Minute: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Hour: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Day: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Week: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Month: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Quarter: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - case IntervalKind::Year: - return execute_tumble(*time_column_vec, std::get<1>(interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr execute_tumble(const ColumnUInt32 & time_column, UInt64 num_units, const DateLUTImpl & time_zone) - { - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - auto start = ColumnVector::create(); - auto end = ColumnVector::create(); - auto & start_data = start->getData(); - auto & end_data = end->getData(); - start_data.resize(size); - end_data.resize(size); - for (size_t i = 0; i != size; ++i) - { - start_data[i] = ToStartOfTransform::execute(time_data[i], num_units, time_zone); - end_data[i] = AddTime::execute(start_data[i], num_units, time_zone); - } - MutableColumns result; - result.emplace_back(std::move(start)); - result.emplace_back(std::move(end)); - return ColumnTuple::create(std::move(result)); - } -}; - -template <> -struct WindowImpl -{ - static constexpr auto name = "TUMBLE_START"; - - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - if (arguments.size() == 1) - { - auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isUInt32()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", ErrorCodes::ILLEGAL_COLUMN); - } - else - { - WindowImpl::getReturnType(arguments, function_name); - } - return std::make_shared(); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto which_type = WhichDataType(arguments[0].type); - ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - else - result_column_ = arguments[0].column; - return executeWindowBound(result_column_, 0, function_name); - } -}; - -template <> -struct WindowImpl -{ - static constexpr auto name = "TUMBLE_END"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - return WindowImpl::getReturnType(arguments, function_name); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) - { - const auto which_type = WhichDataType(arguments[0].type); - ColumnPtr result_column_; - if (which_type.isDateTime()) - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - else - result_column_ = arguments[0].column; - return executeWindowBound(result_column_, 1, function_name); - } -}; - -template <> -struct WindowImpl -{ - static constexpr auto name = "HOP"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - bool result_type_is_date; - IntervalKind interval_kind_1; - IntervalKind interval_kind_2; - - if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - } - else if (arguments.size() == 4) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - checkTimeZoneArgument(arguments.at(3), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 3 or 4", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - if (interval_kind_1 != interval_kind_2) - throw Exception( - "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); - - DataTypePtr dataType = nullptr; - if (result_type_is_date) - dataType = std::make_shared(); - else - dataType = std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 3, 0)); - return std::make_shared(DataTypes{dataType, dataType}); - } - - static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & hop_interval_column = arguments[1]; - const auto & window_interval_column = arguments[2]; - const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); - if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " argument of function " + function_name - + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); - auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - - if (std::get<1>(hop_interval) > std::get<1>(window_interval)) - throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - switch (std::get<0>(window_interval)) - { - case IntervalKind::Second: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Minute: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Hour: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Day: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Week: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Month: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Quarter: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Year: - return execute_hop( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr - execute_hop(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) - { - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - auto start = ColumnVector::create(); - auto end = ColumnVector::create(); - auto & start_data = start->getData(); - auto & end_data = end->getData(); - start_data.resize(size); - end_data.resize(size); - - for (size_t i = 0; i < size; ++i) - { - ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); - ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - wstart = AddTime::execute(wend, -1 * window_num_units, time_zone); - - ToType wend_ = wend; - ToType wend_latest; - - do - { - wend_latest = wend_; - wend_ = AddTime::execute(wend_, -1 * hop_num_units, time_zone); - } while (wend_ > time_data[i]); - - end_data[i] = wend_latest; - start_data[i] = AddTime::execute(wend_latest, -1 * window_num_units, time_zone); - } - MutableColumns result; - result.emplace_back(std::move(start)); - result.emplace_back(std::move(end)); - return ColumnTuple::create(std::move(result)); - } -}; - -template <> -struct WindowImpl -{ - static constexpr auto name = "WINDOW_ID"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - bool result_type_is_date; - IntervalKind interval_kind_1; - IntervalKind interval_kind_2; - - if (arguments.size() == 2) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - } - else if (arguments.size() == 3) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - if (checkIntervalOrTimeZoneArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date)) - { - if (interval_kind_1 != interval_kind_2) - throw Exception( - "Illegal type of window and hop column of function " + function_name + ", must be same", ErrorCodes::ILLEGAL_COLUMN); - } - } - else if (arguments.size() == 4) - { - checkFirstArgument(arguments.at(0), function_name); - checkIntervalArgument(arguments.at(1), function_name, interval_kind_1, result_type_is_date); - checkIntervalArgument(arguments.at(2), function_name, interval_kind_2, result_type_is_date); - checkTimeZoneArgument(arguments.at(3), function_name); - } - else - { - throw Exception( - "Number of arguments for function " + function_name + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2, 3 or 4", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } - - if (result_type_is_date) - return std::make_shared(); - else - return std::make_shared(); - } - - [[maybe_unused]] static ColumnPtr - dispatchForHopColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto & hop_interval_column = arguments[1]; - const auto & window_interval_column = arguments[2]; - const auto & from_datatype = *time_column.type.get(); - const auto * time_column_vec = checkAndGetColumn(time_column.column.get()); - const DateLUTImpl & time_zone = extractTimeZoneFromFunctionArguments(arguments, 3, 0); - if (!WhichDataType(from_datatype).isDateTime() || !time_column_vec) - throw Exception( - "Illegal column " + time_column.name + " argument of function " + function_name - + ". Must contain dates or dates with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - auto hop_interval = dispatchForIntervalColumns(hop_interval_column, function_name); - auto window_interval = dispatchForIntervalColumns(window_interval_column, function_name); - - if (std::get<1>(hop_interval) > std::get<1>(window_interval)) - throw Exception( - "Value for hop interval of function " + function_name + " must not larger than window interval", - ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - switch (std::get<0>(window_interval)) - { - case IntervalKind::Second: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Minute: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Hour: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Day: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Week: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Month: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Quarter: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - case IntervalKind::Year: - return execute_hop_slice( - *time_column_vec, std::get<1>(hop_interval), std::get<1>(window_interval), time_zone); - } - __builtin_unreachable(); - } - - template - static ColumnPtr - execute_hop_slice(const ColumnUInt32 & time_column, UInt64 hop_num_units, UInt64 window_num_units, const DateLUTImpl & time_zone) - { - Int64 gcd_num_units = std::gcd(hop_num_units, window_num_units); - - const auto & time_data = time_column.getData(); - size_t size = time_column.size(); - - auto end = ColumnVector::create(); - auto & end_data = end->getData(); - end_data.resize(size); - for (size_t i = 0; i < size; ++i) - { - ToType wstart = ToStartOfTransform::execute(time_data[i], hop_num_units, time_zone); - ToType wend = AddTime::execute(wstart, hop_num_units, time_zone); - - ToType wend_ = wend; - ToType wend_latest; - - do - { - wend_latest = wend_; - wend_ = AddTime::execute(wend_, -1 * gcd_num_units, time_zone); - } while (wend_ > time_data[i]); - - end_data[i] = wend_latest; - } - return end; - } - - [[maybe_unused]] static ColumnPtr - dispatchForTumbleColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - ColumnPtr column = WindowImpl::dispatchForColumns(arguments, function_name); - return executeWindowBound(column, 1, function_name); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - if (arguments.size() == 2) - return dispatchForTumbleColumns(arguments, function_name); - else - { - const auto & third_column = arguments[2]; - if (arguments.size() == 3 && WhichDataType(third_column.type).isString()) - return dispatchForTumbleColumns(arguments, function_name); - else - return dispatchForHopColumns(arguments, function_name); - } - } -}; - -template <> -struct WindowImpl -{ - static constexpr auto name = "HOP_START"; - - static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - if (arguments.size() == 1) - { - auto type_ = WhichDataType(arguments[0].type); - if (!type_.isTuple() && !type_.isUInt32()) - throw Exception( - "Illegal type of first argument of function " + function_name + " should be DateTime, Tuple or UInt32", - ErrorCodes::ILLEGAL_COLUMN); - } - else - { - WindowImpl::getReturnType(arguments, function_name); - } - return std::make_shared(); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto which_type = WhichDataType(time_column.type); - ColumnPtr result_column_; - if (arguments.size() == 1) - { - if (which_type.isUInt32()) - return time_column.column; - else //isTuple - result_column_ = time_column.column; - } - else - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - return executeWindowBound(result_column_, 0, function_name); - } -}; - -template <> -struct WindowImpl -{ - static constexpr auto name = "HOP_END"; - - [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - return WindowImpl::getReturnType(arguments, function_name); - } - - [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) - { - const auto & time_column = arguments[0]; - const auto which_type = WhichDataType(time_column.type); - ColumnPtr result_column_; - if (arguments.size() == 1) - { - if (which_type.isUInt32()) - return time_column.column; - else //isTuple - result_column_ = time_column.column; - } - else - result_column_ = WindowImpl::dispatchForColumns(arguments, function_name); - - return executeWindowBound(result_column_, 1, function_name); - } + static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name); }; template @@ -753,12 +146,9 @@ public: bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2, 3}; } - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return WindowImpl::getReturnType(arguments, name); } + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override - { - return WindowImpl::dispatchForColumns(arguments, name); - } + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t /*input_rows_count*/) const override; }; using FunctionTumble = FunctionWindow; diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index d7fdf759b3b..2a8a4abaaa2 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -23,18 +23,14 @@ #include #include #include -#include -#include #include #include #include -#include #include #include #include #include #include -#include #include #include @@ -593,7 +589,7 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery(cons return inner_create_query; } -inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) +UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) { IntervalKind window_interval_kind; if (is_tumble) @@ -628,7 +624,7 @@ inline UInt32 StorageWindowView::getWindowLowerBound(UInt32 time_sec) __builtin_unreachable(); } -inline UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) +UInt32 StorageWindowView::getWindowUpperBound(UInt32 time_sec) { IntervalKind window_interval_kind; if (is_tumble) From 7453d0ce58f3d8c5e354b6a9e253685b44fad2ae Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 1 Jun 2021 08:45:04 +0000 Subject: [PATCH 092/609] Fxi checks --- src/AggregateFunctions/IAggregateFunction.h | 2 +- src/DataTypes/DataTypeAggregateFunction.cpp | 2 +- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index dfadd76766a..5da7ba5170c 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -105,7 +105,7 @@ public: virtual void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const = 0; /// Serializes state (to transmit it over the network, for example). - virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional verison = std::nullopt) const = 0; + virtual void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional version = std::nullopt) const = 0; /// Deserializes state. This function is called only for empty (just created) states. virtual void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version = std::nullopt, Arena * arena = nullptr) const = 0; diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index ffdd5b6cb94..15fcab5d038 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -151,7 +151,7 @@ static DataTypePtr create(const ASTPtr & arguments) * it is parametric, or [ASTIdentifier, types...] - otherwise. If aggregate function has version in AST, then it will be: * [ASTLitearl, ASTFunction (or ASTIdentifier), types...]. */ - if (auto version_ast = arguments->children[0]->as()) + if (auto * version_ast = arguments->children[0]->as()) { version = version_ast->value.safeGet(); data_type_ast = arguments->children[1]; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d0aa58f868d..76ca9b1529c 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -395,7 +395,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( { column_type = DataTypeFactory::instance().get(col_decl.type); - auto aggregate_function_type = typeid_cast(column_type.get()); + const auto * aggregate_function_type = typeid_cast(column_type.get()); if (attach && aggregate_function_type && aggregate_function_type->isVersioned()) aggregate_function_type->setVersionIfEmpty(0); From 92a59398bf45aee6031036f10a36d699ec811e16 Mon Sep 17 00:00:00 2001 From: Vxider Date: Wed, 2 Jun 2021 01:52:54 +0000 Subject: [PATCH 093/609] update code style --- src/Functions/FunctionsWindow.cpp | 8 ++++++++ src/Functions/FunctionsWindow.h | 7 ------- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Functions/FunctionsWindow.cpp b/src/Functions/FunctionsWindow.cpp index b73bd151852..61513b18197 100644 --- a/src/Functions/FunctionsWindow.cpp +++ b/src/Functions/FunctionsWindow.cpp @@ -14,6 +14,14 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ARGUMENT_OUT_OF_BOUND; +} + namespace { std::tuple diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 0bbd6bb31aa..ad008f6da39 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -40,13 +40,6 @@ enum WindowFunctionName HOP_END, WINDOW_ID }; -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ARGUMENT_OUT_OF_BOUND; -} template struct ToStartOfTransform; From bbfa407b3864183dce80d9ebfdf4ca541852b837 Mon Sep 17 00:00:00 2001 From: Vxider Date: Thu, 3 Jun 2021 02:34:11 +0000 Subject: [PATCH 094/609] update tests --- tests/queries/0_stateless/01056_window_view_proc_hop_watch.py | 2 +- .../0_stateless/01059_window_view_event_hop_watch_strict_asc.py | 2 +- .../0_stateless/01062_window_view_event_hop_watch_asc.py | 2 +- .../0_stateless/01065_window_view_event_hop_watch_bounded.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index fde6f4acba4..687fe3fa2bb 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py index 8d8746965d7..d53a4be9366 100755 --- a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py +++ b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py index f765ed58c42..ea956b3ccdb 100755 --- a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py +++ b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal diff --git a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py index 0c3b2ddff5c..83947544e8f 100755 --- a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py +++ b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py @@ -1,4 +1,4 @@ -#!/usr/bin/env python +#!/usr/bin/env python3 import os import sys import signal From a2997535e10c37275ade8e1637db55b18e6752e4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 3 Jun 2021 11:31:02 +0000 Subject: [PATCH 095/609] Small improvement --- src/AggregateFunctions/AggregateFunctionSumMap.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 549cd5b3137..ec9f2ecaf13 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -286,8 +286,8 @@ public: serialize = [&](size_t col_idx, const Array & values) { const auto & type = values_types[col_idx]; - if (isInteger(type)) - SerializationNumber().serializeBinary(values[col_idx], buf); + if (!type->isNullable()) + type->promoteNumericType()->getDefaultSerialization()->serializeBinary(values[col_idx], buf); else values_serializations[col_idx]->serializeBinary(values[col_idx], buf); }; @@ -325,8 +325,8 @@ public: deserialize = [&](size_t col_idx, Array & values) { const auto & type = values_types[col_idx]; - if (isInteger(type)) - SerializationNumber().deserializeBinary(values[col_idx], buf); + if (!type->isNullable()) + type->promoteNumericType()->getDefaultSerialization()->deserializeBinary(values[col_idx], buf); else values_serializations[col_idx]->deserializeBinary(values[col_idx], buf); }; From ea936e6f94736e9e4a47e1494a105c2256324873 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Tue, 10 Aug 2021 14:52:34 +0300 Subject: [PATCH 096/609] Adding interface for local fuzzing --- programs/local/CMakeLists.txt | 13 + programs/local/LocalServer.cpp | 289 +++++++++++------- programs/local/LocalServer.h | 3 +- src/Core/Settings.h | 2 +- src/Functions/CMakeLists.txt | 4 + src/Functions/getFuzzerData.cpp | 51 ++++ .../registerFunctionsMiscellaneous.cpp | 8 + 7 files changed, 263 insertions(+), 107 deletions(-) create mode 100644 src/Functions/getFuzzerData.cpp diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index 530128c2041..62811b9c1b9 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -17,3 +17,16 @@ clickhouse_program_add(local) if(NOT CLICKHOUSE_ONE_SHARED) target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib) endif() + +if (ENABLE_FUZZING) + add_compile_definitions(FUZZING_MODE=1) + add_executable(fuzz-clickhouse-local LocalServer.cpp ${SRCS}) + target_link_libraries(fuzz-clickhouse-local PRIVATE + dbms + ${LIB_FUZZING_ENGINE} + loggers + clickhouse_functions + clickhouse_aggregate_functions + clickhouse_storages_system + clickhouse_table_functions) +endif () diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index e256338a538..7f3a3be3815 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -211,6 +211,11 @@ try throw Exception("Specify either `query` or `queries-file` option", ErrorCodes::BAD_ARGUMENTS); } + // Non-fuzzer mode: main function runs only one time, + // so first_time clauses does not affect anything + static bool first_time = true; + if (first_time) + { shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); @@ -299,15 +304,119 @@ try { attachSystemTables(global_context); } + status.reset(); + } - processQueries(); + /// processing queries + static String initial_create_query = getInitialCreateTableQuery(); + static String queries_str = initial_create_query; + + if (first_time) + { + if (config().has("query")) + queries_str += config().getRawString("query"); + else + { + String queries_from_file; + ReadBufferFromFile in(config().getString("queries-file")); + readStringUntilEOF(queries_from_file, in); + queries_str += queries_from_file; + } + } + + static const auto & settings = global_context->getSettingsRef(); + + static std::vector queries; + static auto parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); + + if (!parse_res.second) + throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); + + /// we can't mutate global global_context (can lead to races, as it was already passed to some background threads) + /// so we can't reuse it safely as a query context and need a copy here + auto context = Context::createCopy(global_context); + + context->makeSessionContext(); + context->makeQueryContext(); + + context->setUser("default", "", Poco::Net::SocketAddress{}); + context->setCurrentQueryId(""); + applyCmdSettings(context); + + /// Use the same query_id (and thread group) for all queries + CurrentThread::QueryScope query_scope_holder(context); + + /// Set progress show + need_render_progress = config().getBool("progress", false); + + std::function finalize_progress; + if (need_render_progress) + { + /// Set progress callback, which can be run from multiple threads. + context->setProgressCallback([&](const Progress & value) + { + /// Write progress only if progress was updated + if (progress_indication.updateProgress(value)) + progress_indication.writeProgress(); + }); + + /// Set finalizing callback for progress, which is called right before finalizing query output. + finalize_progress = [&]() + { + progress_indication.clearProgressOutput(); + }; + + /// Set callback for file processing progress. + progress_indication.setFileProgressCallback(context); + } + + bool echo_queries = config().hasOption("echo") || config().hasOption("verbose"); + + std::exception_ptr exception; + first_time = false; + + for (const auto & query : queries) + { + written_first_block = false; + progress_indication.resetProgress(); + + ReadBufferFromString read_buf(query); + WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO); + if (echo_queries) + { + writeString(query, write_buf); + writeChar('\n', write_buf); + write_buf.next(); + } + + try + { + executeQuery(read_buf, write_buf, /* allow_into_outfile = */ true, context, {}, {}, finalize_progress); + } + catch (...) + { + if (!config().hasOption("ignore-error")) + { + throw; + } + + if (!exception) + exception = std::current_exception(); + + std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; + } + } + + if (exception) + std::rethrow_exception(exception); + +#ifndef FUZZING_MODE global_context->shutdown(); global_context.reset(); - status.reset(); cleanup(); - +#endif return Application::EXIT_OK; } catch (const Exception & e) @@ -350,107 +459,6 @@ std::string LocalServer::getInitialCreateTableQuery() "; "; } - -void LocalServer::processQueries() -{ - String initial_create_query = getInitialCreateTableQuery(); - String queries_str = initial_create_query; - - if (config().has("query")) - queries_str += config().getRawString("query"); - else - { - String queries_from_file; - ReadBufferFromFile in(config().getString("queries-file")); - readStringUntilEOF(queries_from_file, in); - queries_str += queries_from_file; - } - - const auto & settings = global_context->getSettingsRef(); - - std::vector queries; - auto parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); - - if (!parse_res.second) - throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); - - /// we can't mutate global global_context (can lead to races, as it was already passed to some background threads) - /// so we can't reuse it safely as a query context and need a copy here - auto context = Context::createCopy(global_context); - - context->makeSessionContext(); - context->makeQueryContext(); - - context->setUser("default", "", Poco::Net::SocketAddress{}); - context->setCurrentQueryId(""); - applyCmdSettings(context); - - /// Use the same query_id (and thread group) for all queries - CurrentThread::QueryScope query_scope_holder(context); - - /// Set progress show - need_render_progress = config().getBool("progress", false); - - std::function finalize_progress; - if (need_render_progress) - { - /// Set progress callback, which can be run from multiple threads. - context->setProgressCallback([&](const Progress & value) - { - /// Write progress only if progress was updated - if (progress_indication.updateProgress(value)) - progress_indication.writeProgress(); - }); - - /// Set finalizing callback for progress, which is called right before finalizing query output. - finalize_progress = [&]() - { - progress_indication.clearProgressOutput(); - }; - - /// Set callback for file processing progress. - progress_indication.setFileProgressCallback(context); - } - - bool echo_queries = config().hasOption("echo") || config().hasOption("verbose"); - - std::exception_ptr exception; - - for (const auto & query : queries) - { - written_first_block = false; - progress_indication.resetProgress(); - - ReadBufferFromString read_buf(query); - WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO); - - if (echo_queries) - { - writeString(query, write_buf); - writeChar('\n', write_buf); - write_buf.next(); - } - - try - { - executeQuery(read_buf, write_buf, /* allow_into_outfile = */ true, context, {}, {}, finalize_progress); - } - catch (...) - { - if (!config().hasOption("ignore-error")) - throw; - - if (!exception) - exception = std::current_exception(); - - std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; - } - } - - if (exception) - std::rethrow_exception(exception); -} - static const char * minimal_default_user_xml = "" " " @@ -661,9 +669,80 @@ void LocalServer::applyCmdOptions(ContextMutablePtr context) #pragma GCC diagnostic ignored "-Wunused-function" #pragma GCC diagnostic ignored "-Wmissing-declarations" -int mainEntryClickHouseLocal(int argc, char ** argv) +#ifdef FUZZING_MODE +#include + +class FuzzApp { DB::LocalServer app; + +public: + inline void init(int argc, char ** argv) + { + app.init(argc, argv); + } + + inline int run() + { + return app.run(); + } +} fuzz_app; + +extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) +{ + int & argc = *pargc; + char ** argv = *pargv; + + // position of delimiter "--" that separates arguments + // of clickhouse-local and fuzzer + int pos_delim = argc; + for (int i = 0; i < argc; ++i) + { + if (strcmp(argv[i], "--") == 0) + { + pos_delim = i; + break; + } + } + + fuzz_app.init(pos_delim, argv); + for (int i = pos_delim + 1; i < argc; ++i) + std::swap(argv[i], argv[i - pos_delim]); + argc -= pos_delim; + if (argc == 0) // no delimiter provided + ++argc; + return 0; +} + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +{ + try + { + // inappropriate symbol for fuzzing at the end + if (size) + --size; + auto cur_str = String(reinterpret_cast(data), size); + // to clearly see the beginning and the end + std::cerr << '>' << cur_str << '<' << std::endl; + DB::FunctionGetFuzzerData::update(cur_str); + fuzz_app.run(); + } + catch (...) + { + std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; + return 1; + } + return 0; +} +#endif + +int mainEntryClickHouseLocal(int argc, char ** argv) +{ +#ifdef FUZZING_MODE + FuzzApp & app = fuzz_app; +#else + DB::LocalServer app; +#endif try { app.init(argc, argv); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index e82caad7542..ed79e482bc9 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -38,7 +38,6 @@ private: void tryInitPath(); void applyCmdOptions(ContextMutablePtr context); void applyCmdSettings(ContextMutablePtr context); - void processQueries(); void setupUsers(); void cleanup(); @@ -60,3 +59,5 @@ protected: }; } + +int mainEntryClickHouseLocal(int argc, char ** argv); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e1bd1d29153..23f4e7911f3 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -575,7 +575,7 @@ class IColumn; \ M(String, format_regexp, "", "Regular expression (for Regexp format)", 0) \ M(String, format_regexp_escaping_rule, "Raw", "Field escaping rule (for Regexp format)", 0) \ - M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ + M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format)", 0) \ \ M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ M(Bool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ diff --git a/src/Functions/CMakeLists.txt b/src/Functions/CMakeLists.txt index 04e5f80468b..f35f8a411b1 100644 --- a/src/Functions/CMakeLists.txt +++ b/src/Functions/CMakeLists.txt @@ -127,3 +127,7 @@ set_source_files_properties("pointInPolygon.cpp" PROPERTIES COMPILE_FLAGS -fno-s # target_link_libraries(clickhouse_functions PRIVATE ${S2_LIBRARY}) target_include_directories(clickhouse_functions SYSTEM PUBLIC ${S2_GEOMETRY_INCLUDE_DIR}) + +if (ENABLE_FUZZING) + add_compile_definitions(FUZZING_MODE=1) +endif () diff --git a/src/Functions/getFuzzerData.cpp b/src/Functions/getFuzzerData.cpp new file mode 100644 index 00000000000..ef944593b2c --- /dev/null +++ b/src/Functions/getFuzzerData.cpp @@ -0,0 +1,51 @@ +#include +#include +#include +#include + +namespace DB +{ +class FunctionGetFuzzerData : public IFunction +{ + inline static String fuzz_data; + +public: + static constexpr auto name = "getFuzzerData"; + + inline static FunctionPtr create(ContextPtr) { return create(); } + + static FunctionPtr create() + { + return std::make_shared(); + } + + inline String getName() const override { return name; } + + inline size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + inline bool isDeterministic() const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, + const DataTypePtr &, + size_t input_rows_count) const override + { + return DataTypeString().createColumnConst(input_rows_count, fuzz_data); + } + + static void update(const String & fuzz_data_) + { + fuzz_data = fuzz_data_; + } +}; + +void registerFunctionGetFuzzerData(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerAlias("get_fuzzer_data", FunctionGetFuzzerData::name, FunctionFactory::CaseInsensitive); +} +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 12c54aeeefd..3bd644a14f5 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -83,6 +83,10 @@ void registerFunctionInitialQueryID(FunctionFactory & factory); void registerFunctionConvertCharset(FunctionFactory &); #endif +#ifdef FUZZING_MODE +void registerFunctionGetFuzzerData(FunctionFactory & factory); +#endif + void registerFunctionsMiscellaneous(FunctionFactory & factory) { registerFunctionCurrentDatabase(factory); @@ -160,6 +164,10 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) #if USE_ICU registerFunctionConvertCharset(factory); #endif + +#ifdef FUZZING_MODE + registerFunctionGetFuzzerData(factory); +#endif } } From 5826c7592a54a9776aaff82cc11008573e436820 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Thu, 12 Aug 2021 15:31:19 +0300 Subject: [PATCH 097/609] Update (see description) Improved parsing exception handling and changed the flag for file querying --- programs/local/LocalServer.cpp | 31 ++++++++++++++++++++++++++----- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 7f3a3be3815..5d71baef33e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -328,11 +328,33 @@ try static const auto & settings = global_context->getSettingsRef(); static std::vector queries; - static auto parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); + if (first_time) + { + std::pair parse_res; +#ifdef FUZZING_MODE + try + { +#endif + parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); +#ifdef FUZZING_MODE + } + catch (const Exception &) + { + // will be caught at the end of the main + throw; + } + catch (...) + { + std::cerr << "Undefined error while parsing" << std::endl; + exit(1); + } +#endif if (!parse_res.second) throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); + } + first_time = false; /// we can't mutate global global_context (can lead to races, as it was already passed to some background threads) /// so we can't reuse it safely as a query context and need a copy here auto context = Context::createCopy(global_context); @@ -349,7 +371,6 @@ try /// Set progress show need_render_progress = config().getBool("progress", false); - std::function finalize_progress; if (need_render_progress) { @@ -374,7 +395,6 @@ try bool echo_queries = config().hasOption("echo") || config().hasOption("verbose"); std::exception_ptr exception; - first_time = false; for (const auto & query : queries) { @@ -565,7 +585,7 @@ void LocalServer::init(int argc, char ** argv) ("help", "produce help message") ("config-file,c", po::value(), "config-file path") ("query,q", po::value(), "query") - ("queries-file, qf", po::value(), "file path with queries to execute") + ("queries-file,Q", po::value(), "file path with queries to execute") ("database,d", po::value(), "database") ("table,N", po::value(), "name of the initial table") @@ -730,7 +750,8 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 1; + auto code = DB::getCurrentExceptionCode(); + return code ? code : 1; } return 0; } From a1e8d5d638b9a65c60bed452724cfbdba3d4ce88 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Thu, 12 Aug 2021 21:21:32 +0300 Subject: [PATCH 098/609] Fix creation global app clickhouse-server cannot run if FuzzApp is constructed. --- programs/local/LocalServer.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 5d71baef33e..98fb31c0b8f 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -706,7 +706,9 @@ public: { return app.run(); } -} fuzz_app; +}; + +std::optional fuzz_app; extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) { @@ -725,7 +727,8 @@ extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) } } - fuzz_app.init(pos_delim, argv); + fuzz_app.emplace(); + fuzz_app->init(pos_delim, argv); for (int i = pos_delim + 1; i < argc; ++i) std::swap(argv[i], argv[i - pos_delim]); argc -= pos_delim; @@ -745,7 +748,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) // to clearly see the beginning and the end std::cerr << '>' << cur_str << '<' << std::endl; DB::FunctionGetFuzzerData::update(cur_str); - fuzz_app.run(); + fuzz_app->run(); } catch (...) { @@ -759,11 +762,7 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) int mainEntryClickHouseLocal(int argc, char ** argv) { -#ifdef FUZZING_MODE - FuzzApp & app = fuzz_app; -#else DB::LocalServer app; -#endif try { app.init(argc, argv); From 8f6b98c7621b47a5211188a8bebf49593b4ebdd6 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Fri, 13 Aug 2021 14:32:50 +0300 Subject: [PATCH 099/609] Removed redundant class --- programs/local/LocalServer.cpp | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 98fb31c0b8f..9beb2ba7cb3 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -692,23 +692,7 @@ void LocalServer::applyCmdOptions(ContextMutablePtr context) #ifdef FUZZING_MODE #include -class FuzzApp -{ - DB::LocalServer app; - -public: - inline void init(int argc, char ** argv) - { - app.init(argc, argv); - } - - inline int run() - { - return app.run(); - } -}; - -std::optional fuzz_app; +std::optional fuzz_app; extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) { From f5b37883085679156b52b0f9a90c327a3fe1dc84 Mon Sep 17 00:00:00 2001 From: Alexey Boykov <33257111+mathalex@users.noreply.github.com> Date: Mon, 16 Aug 2021 20:22:39 +0300 Subject: [PATCH 100/609] Update LocalServer.h --- programs/local/LocalServer.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index ed79e482bc9..4da8142a6ff 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -59,5 +59,3 @@ protected: }; } - -int mainEntryClickHouseLocal(int argc, char ** argv); From f1976bdf96c91666f3ab4e8805bc2f93c594472d Mon Sep 17 00:00:00 2001 From: Alexey Boykov <33257111+mathalex@users.noreply.github.com> Date: Mon, 23 Aug 2021 19:34:14 +0300 Subject: [PATCH 101/609] Update getFuzzerData.cpp --- src/Functions/getFuzzerData.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/getFuzzerData.cpp b/src/Functions/getFuzzerData.cpp index ef944593b2c..2570347598c 100644 --- a/src/Functions/getFuzzerData.cpp +++ b/src/Functions/getFuzzerData.cpp @@ -29,6 +29,8 @@ public: } inline bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, From fbe453ff9d411939254eff32645b8356f9db63f5 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Tue, 31 Aug 2021 15:15:04 +0300 Subject: [PATCH 102/609] style --- programs/local/LocalServer.cpp | 6 ++---- src/Functions/getFuzzerData.cpp | 2 +- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 1885ca68e83..faa9d2a60a5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -394,9 +394,9 @@ try /// Set finalizing callback for progress, which is called right before finalizing query output. finalize_progress = [&]() - { + { progress_indication.clearProgressOutput(); - }; + }; /// Set callback for file processing progress. progress_indication.setFileProgressCallback(context); @@ -427,9 +427,7 @@ try catch (...) { if (!config().hasOption("ignore-error")) - { throw; - } if (!exception) exception = std::current_exception(); diff --git a/src/Functions/getFuzzerData.cpp b/src/Functions/getFuzzerData.cpp index 2570347598c..c01f575f0be 100644 --- a/src/Functions/getFuzzerData.cpp +++ b/src/Functions/getFuzzerData.cpp @@ -29,7 +29,7 @@ public: } inline bool isDeterministic() const override { return false; } - + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } ColumnPtr executeImpl(const ColumnsWithTypeAndName &, From eb3d251bb0dfe62bfb8c3944fd79dfa6445b165b Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Tue, 31 Aug 2021 17:21:23 +0300 Subject: [PATCH 103/609] try fix --- programs/local/LocalServer.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index faa9d2a60a5..6e65193857a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -213,6 +213,8 @@ try throw Exception("Specify either `query` or `queries-file` option", ErrorCodes::BAD_ARGUMENTS); } + std::optional status; + // Non-fuzzer mode: main function runs only one time, // so first_time clauses does not affect anything static bool first_time = true; @@ -224,8 +226,6 @@ try global_context->setApplicationType(Context::ApplicationType::LOCAL); tryInitPath(); - std::optional status; - /// Skip temp path installation /// We will terminate process on error @@ -315,7 +315,6 @@ try { attachSystemTables(global_context); } - status.reset(); } /// processing queries @@ -443,6 +442,7 @@ try global_context->shutdown(); global_context.reset(); + status.reset(); cleanup(); #endif return Application::EXIT_OK; From 24f47e9e3a6db4aa25cc00da01fcb1cd11e8f0ca Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Wed, 1 Sep 2021 16:24:31 +0300 Subject: [PATCH 104/609] delete static Attempt to fix recursive_mutex lock failed. --- programs/local/CMakeLists.txt | 1 + programs/local/LocalServer.cpp | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index 62811b9c1b9..d965ecf05be 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -20,6 +20,7 @@ endif() if (ENABLE_FUZZING) add_compile_definitions(FUZZING_MODE=1) + set (WITH_COVERAGE ON) add_executable(fuzz-clickhouse-local LocalServer.cpp ${SRCS}) target_link_libraries(fuzz-clickhouse-local PRIVATE dbms diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 6e65193857a..0a422576545 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -319,8 +319,8 @@ try /// processing queries - static String initial_create_query = getInitialCreateTableQuery(); - static String queries_str = initial_create_query; + String initial_create_query = getInitialCreateTableQuery(); + String queries_str = initial_create_query; if (first_time) { @@ -335,7 +335,7 @@ try } } - static const auto & settings = global_context->getSettingsRef(); + const auto & settings = global_context->getSettingsRef(); static std::vector queries; if (first_time) From dbb697491eb90301b824475ebb7398ad598fa342 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Thu, 2 Sep 2021 16:28:17 +0300 Subject: [PATCH 105/609] Try make full compatibility --- programs/local/LocalServer.cpp | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0a422576545..86de654b939 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -215,11 +215,11 @@ try std::optional status; - // Non-fuzzer mode: main function runs only one time, - // so first_time clauses does not affect anything +#ifdef FUZZING_MODE static bool first_time = true; if (first_time) { +#endif shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); @@ -315,15 +315,19 @@ try { attachSystemTables(global_context); } +#ifdef FUZZING_MODE } +#endif /// processing queries String initial_create_query = getInitialCreateTableQuery(); String queries_str = initial_create_query; +#ifdef FUZZING_MODE if (first_time) { +#endif if (config().has("query")) queries_str += config().getRawString("query"); else @@ -333,13 +337,19 @@ try readStringUntilEOF(queries_from_file, in); queries_str += queries_from_file; } +#ifdef FUZZING_MODE } +#endif const auto & settings = global_context->getSettingsRef(); +#ifdef FUZZING_MODE static std::vector queries; if (first_time) { +#else + std::vector queries; +#endif std::pair parse_res; #ifdef FUZZING_MODE try @@ -362,9 +372,11 @@ try if (!parse_res.second) throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); +#ifdef FUZZING_MODE } - first_time = false; +#endif + /// Authenticate and create a context to execute queries. Session session{global_context, ClientInfo::Interface::LOCAL}; session.authenticate("default", "", {}); @@ -445,6 +457,7 @@ try status.reset(); cleanup(); #endif + return Application::EXIT_OK; } catch (const Exception & e) From fa933f6f8c36d2bf3a13882c4d3ccfc21a12a5e8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 12 Sep 2021 12:33:54 +0000 Subject: [PATCH 106/609] Full support for non-default schema --- .../materialized-postgresql.md | 37 +++- .../MaterializedPostgreSQLConsumer.cpp | 26 ++- .../MaterializedPostgreSQLConsumer.h | 2 + .../MaterializedPostgreSQLSettings.h | 4 + .../PostgreSQLReplicationHandler.cpp | 80 ++++++-- .../PostgreSQL/PostgreSQLReplicationHandler.h | 15 +- .../StorageMaterializedPostgreSQL.h | 5 + .../test.py | 194 ++++++++++++++++-- 8 files changed, 322 insertions(+), 41 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 77a5f2af0e0..7f4031a0a81 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -25,12 +25,14 @@ ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'p ## Settings {#settings} -- [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size) - - [materialized_postgresql_tables_list](../../operations/settings/settings.md#materialized-postgresql-tables-list) +- [materialized_postgresql_schema](../../operations/settings/settings.md#materialized-postgresql-schema) + - [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update) +- [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size) + - [materialized_postgresql_replication_slot](../../operations/settings/settings.md#materialized-postgresql-replication-slot) - [materialized_postgresql_snapshot](../../operations/settings/settings.md#materialized-postgresql-snapshot) @@ -38,12 +40,39 @@ ENGINE = MaterializedPostgreSQL('host:port', ['database' | database], 'user', 'p ``` sql CREATE DATABASE database1 ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') -SETTINGS materialized_postgresql_max_block_size = 65536, - materialized_postgresql_tables_list = 'table1,table2,table3'; +SETTINGS materialized_postgresql_tables_list = 'table1,table2,table3'; SELECT * FROM database1.table1; ``` +PostgreSQL [schema](https://www.postgresql.org/docs/9.1/ddl-schemas.html) can be used in two ways. + +1. One schema for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_schema`. +Tables are accessed via table name only: + +``` sql +CREATE DATABASE postgres_database +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') +SETTINGS materialized_postgresql_schema = 'postgres_schema'; + +SELECT * FROM postgres_database.table1; +``` + +2. Any number of schemas for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_tables_list`. Each table is written along with its schema. +Tables are accessed via schema name and table name at the same time: + +``` sql +CREATE DATABASE database1 +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') +SETTINGS materialized_postgresql_tables_list = 'schema1.table1,schema2.table2,schema1.table3'; + +SELECT * FROM database1.`schema1.table1`; +SELECT * FROM database1.`schema2.table2`; +``` + +But in this case all tables in `materialized_postgresql_tables_list` must be written with its schema name. + + ## Requirements {#requirements} 1. The [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) setting must have a value `logical` and `max_replication_slots` parameter must have a value at least `2` in the PostgreSQL config file. diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 1fc279bff23..7126f126cc0 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -25,6 +25,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( const std::string & publication_name_, const std::string & start_lsn, const size_t max_block_size_, + bool schema_as_a_part_of_table_name_, bool allow_automatic_update_, Storages storages_) : log(&Poco::Logger::get("PostgreSQLReaplicaConsumer")) @@ -35,6 +36,7 @@ MaterializedPostgreSQLConsumer::MaterializedPostgreSQLConsumer( , current_lsn(start_lsn) , lsn_value(getLSNValue(start_lsn)) , max_block_size(max_block_size_) + , schema_as_a_part_of_table_name(schema_as_a_part_of_table_name_) , allow_automatic_update(allow_automatic_update_) , storages(storages_) { @@ -371,19 +373,25 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl readString(replication_message, pos, size, relation_namespace); readString(replication_message, pos, size, relation_name); + String table_name; + if (!relation_namespace.empty() && schema_as_a_part_of_table_name) + table_name = relation_namespace + '.' + relation_name; + else + table_name = relation_name; + if (!isSyncAllowed(relation_id)) return; - if (storages.find(relation_name) == storages.end()) + if (storages.find(table_name) == storages.end()) { - markTableAsSkipped(relation_id, relation_name); + markTableAsSkipped(relation_id, table_name); LOG_ERROR(log, "Storage for table {} does not exist, but is included in replication stream. (Storages number: {})", - relation_name, storages.size()); + table_name, storages.size()); return; } - assert(buffers.count(relation_name)); + assert(buffers.count(table_name)); /// 'd' - default (primary key if any) @@ -397,7 +405,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { LOG_WARNING(log, "Table has replica identity {} - not supported. A table must have a primary key or a replica identity index"); - markTableAsSkipped(relation_id, relation_name); + markTableAsSkipped(relation_id, table_name); return; } @@ -409,7 +417,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl bool new_relation_definition = false; if (schema_data.find(relation_id) == schema_data.end()) { - relation_id_to_name[relation_id] = relation_name; + relation_id_to_name[relation_id] = table_name; schema_data.emplace(relation_id, SchemaData(num_columns)); new_relation_definition = true; } @@ -418,7 +426,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl if (current_schema_data.number_of_columns != num_columns) { - markTableAsSkipped(relation_id, relation_name); + markTableAsSkipped(relation_id, table_name); return; } @@ -440,13 +448,13 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl if (current_schema_data.column_identifiers[i].first != data_type_id || current_schema_data.column_identifiers[i].second != type_modifier) { - markTableAsSkipped(relation_id, relation_name); + markTableAsSkipped(relation_id, table_name); return; } } } - tables_to_sync.insert(relation_name); + tables_to_sync.insert(table_name); break; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 8f3224784f1..a1f0b053110 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -26,6 +26,7 @@ public: const String & publication_name_, const String & start_lsn, const size_t max_block_size_, + bool schema_as_a_part_of_table_name_, bool allow_automatic_update_, Storages storages_); @@ -106,6 +107,7 @@ private: UInt64 lsn_value; const size_t max_block_size; + bool schema_as_a_part_of_table_name; bool allow_automatic_update; String table_to_insert; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index cc147a01d32..7c661f95edf 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -19,6 +19,10 @@ namespace DB M(Bool, materialized_postgresql_allow_automatic_update, false, "Allow to reload table in the background, when schema changes are detected", 0) \ M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \ M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \ + M(String, materialized_postgresql_schema, "", "PostgreSQL schema", 0) \ + M(Bool, materialized_postgresql_tables_list_with_schema, true, \ + "Consider by default that if there is a dot in tables list 'name.name', " \ + "then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \ DECLARE_SETTINGS_TRAITS(MaterializedPostgreSQLSettingsTraits, LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 1bda6d13e11..c91600266e2 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -27,7 +27,7 @@ namespace ErrorCodes PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( const String & replication_identifier, - const String & remote_database_name_, + const String & postgres_database_, const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, ContextPtr context_, @@ -37,13 +37,15 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( : log(&Poco::Logger::get("PostgreSQLReplicationHandler")) , context(context_) , is_attach(is_attach_) - , remote_database_name(remote_database_name_) + , postgres_database(postgres_database_) + , postgres_schema(replication_settings.materialized_postgresql_schema) , current_database_name(current_database_name_) , connection_info(connection_info_) , max_block_size(replication_settings.materialized_postgresql_max_block_size) , allow_automatic_update(replication_settings.materialized_postgresql_allow_automatic_update) , is_materialized_postgresql_database(is_materialized_postgresql_database_) , tables_list(replication_settings.materialized_postgresql_tables_list) + , schema_can_be_in_tables_list(replication_settings.materialized_postgresql_tables_list_with_schema) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) , connection(std::make_shared(connection_info_)) , milliseconds_to_wait(RESCHEDULE_MS) @@ -73,6 +75,27 @@ void PostgreSQLReplicationHandler::startup() } +String PostgreSQLReplicationHandler::doubleQuoteWithPossibleSchema(const String & table_name) const +{ + if (table_name.starts_with("\"")) + return table_name; + + if (auto pos = table_name.find('.'); schema_can_be_in_tables_list && !tables_list.empty() && pos != std::string::npos) + { + schema_as_a_part_of_table_name = true; + + auto schema = table_name.substr(0, pos); + auto table = table_name.substr(pos + 1); + return doubleQuoteString(schema) + '.' + doubleQuoteString(table); + } + + if (postgres_schema.empty()) + return doubleQuoteString(table_name); + + return doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(table_name); +} + + void PostgreSQLReplicationHandler::waitConnectionAndStart() { try @@ -148,7 +171,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } catch (Exception & e) { - e.addMessage("while loading table {}.{}", remote_database_name, table_name); + e.addMessage("while loading table {}.{}", postgres_database, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); /// Throw in case of single MaterializedPostgreSQL storage, because initial setup is done immediately @@ -191,7 +214,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } catch (Exception & e) { - e.addMessage("while loading table {}.{}", remote_database_name, table_name); + e.addMessage("while loading table {}.{}", postgres_database, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); if (throw_on_error) @@ -214,6 +237,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) publication_name, start_lsn, max_block_size, + schema_as_a_part_of_table_name, allow_automatic_update, nested_storages); @@ -234,7 +258,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. - query_str = fmt::format("SELECT * FROM {}", doubleQuoteString(table_name)); + query_str = fmt::format("SELECT * FROM {}", doubleQuoteWithPossibleSchema(table_name)); materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); auto nested_storage = materialized_storage->getNested(); @@ -339,12 +363,17 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx) { if (tables_list.empty()) { + if (materialized_storages.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No tables to replicate"); + + WriteBufferFromOwnString buf; for (const auto & storage_data : materialized_storages) { - if (!tables_list.empty()) - tables_list += ", "; - tables_list += doubleQuoteString(storage_data.first); + buf << doubleQuoteWithPossibleSchema(storage_data.first); + buf << ","; } + tables_list = buf.str(); + tables_list.resize(tables_list.size() - 1); } if (tables_list.empty()) @@ -565,17 +594,44 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables(postgres::Connection & } tx.commit(); + + /// `schema1.table1, schema2.table2, ...` -> `"schema1"."table1", "schema2"."table2", ...` + /// or + /// `table1, table2, ...` + setting `schema` -> `"schema"."table1", "schema"."table2", ...` + if (!tables_list.empty()) + { + Strings tables_names; + splitInto<','>(tables_names, tables_list); + if (tables_names.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty list of tables"); + + WriteBufferFromOwnString buf; + for (auto & table_name : tables_names) + { + boost::trim(table_name); + buf << doubleQuoteWithPossibleSchema(table_name); + buf << ","; + } + tables_list = buf.str(); + tables_list.resize(tables_list.size() - 1); + } + /// Also we make sure that queries in postgres always use quoted version "table_schema"."table_name". + /// But tables in ClickHouse in case of multi-schame database are never double-quoted. + /// It is ok, because they are accessed with backticks: postgres_database.`table_schema.table_name`. + /// We do quote tables_list table AFTER collected expected_tables, because expected_tables are future clickhouse tables. + return result_tables; } NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx) { - std::string query = fmt::format("SELECT tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); + std::string query = fmt::format("SELECT schemaname, tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); std::unordered_set tables; + String schema, table; - for (auto table_name : tx.stream(query)) - tables.insert(std::get<0>(table_name)); + for (const auto & [schema, table] : tx.stream(query)) + tables.insert(schema.empty() ? table : schema + '.' + table); return tables; } @@ -587,7 +643,7 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( if (!is_materialized_postgresql_database) return nullptr; - return std::make_unique(fetchPostgreSQLTableStructure(tx, table_name, true, true, true)); + return std::make_unique(fetchPostgreSQLTableStructure(tx, doubleQuoteWithPossibleSchema(table_name), true, true, true)); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index eacf6b69b3b..e5eb60a471c 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -21,7 +21,7 @@ class PostgreSQLReplicationHandler public: PostgreSQLReplicationHandler( const String & replication_identifier, - const String & remote_database_name_, + const String & postgres_database_, const String & current_database_name_, const postgres::ConnectionInfo & connection_info_, ContextPtr context_, @@ -80,6 +80,8 @@ private: PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name) const; + String doubleQuoteWithPossibleSchema(const String & table_name) const; + Poco::Logger * log; ContextPtr context; @@ -89,7 +91,9 @@ private: /// If new publication is created at start up - always drop replication slot if it exists. bool new_publication = false; - const String remote_database_name, current_database_name; + String postgres_database; + String postgres_schema; + String current_database_name; /// Connection string and address for logs. postgres::ConnectionInfo connection_info; @@ -97,6 +101,10 @@ private: /// max_block_size for replication stream. const size_t max_block_size; + /// Schema can be as a part of table name, i.e. as a clickhouse table it is accessed like db.`schema.table`. + /// This is possible to allow replicating tables from multiple schemas in the same MaterializedPostgreSQL database engine. + mutable bool schema_as_a_part_of_table_name = false; + /// Table structure changes are always tracked. By default, table with changed schema will get into a skip list. /// This setting allows to reloas table in the background. bool allow_automatic_update = false; @@ -106,6 +114,7 @@ private: /// A coma-separated list of tables, which are going to be replicated for database engine. By default, a whole database is replicated. String tables_list; + bool schema_can_be_in_tables_list; bool user_managed_slot = true; String user_provided_snapshot; @@ -126,8 +135,6 @@ private: MaterializedStorages materialized_storages; UInt64 milliseconds_to_wait; - - String postgres_schema; }; } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index becb4f6ba10..e879a7df891 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -24,6 +24,11 @@ namespace DB { +/** TODO list: + * - Actually I think we can support ddl even though logical replication does not fully support it. + * But some basic ddl like adding/dropping columns, changing column type, column names -- is managable. + */ + /** Case of single MaterializedPostgreSQL table engine. * * A user creates a table with engine MaterializedPostgreSQL. Order by expression must be specified (needed for diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 1dd096087ff..32d9c15e4db 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -30,6 +30,10 @@ postgres_table_template_3 = """ CREATE TABLE IF NOT EXISTS "{}" ( key1 Integer NOT NULL, value1 Integer, key2 Integer NOT NULL, value2 Integer NOT NULL) """ +postgres_table_template_4 = """ + CREATE TABLE IF NOT EXISTS "{}"."{}" ( + key Integer NOT NULL, value Integer, PRIMARY KEY(key)) + """ def get_postgres_conn(ip, port, database=False, auto_commit=True, database_name='postgres_database', replication=False): if database == True: @@ -65,13 +69,19 @@ def create_postgres_db(cursor, name='postgres_database'): def drop_postgres_db(cursor, name='postgres_database'): cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) -def create_clickhouse_postgres_db(ip, port, name='postgres_database'): - instance.query(''' - CREATE DATABASE {} - ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')'''.format(name, ip, port, name)) +def create_clickhouse_postgres_db(ip, port, name='postgres_database', database_name='postgres_database', schema_name=''): + drop_clickhouse_postgres_db(name) + if len(schema_name) == 0: + instance.query(''' + CREATE DATABASE {} + ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword')'''.format(name, ip, port, database_name)) + else: + instance.query(''' + CREATE DATABASE {} + ENGINE = PostgreSQL('{}:{}', '{}', 'postgres', 'mysecretpassword', '{}')'''.format(name, ip, port, database_name, schema_name)) def drop_clickhouse_postgres_db(name='postgres_database'): - instance.query('DROP DATABASE {}'.format(name)) + instance.query('DROP DATABASE IF EXISTS {}'.format(name)) def create_materialized_db(ip, port, materialized_database='test_database', @@ -94,12 +104,19 @@ def drop_materialized_db(materialized_database='test_database'): def drop_postgres_table(cursor, table_name): cursor.execute("""DROP TABLE IF EXISTS "{}" """.format(table_name)) +def drop_postgres_table_with_schema(cursor, schema_name, table_name): + cursor.execute("""DROP TABLE IF EXISTS "{}"."{}" """.format(schema_name, table_name)) + def create_postgres_table(cursor, table_name, replica_identity_full=False, template=postgres_table_template): drop_postgres_table(cursor, table_name) cursor.execute(template.format(table_name)) if replica_identity_full: cursor.execute('ALTER TABLE {} REPLICA IDENTITY FULL;'.format(table_name)) +def create_postgres_table_with_schema(cursor, schema_name, table_name): + drop_postgres_table_with_schema(cursor, schema_name, table_name) + cursor.execute(postgres_table_template_4.format(schema_name, table_name)) + queries = [ 'INSERT INTO postgresql_replica_{} select i, i from generate_series(0, 10000) as t(i);', 'DELETE FROM postgresql_replica_{} WHERE (value*value) % 3 = 0;', @@ -123,24 +140,36 @@ queries = [ ] -def assert_nested_table_is_created(table_name, materialized_database='test_database'): +def assert_nested_table_is_created(table_name, materialized_database='test_database', schema_name=''): + if len(schema_name) == 0: + table = table_name + else: + table = schema_name + "." + table_name + print('Checking table exists:', table) database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database)) - while table_name not in database_tables: + while table not in database_tables: time.sleep(0.2) database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database)) - assert(table_name in database_tables) + assert(table in database_tables) @pytest.mark.timeout(320) -def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'): - assert_nested_table_is_created(table_name, materialized_database) +def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database', schema_name=''): + assert_nested_table_is_created(table_name, materialized_database, schema_name) + print("Checking table is synchronized:", table_name) expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) - result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) + if len(schema_name) == 0: + result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) + else: + result = instance.query('select * from {}.`{}.{}` order by {};'.format(materialized_database, schema_name, table_name, order_by)) while result != expected: time.sleep(0.5) - result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) + if len(schema_name) == 0: + result = instance.query('select * from {}.{} order by {};'.format(materialized_database, table_name, order_by)) + else: + result = instance.query('select * from {}.`{}.{}` order by {};'.format(materialized_database, schema_name, table_name, order_by)) assert(result == expected) @@ -984,6 +1013,147 @@ def test_user_managed_slots(started_cluster): drop_replication_slot(replication_connection, slot_name) +def test_schema_1(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) + cursor = conn.cursor() + + schema_name = 'test_schema' + cursor.execute('DROP SCHEMA IF EXISTS {} CASCADE'.format(schema_name)) + cursor.execute('CREATE SCHEMA {}'.format(schema_name)) + + clickhouse_postgres_db = 'postgres_database_with_schema' + create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, + name=clickhouse_postgres_db, schema_name=schema_name) + + NUM_TABLES=5 + publication_tables = '' + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table_with_schema(cursor, schema_name, table_name); + instance.query("INSERT INTO {}.{} SELECT number, number from numbers(1000)".format(clickhouse_postgres_db, table_name)) + + if publication_tables != '': + publication_tables += ', ' + publication_tables += schema_name + '.' + table_name + + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=["materialized_postgresql_tables_list = '{}'".format(publication_tables)]) + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + instance.query("INSERT INTO {}.{} SELECT number, number from numbers(1000, 1000)".format(clickhouse_postgres_db, table_name)) + + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + + result = instance.query('SHOW TABLES FROM test_database') + assert(result == "test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") + print('Ok') + + instance.restart_clickhouse() + + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + + result = instance.query('SHOW TABLES FROM test_database') + assert(result == "test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") + print('Ok') + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + instance.query("INSERT INTO {}.{} SELECT number, number from numbers(2000, 1000)".format(clickhouse_postgres_db, table_name)) + + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + print('Ok') + + altered_table = random.randint(0, NUM_TABLES-1) + cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + + table_name = 'postgresql_replica_{}'.format(altered_table) + instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) + check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + print('Ok') + + drop_materialized_db() + + +def test_schema_2(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + database=True) + cursor = conn.cursor() + + schema_name = 'test_schema' + cursor.execute('DROP SCHEMA IF EXISTS {} CASCADE'.format(schema_name)) + cursor.execute('CREATE SCHEMA {}'.format(schema_name)) + + clickhouse_postgres_db = 'postgres_database_with_schema' + create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, + name=clickhouse_postgres_db, schema_name=schema_name) + + NUM_TABLES=5 + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table_with_schema(cursor, schema_name, table_name); + instance.query("INSERT INTO {}.{} SELECT number, number from numbers(1000)".format(clickhouse_postgres_db, table_name)) + + create_materialized_db(ip=started_cluster.postgres_ip, + port=started_cluster.postgres_port, + settings=["materialized_postgresql_schema = '{}'".format(schema_name), + "materialized_postgresql_allow_automatic_update = 1"]) + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + instance.query("INSERT INTO {}.{} SELECT number, number from numbers(3000, 1000)".format(clickhouse_postgres_db, table_name)) + + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); + + result = instance.query('SHOW TABLES FROM test_database') + assert(result == "postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") + print('Ok') + + instance.restart_clickhouse() + + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); + + result = instance.query('SHOW TABLES FROM test_database') + assert(result == "postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") + print('Ok') + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + instance.query("INSERT INTO {}.{} SELECT number, number from numbers(4000, 1000)".format(clickhouse_postgres_db, table_name)) + + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); + print('Ok') + + altered_table = random.randint(0, NUM_TABLES-1) + cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); + + table_name = 'postgresql_replica_{}'.format(altered_table) + instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) + check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); + print('Ok') + + drop_materialized_db() + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") From 7295d3660744cdcde1c1aecdb846fa7714fcd7c5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 12 Sep 2021 21:45:43 +0000 Subject: [PATCH 107/609] Save --- .../MaterializedPostgreSQLConsumer.cpp | 4 ++- .../PostgreSQLReplicationHandler.cpp | 29 ++++++++++--------- .../test.py | 3 +- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 7126f126cc0..956d7a301fa 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -379,6 +379,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl else table_name = relation_name; + LOG_TRACE(log, "GOT MESSAGE FOR TABLE: {}", table_name); if (!isSyncAllowed(relation_id)) return; @@ -391,7 +392,7 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl return; } - assert(buffers.count(table_name)); + assert(buffers.contains(table_name)); /// 'd' - default (primary key if any) @@ -566,6 +567,7 @@ void MaterializedPostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const /// Empty lsn string means - continue waiting for valid lsn. skip_list.insert({relation_id, ""}); + LOG_TRACE(log, "trying to mark table as skipped: {}", relation_name); if (storages.count(relation_name)) { /// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index c91600266e2..3db28f4024f 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -148,7 +148,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) auto initial_sync = [&]() { - LOG_TRACE(log, "Starting tables sync load"); + LOG_DEBUG(log, "Starting tables sync load"); if (user_managed_slot) { @@ -221,7 +221,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) throw; } } - LOG_TRACE(log, "Loaded {} tables", nested_storages.size()); + LOG_DEBUG(log, "Loaded {} tables", nested_storages.size()); } tx.commit(); @@ -288,7 +288,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); - LOG_TRACE(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); + LOG_DEBUG(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); return nested_storage; } @@ -300,6 +300,7 @@ void PostgreSQLReplicationHandler::consumerFunc() bool schedule_now = consumer->consume(skipped_tables); + LOG_DEBUG(log, "checking for skipped tables: {}", skipped_tables.size()); if (!skipped_tables.empty()) { try @@ -314,7 +315,7 @@ void PostgreSQLReplicationHandler::consumerFunc() if (stop_synchronization) { - LOG_TRACE(log, "Replication thread is stopped"); + LOG_DEBUG(log, "Replication thread is stopped"); return; } @@ -331,7 +332,7 @@ void PostgreSQLReplicationHandler::consumerFunc() if (milliseconds_to_wait < BACKOFF_TRESHOLD_MS) milliseconds_to_wait *= 2; - LOG_TRACE(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); + LOG_DEBUG(log, "Scheduling replication thread: after {} ms", milliseconds_to_wait); } } @@ -384,7 +385,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx) try { tx.exec(query_str); - LOG_TRACE(log, "Created publication {} with tables list: {}", publication_name, tables_list); + LOG_DEBUG(log, "Created publication {} with tables list: {}", publication_name, tables_list); new_publication = true; } catch (Exception & e) @@ -395,7 +396,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::work & tx) } else { - LOG_TRACE(log, "Using existing publication ({}) version", publication_name); + LOG_DEBUG(log, "Using existing publication ({}) version", publication_name); } } @@ -417,7 +418,7 @@ bool PostgreSQLReplicationHandler::isReplicationSlotExist(pqxx::nontransaction & start_lsn = result[0][2].as(); - LOG_TRACE(log, "Replication slot {} already exists (active: {}). Restart lsn position: {}, confirmed flush lsn: {}", + LOG_DEBUG(log, "Replication slot {} already exists (active: {}). Restart lsn position: {}, confirmed flush lsn: {}", slot_name, result[0][0].as(), result[0][1].as(), start_lsn); return true; @@ -442,7 +443,7 @@ void PostgreSQLReplicationHandler::createReplicationSlot( pqxx::result result{tx.exec(query_str)}; start_lsn = result[0][1].as(); snapshot_name = result[0][2].as(); - LOG_TRACE(log, "Created replication slot: {}, start lsn: {}", replication_slot, start_lsn); + LOG_DEBUG(log, "Created replication slot: {}, start lsn: {}", replication_slot, start_lsn); } catch (Exception & e) { @@ -465,7 +466,7 @@ void PostgreSQLReplicationHandler::dropReplicationSlot(pqxx::nontransaction & tx std::string query_str = fmt::format("SELECT pg_drop_replication_slot('{}')", slot_name); tx.exec(query_str); - LOG_TRACE(log, "Dropped replication slot: {}", slot_name); + LOG_DEBUG(log, "Dropped replication slot: {}", slot_name); } @@ -473,7 +474,7 @@ void PostgreSQLReplicationHandler::dropPublication(pqxx::nontransaction & tx) { std::string query_str = fmt::format("DROP PUBLICATION IF EXISTS {}", publication_name); tx.exec(query_str); - LOG_TRACE(log, "Dropped publication: {}", publication_name); + LOG_DEBUG(log, "Dropped publication: {}", publication_name); } @@ -680,8 +681,8 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorgetNestedStorageID(); auto temp_table_id = temp_nested_storage->getStorageID(); - LOG_TRACE(log, "Starting background update of table {} with table {}", - table_id.getNameForLogs(), temp_table_id.getNameForLogs()); + LOG_DEBUG(log, "Starting background update of table {} ({} with {})", + table_name, table_id.getNameForLogs(), temp_table_id.getNameForLogs()); auto ast_rename = std::make_shared(); ASTRenameQuery::Element elem @@ -710,7 +711,7 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorgetInMemoryMetadataPtr(); auto nested_sample_block = nested_storage_metadata->getSampleBlock(); - LOG_TRACE(log, "Updated table {}. New structure: {}", + LOG_DEBUG(log, "Updated table {}. New structure: {}", nested_table_id.getNameForLogs(), nested_sample_block.dumpStructure()); auto materialized_storage_metadata = nested_storage->getInMemoryMetadataPtr(); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 32d9c15e4db..25de88a9268 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -1041,7 +1041,8 @@ def test_schema_1(started_cluster): create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, - settings=["materialized_postgresql_tables_list = '{}'".format(publication_tables)]) + settings=["materialized_postgresql_tables_list = '{}'".format(publication_tables), + "materialized_postgresql_allow_automatic_update = 1"]) for i in range(NUM_TABLES): table_name = 'postgresql_replica_{}'.format(i) From 3354bca55396bf9b03595332c2b3f9579d714ac3 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Thu, 30 Sep 2021 16:51:15 +0300 Subject: [PATCH 108/609] Make code exactly similar in the non-fuzzer mode --- programs/local/LocalServer.cpp | 426 +++++++++++++++++++++++++-------- programs/local/LocalServer.h | 1 + 2 files changed, 328 insertions(+), 99 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b0b1ac45630..7d7f0d26eaa 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -192,6 +192,7 @@ static DatabasePtr createMemoryDatabaseIfNotExists(ContextPtr context, const Str return system_database; } +#ifndef FUZZING_MODE int LocalServer::main(const std::vector & /*args*/) try { @@ -212,19 +213,14 @@ try throw Exception("Specify either `query` or `queries-file` option", ErrorCodes::BAD_ARGUMENTS); } - std::optional status; - -#ifdef FUZZING_MODE - static bool first_time = true; - if (first_time) - { -#endif shared_context = Context::createShared(); global_context = Context::createGlobal(shared_context.get()); global_context->makeGlobalContext(); global_context->setApplicationType(Context::ApplicationType::LOCAL); tryInitPath(); + std::optional status; + /// Skip temp path installation /// We will terminate process on error @@ -321,19 +317,62 @@ try attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); } -#ifdef FUZZING_MODE + + processQueries(); + + global_context->shutdown(); + global_context.reset(); + + status.reset(); + cleanup(); + + return Application::EXIT_OK; +} +catch (const Exception & e) +{ + try + { + cleanup(); } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; + + /// If exception code isn't zero, we should return non-zero return code anyway. + return e.code() ? e.code() : -1; +} #endif - /// processing queries +std::string LocalServer::getInitialCreateTableQuery() +{ + if (!config().has("table-structure")) + return {}; + auto table_name = backQuoteIfNeed(config().getString("table-name", "table")); + auto table_structure = config().getString("table-structure"); + auto data_format = backQuoteIfNeed(config().getString("table-data-format", "TSV")); + String table_file; + if (!config().has("table-file") || config().getString("table-file") == "-") /// Use Unix tools stdin naming convention + table_file = "stdin"; + else /// Use regular file + table_file = quoteString(config().getString("table-file")); + + return + "CREATE TABLE " + table_name + + " (" + table_structure + ") " + + "ENGINE = " + "File(" + data_format + ", " + table_file + ")" + "; "; +} + +void LocalServer::processQueries() +{ String initial_create_query = getInitialCreateTableQuery(); String queries_str = initial_create_query; -#ifdef FUZZING_MODE - if (first_time) - { -#endif if (config().has("query")) queries_str += config().getRawString("query"); else @@ -343,45 +382,14 @@ try readStringUntilEOF(queries_from_file, in); queries_str += queries_from_file; } -#ifdef FUZZING_MODE - } -#endif const auto & settings = global_context->getSettingsRef(); -#ifdef FUZZING_MODE - static std::vector queries; - if (first_time) - { -#else std::vector queries; -#endif - std::pair parse_res; -#ifdef FUZZING_MODE - try - { -#endif - parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); -#ifdef FUZZING_MODE - } - catch (const Exception &) - { - // will be caught at the end of the main - throw; - } - catch (...) - { - std::cerr << "Undefined error while parsing" << std::endl; - exit(1); - } -#endif + auto parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); if (!parse_res.second) throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); -#ifdef FUZZING_MODE - } - first_time = false; -#endif /// Authenticate and create a context to execute queries. Session session{global_context, ClientInfo::Interface::LOCAL}; @@ -399,16 +407,17 @@ try /// Set progress show need_render_progress = config().getBool("progress", false); + std::function finalize_progress; if (need_render_progress) { /// Set progress callback, which can be run from multiple threads. context->setProgressCallback([&](const Progress & value) - { - /// Write progress only if progress was updated - if (progress_indication.updateProgress(value)) - progress_indication.writeProgress(); - }); + { + /// Write progress only if progress was updated + if (progress_indication.updateProgress(value)) + progress_indication.writeProgress(); + }); /// Set finalizing callback for progress, which is called right before finalizing query output. finalize_progress = [&]() @@ -431,6 +440,7 @@ try ReadBufferFromString read_buf(query); WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO); + if (echo_queries) { writeString(query, write_buf); @@ -456,55 +466,6 @@ try if (exception) std::rethrow_exception(exception); - -#ifndef FUZZING_MODE - global_context->shutdown(); - global_context.reset(); - - status.reset(); - cleanup(); -#endif - - return Application::EXIT_OK; -} -catch (const Exception & e) -{ - try - { - cleanup(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; - - /// If exception code isn't zero, we should return non-zero return code anyway. - return e.code() ? e.code() : -1; -} - - -std::string LocalServer::getInitialCreateTableQuery() -{ - if (!config().has("table-structure")) - return {}; - - auto table_name = backQuoteIfNeed(config().getString("table-name", "table")); - auto table_structure = config().getString("table-structure"); - auto data_format = backQuoteIfNeed(config().getString("table-data-format", "TSV")); - String table_file; - if (!config().has("table-file") || config().getString("table-file") == "-") /// Use Unix tools stdin naming convention - table_file = "stdin"; - else /// Use regular file - table_file = quoteString(config().getString("table-file")); - - return - "CREATE TABLE " + table_name + - " (" + table_structure + ") " + - "ENGINE = " - "File(" + data_format + ", " + table_file + ")" - "; "; } static const char * minimal_default_user_xml = @@ -718,6 +679,273 @@ void LocalServer::applyCmdOptions(ContextMutablePtr context) #pragma GCC diagnostic ignored "-Wmissing-declarations" #ifdef FUZZING_MODE +/// This main will not lead to a crash after reuse +int DB::LocalServer::main(const std::vector & /*args*/) +try +{ + Poco::Logger * log = &logger(); + ThreadStatus thread_status; + UseSSL use_ssl; + + if (!config().has("query") && !config().has("table-structure") && !config().has("queries-file")) /// Nothing to process + { + if (config().hasOption("verbose")) + std::cerr << "There are no queries to process." << '\n'; + + return Application::EXIT_OK; + } + + if (config().has("query") && config().has("queries-file")) + { + throw Exception("Specify either `query` or `queries-file` option", ErrorCodes::BAD_ARGUMENTS); + } + + std::optional status; + + static bool first_time = true; + if (first_time) + { + shared_context = Context::createShared(); + global_context = Context::createGlobal(shared_context.get()); + global_context->makeGlobalContext(); + global_context->setApplicationType(Context::ApplicationType::LOCAL); + tryInitPath(); + + /// Skip temp path installation + + /// We will terminate process on error + static KillingErrorHandler error_handler; + Poco::ErrorHandler::set(&error_handler); + + /// Don't initialize DateLUT + + registerFunctions(); + registerAggregateFunctions(); + registerTableFunctions(); + registerStorages(); + registerDictionaries(); + registerDisks(); + registerFormats(); + + /// Maybe useless + if (config().has("macros")) + global_context->setMacros(std::make_unique(config(), "macros", log)); + + /// Skip networking + + /// Sets external authenticators config (LDAP, Kerberos). + global_context->setExternalAuthenticatorsConfig(config()); + + global_context->initializeBackgroundExecutors(); + + setupUsers(); + + /// Limit on total number of concurrently executing queries. + /// There is no need for concurrent queries, override max_concurrent_queries. + global_context->getProcessList().setMaxSize(0); + + /// Size of cache for uncompressed blocks. Zero means disabled. + size_t uncompressed_cache_size = config().getUInt64("uncompressed_cache_size", 0); + if (uncompressed_cache_size) + global_context->setUncompressedCache(uncompressed_cache_size); + + /// Size of cache for marks (index of MergeTree family of tables). It is necessary. + /// Specify default value for mark_cache_size explicitly! + size_t mark_cache_size = config().getUInt64("mark_cache_size", 5368709120); + if (mark_cache_size) + global_context->setMarkCache(mark_cache_size); + + /// A cache for mmapped files. + size_t mmap_cache_size = config().getUInt64("mmap_cache_size", 1000); /// The choice of default is arbitrary. + if (mmap_cache_size) + global_context->setMMappedFileCache(mmap_cache_size); + + /// Load global settings from default_profile and system_profile. + global_context->setDefaultProfiles(config()); + + /// We load temporary database first, because projections need it. + DatabaseCatalog::instance().initializeAndLoadTemporaryDatabase(); + + /** Init dummy default DB + * NOTE: We force using isolated default database to avoid conflicts with default database from server environment + * Otherwise, metadata of temporary File(format, EXPLICIT_PATH) tables will pollute metadata/ directory; + * if such tables will not be dropped, clickhouse-server will not be able to load them due to security reasons. + */ + std::string default_database = config().getString("default_database", "_local"); + DatabaseCatalog::instance().attachDatabase(default_database, std::make_shared(default_database, global_context)); + global_context->setCurrentDatabase(default_database); + applyCmdOptions(global_context); + + if (config().has("path")) + { + String path = global_context->getPath(); + + /// Lock path directory before read + status.emplace(path + "status", StatusFile::write_full_info); + + fs::create_directories(fs::path(path) / "user_defined/"); + LOG_DEBUG(log, "Loading user defined objects from {}", path); + Poco::File(path + "user_defined/").createDirectories(); + UserDefinedSQLObjectsLoader::instance().loadObjects(global_context); + LOG_DEBUG(log, "Loaded user defined objects."); + + LOG_DEBUG(log, "Loading metadata from {}", path); + fs::create_directories(fs::path(path) / "data/"); + fs::create_directories(fs::path(path) / "metadata/"); + loadMetadataSystem(global_context); + attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); + attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); + attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); + loadMetadata(global_context); + startupSystemTables(); + DatabaseCatalog::instance().loadDatabases(); + LOG_DEBUG(log, "Loaded metadata."); + } + else if (!config().has("no-system-tables")) + { + attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); + attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); + attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); + } + } + + /// processing queries + + String initial_create_query = getInitialCreateTableQuery(); + String queries_str = initial_create_query; + + if (first_time) + { + if (config().has("query")) + queries_str += config().getRawString("query"); + else + { + String queries_from_file; + ReadBufferFromFile in(config().getString("queries-file")); + readStringUntilEOF(queries_from_file, in); + queries_str += queries_from_file; + } + } + + const auto & settings = global_context->getSettingsRef(); + + static std::vector queries; + if (first_time) + { + std::pair parse_res; + try + { + parse_res = splitMultipartQuery(queries_str, queries, settings.max_query_size, settings.max_parser_depth); + } + catch (const Exception &) + { + // will be caught at the end of the main + throw; + } + catch (...) + { + std::cerr << "Undefined error while parsing" << std::endl; + exit(1); + } + + if (!parse_res.second) + throw Exception("Cannot parse and execute the following part of query: " + String(parse_res.first), ErrorCodes::SYNTAX_ERROR); + } + first_time = false; + + /// Authenticate and create a context to execute queries. + Session session{global_context, ClientInfo::Interface::LOCAL}; + session.authenticate("default", "", {}); + + /// Use the same context for all queries. + auto context = session.makeQueryContext(); + context->makeSessionContext(); /// initial_create_query requires a session context to be set. + context->setCurrentQueryId(""); + + applyCmdSettings(context); + + /// Use the same query_id (and thread group) for all queries + CurrentThread::QueryScope query_scope_holder(context); + + /// Set progress show + need_render_progress = config().getBool("progress", false); + std::function finalize_progress; + if (need_render_progress) + { + /// Set progress callback, which can be run from multiple threads. + context->setProgressCallback([&](const Progress & value) + { + /// Write progress only if progress was updated + if (progress_indication.updateProgress(value)) + progress_indication.writeProgress(); + }); + + /// Set finalizing callback for progress, which is called right before finalizing query output. + finalize_progress = [&]() + { + progress_indication.clearProgressOutput(); + }; + + /// Set callback for file processing progress. + progress_indication.setFileProgressCallback(context); + } + + bool echo_queries = config().hasOption("echo") || config().hasOption("verbose"); + + std::exception_ptr exception; + + for (const auto & query : queries) + { + written_first_block = false; + progress_indication.resetProgress(); + + ReadBufferFromString read_buf(query); + WriteBufferFromFileDescriptor write_buf(STDOUT_FILENO); + if (echo_queries) + { + writeString(query, write_buf); + writeChar('\n', write_buf); + write_buf.next(); + } + + try + { + executeQuery(read_buf, write_buf, /* allow_into_outfile = */ true, context, {}, {}, finalize_progress); + } + catch (...) + { + if (!config().hasOption("ignore-error")) + throw; + + if (!exception) + exception = std::current_exception(); + + std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; + } + } + + if (exception) + std::rethrow_exception(exception); + + return Application::EXIT_OK; +} +catch (const Exception & e) +{ + try + { + cleanup(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + std::cerr << getCurrentExceptionMessage(config().hasOption("stacktrace")) << '\n'; + + /// If exception code isn't zero, we should return non-zero return code anyway. + return e.code() ? e.code() : -1; +} + #include std::optional fuzz_app; diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index 4da8142a6ff..e82caad7542 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -38,6 +38,7 @@ private: void tryInitPath(); void applyCmdOptions(ContextMutablePtr context); void applyCmdSettings(ContextMutablePtr context); + void processQueries(); void setupUsers(); void cleanup(); From f5b1e1daaae2931b181f1e89658e3bc00240d8fd Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Thu, 30 Sep 2021 16:53:53 +0300 Subject: [PATCH 109/609] Spaces --- programs/local/LocalServer.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 7d7f0d26eaa..6eeed5160c4 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -413,11 +413,11 @@ void LocalServer::processQueries() { /// Set progress callback, which can be run from multiple threads. context->setProgressCallback([&](const Progress & value) - { - /// Write progress only if progress was updated - if (progress_indication.updateProgress(value)) - progress_indication.writeProgress(); - }); + { + /// Write progress only if progress was updated + if (progress_indication.updateProgress(value)) + progress_indication.writeProgress(); + }); /// Set finalizing callback for progress, which is called right before finalizing query output. finalize_progress = [&]() From c601db109f1e7338c533cb93367adc3f6881ab12 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 1 Oct 2021 15:54:01 +0000 Subject: [PATCH 110/609] Correct nerge --- .../materialized-postgresql.md | 15 ++++++---- .../MaterializedPostgreSQLConsumer.h | 2 ++ .../PostgreSQLReplicationHandler.cpp | 1 - .../test.py | 28 +++++++++---------- 4 files changed, 25 insertions(+), 21 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 69a17144c6c..8f623568765 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -59,6 +59,15 @@ SETTINGS materialized_postgresql_tables_list = 'table1,table2,table3'; SELECT * FROM database1.table1; ``` +It is also possible to change settings at run time. + +``` sql +ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = ; +``` + + +## PostgreSQL schema {#schema} + PostgreSQL [schema](https://www.postgresql.org/docs/9.1/ddl-schemas.html) can be used in two ways. 1. One schema for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_schema`. @@ -87,12 +96,6 @@ SELECT * FROM database1.`schema2.table2`; But in this case all tables in `materialized_postgresql_tables_list` must be written with its schema name. -It is also possible to change settings at run time. - -``` sql -ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_block_size = ; -``` - ## Requirements {#requirements} 1. The [wal_level](https://www.postgresql.org/docs/current/runtime-config-wal.html) setting must have a value `logical` and `max_replication_slots` parameter must have a value at least `2` in the PostgreSQL config file. diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h index 8f924f4736a..88bc9919b49 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.h @@ -115,7 +115,9 @@ private: UInt64 lsn_value; size_t max_block_size; + bool schema_as_a_part_of_table_name; + bool allow_automatic_update; String table_to_insert; diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 1fbfe26b62a..ff401555142 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -674,7 +674,6 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx { std::string query = fmt::format("SELECT schemaname, tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); std::unordered_set tables; - String schema, table; for (const auto & [schema, table] : tx.stream(query)) tables.insert(schema.empty() ? table : schema + '.' + table); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 03a0723113d..9fe5a0c9d88 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -1224,14 +1224,14 @@ def test_schema_1(started_cluster): check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); print('Ok') - altered_table = random.randint(0, NUM_TABLES-1) - cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) - check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + #altered_table = random.randint(0, NUM_TABLES-1) + #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - table_name = 'postgresql_replica_{}'.format(altered_table) - instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) - check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - print('Ok') + #table_name = 'postgresql_replica_{}'.format(altered_table) + #instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) + #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + #print('Ok') drop_materialized_db() @@ -1292,14 +1292,14 @@ def test_schema_2(started_cluster): check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); print('Ok') - altered_table = random.randint(0, NUM_TABLES-1) - cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) - check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); + #altered_table = random.randint(0, NUM_TABLES-1) + #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); - table_name = 'postgresql_replica_{}'.format(altered_table) - instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) - check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); - print('Ok') + #table_name = 'postgresql_replica_{}'.format(altered_table) + #instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) + #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); + #print('Ok') drop_materialized_db() From 28aeb30d4a19e9c392959f64168423243b8148b2 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 2 Oct 2021 11:11:18 +0000 Subject: [PATCH 111/609] Support full schema list --- .../materialized-postgresql.md | 20 +- docs/en/operations/settings/settings.md | 8 + .../DatabaseMaterializedPostgreSQL.cpp | 2 +- .../fetchPostgreSQLTableStructure.cpp | 35 ++- .../MaterializedPostgreSQLSettings.h | 1 + .../PostgreSQLReplicationHandler.cpp | 55 +++- .../PostgreSQL/PostgreSQLReplicationHandler.h | 5 +- .../test.py | 235 +++++++++++------- 8 files changed, 252 insertions(+), 109 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 8f623568765..9bd626cde32 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -43,6 +43,8 @@ DETACH TABLE postgres_database.table_to_remove; - [materialized_postgresql_schema](../../operations/settings/settings.md#materialized-postgresql-schema) +- [materialized_postgresql_schema_list](../../operations/settings/settings.md#materialized-postgresql-schema-list) + - [materialized_postgresql_allow_automatic_update](../../operations/settings/settings.md#materialized-postgresql-allow-automatic-update) - [materialized_postgresql_max_block_size](../../operations/settings/settings.md#materialized-postgresql-max-block-size) @@ -81,7 +83,7 @@ SETTINGS materialized_postgresql_schema = 'postgres_schema'; SELECT * FROM postgres_database.table1; ``` -2. Any number of schemas for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_tables_list`. Each table is written along with its schema. +2. Any number of schemas with specified set of tables for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_tables_list`. Each table is written along with its schema. Tables are accessed via schema name and table name at the same time: ``` sql @@ -95,6 +97,22 @@ SELECT * FROM database1.`schema2.table2`; But in this case all tables in `materialized_postgresql_tables_list` must be written with its schema name. +Warning: for this case dots in table name are not allowed. + +3. Any number of schemas with full set of tables for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_schema_list`. + +``` sql +CREATE DATABASE database1 +ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') +SETTINGS materialized_postgresql_schema_list = 'schema1,schema2,schema3'; + +SELECT * FROM database1.`schema1.table1`; +SELECT * FROM database1.`schema1.table2`; +SELECT * FROM database1.`schema2.table2`; +``` + +Warning: for this case dots in table name are not allowed. + ## Requirements {#requirements} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0e4e9d3b489..df5b3137469 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3540,6 +3540,14 @@ Sets a comma-separated list of PostgreSQL database tables, which will be replica Default value: empty list — means whole PostgreSQL database will be replicated. +## materialized_postgresql_schema {#materialized-postgresql-schema} + +Default value: empty string. (Default schema is used) + +## materialized_postgresql_schema_list {#materialized-postgresql-schema-list} + +Default value: empty list. (Default schema is used) + ## materialized_postgresql_allow_automatic_update {#materialized-postgresql-allow-automatic-update} Allow reloading table in the background, when schema changes are detected. DDL queries on the PostgreSQL side are not replicated via ClickHouse [MaterializedPostgreSQL](../../engines/database-engines/materialized-postgresql.md) engine, because it is not allowed with PostgreSQL logical replication protocol, but the fact of DDL changes is detected transactionally. In this case, the default behaviour is to stop replicating those tables once DDL is detected. However, if this setting is enabled, then, instead of stopping the replication of those tables, they will be reloaded in the background via database snapshot without data losses and replication will continue for them. diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 8fb75473d15..1b82e79400d 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -78,7 +78,7 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() } if (tables_to_replicate.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Got empty list of tables to replicate"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Got empty list of tables to replicate"); for (const auto & table_name : tables_to_replicate) { diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 0495dd8723a..ca91b51be57 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -14,6 +14,8 @@ #include #include #include +#include +#include namespace DB @@ -29,13 +31,34 @@ namespace ErrorCodes template std::unordered_set fetchPostgreSQLTablesList(T & tx, const String & postgres_schema) { - std::unordered_set tables; - std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " - "WHERE schemaname != 'pg_catalog' AND {}", - postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(postgres_schema)); + Names schemas; + boost::split(schemas, postgres_schema, [](char c){ return c == ','; }); + for (String & key : schemas) + boost::trim(key); - for (auto table_name : tx.template stream(query)) - tables.insert(std::get<0>(table_name)); + std::unordered_set tables; + if (schemas.size() <= 1) + { + std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " + "WHERE schemaname != 'pg_catalog' AND {}", + postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(postgres_schema)); + for (auto table_name : tx.template stream(query)) + tables.insert(std::get<0>(table_name)); + + return tables; + } + + /// We add schema to table name only in case of multiple schemas for the whole database engine. + /// Because there is no need to add it if there is only one schema. + /// If we add schema to table name then table can be accessed only this way: database_name.`schema_name.table_name` + for (const auto & schema : schemas) + { + std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " + "WHERE schemaname != 'pg_catalog' AND {}", + postgres_schema.empty() ? "schemaname != 'information_schema'" : "schemaname = " + quoteString(schema)); + for (auto table_name : tx.template stream(query)) + tables.insert(schema + '.' + std::get<0>(table_name)); + } return tables; } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index 7c661f95edf..3f09f752cb0 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -16,6 +16,7 @@ namespace DB #define LIST_OF_MATERIALIZED_POSTGRESQL_SETTINGS(M) \ M(UInt64, materialized_postgresql_max_block_size, 65536, "Number of row collected before flushing data into table.", 0) \ M(String, materialized_postgresql_tables_list, "", "List of tables for MaterializedPostgreSQL database engine", 0) \ + M(String, materialized_postgresql_schema_list, "", "List of schemas for MaterializedPostgreSQL database engine", 0) \ M(Bool, materialized_postgresql_allow_automatic_update, false, "Allow to reload table in the background, when schema changes are detected", 0) \ M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \ M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \ diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index ff401555142..06540582b24 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -46,10 +46,17 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , allow_automatic_update(replication_settings.materialized_postgresql_allow_automatic_update) , is_materialized_postgresql_database(is_materialized_postgresql_database_) , tables_list(replication_settings.materialized_postgresql_tables_list) + , schema_list(replication_settings.materialized_postgresql_schema_list) , schema_can_be_in_tables_list(replication_settings.materialized_postgresql_tables_list_with_schema) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) , milliseconds_to_wait(RESCHEDULE_MS) { + if (!schema_list.empty() && !tables_list.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and tables list at the same time"); + + if (!schema_list.empty() && !postgres_schema.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot have schema list and common schema at the same time"); + replication_slot = replication_settings.materialized_postgresql_replication_slot; if (replication_slot.empty()) { @@ -58,8 +65,11 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( } publication_name = fmt::format("{}_ch_publication", replication_identifier); - startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ waitConnectionAndStart(); }); + startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); + + if (!schema_list.empty()) + schema_as_a_part_of_table_name = true; } @@ -71,6 +81,8 @@ void PostgreSQLReplicationHandler::addStorage(const std::string & table_name, St void PostgreSQLReplicationHandler::startup() { + /// We load tables in a separate thread, because this database is not created yet. + /// (will get "database is currently dropped or renamed") startup_task->activateAndSchedule(); } @@ -80,7 +92,14 @@ String PostgreSQLReplicationHandler::doubleQuoteWithPossibleSchema(const String if (table_name.starts_with("\"")) return table_name; - if (auto pos = table_name.find('.'); schema_can_be_in_tables_list && !tables_list.empty() && pos != std::string::npos) + /// !schema_list.empty() -- We replicate all tables from specifies schemas. + /// In this case when tables list is fetched, we append schema with dot. But without quotes. + + /// If there is a setting `tables_list`, then table names can be put there along with schema, + /// separated by dot and with no quotes. We add double quotes in this case. + bool schema_in_name = (schema_can_be_in_tables_list && !tables_list.empty()) || !schema_list.empty(); + + if (auto pos = table_name.find('.'); schema_in_name && pos != std::string::npos) { schema_as_a_part_of_table_name = true; @@ -96,21 +115,27 @@ String PostgreSQLReplicationHandler::doubleQuoteWithPossibleSchema(const String } -void PostgreSQLReplicationHandler::waitConnectionAndStart() +void PostgreSQLReplicationHandler::checkConnectionAndStart() { try { postgres::Connection connection(connection_info); connection.connect(); /// Will throw pqxx::broken_connection if no connection at the moment - startSynchronization(false); + startSynchronization(is_attach); } catch (const pqxx::broken_connection & pqxx_error) { + if (!is_attach) + throw; + LOG_ERROR(log, "Unable to set up connection. Reconnection attempt will continue. Error message: {}", pqxx_error.what()); startup_task->scheduleAfter(RESCHEDULE_MS); } catch (...) { + if (!is_attach) + throw; + tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -170,7 +195,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) } catch (Exception & e) { - e.addMessage("while loading table {}.{}", postgres_database, table_name); + e.addMessage("while loading table `{}`.`{}`", postgres_database, table_name); tryLogCurrentException(__PRETTY_FUNCTION__); /// Throw in case of single MaterializedPostgreSQL storage, because initial setup is done immediately @@ -269,7 +294,9 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. - query_str = fmt::format("SELECT * FROM {}", doubleQuoteWithPossibleSchema(table_name)); + auto quoted_name = doubleQuoteWithPossibleSchema(table_name); + query_str = fmt::format("SELECT * FROM {}", quoted_name); + LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); materialized_storage->createNestedIfNeeded(fetchTableStructure(*tx, table_name)); auto nested_storage = materialized_storage->getNested(); @@ -566,6 +593,8 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() boost::trim(table_name); } + /// Try to fetch tables list from publication if there is not tables list. + /// If there is a tables list -- check that lists are consistent and if not -- remove publication, it will be recreated. if (publication_exists_before_startup) { if (!is_attach) @@ -586,7 +615,7 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() { pqxx::nontransaction tx(connection.getRef()); - result_tables = fetchPostgreSQLTablesList(tx, postgres_schema); + result_tables = fetchPostgreSQLTablesList(tx, schema_list.empty() ? postgres_schema : schema_list); } } /// Check tables list from publication is the same as expected tables list. @@ -624,20 +653,20 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() if (result_tables.empty()) { - if (!tables_list.empty()) - { - result_tables = NameSet(expected_tables.begin(), expected_tables.end()); - } - else + if (tables_list.empty()) { /// Fetch all tables list from database. Publication does not exist yet, which means /// that no replication took place. Publication will be created in /// startSynchronization method. { pqxx::nontransaction tx(connection.getRef()); - result_tables = fetchPostgreSQLTablesList(tx, postgres_schema); + result_tables = fetchPostgreSQLTablesList(tx, schema_list.empty() ? postgres_schema : schema_list); } } + else + { + result_tables = NameSet(expected_tables.begin(), expected_tables.end()); + } } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index bed7aea9535..dfe72361de5 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -79,7 +79,7 @@ private: /// Methods to manage replication. - void waitConnectionAndStart(); + void checkConnectionAndStart(); void consumerFunc(); @@ -123,6 +123,9 @@ private: /// A coma-separated list of tables, which are going to be replicated for database engine. By default, a whole database is replicated. String tables_list; + + String schema_list; + bool schema_can_be_in_tables_list; bool user_managed_slot = true; diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 9fe5a0c9d88..c06a537d2fa 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -69,6 +69,13 @@ def create_postgres_db(cursor, name='postgres_database'): def drop_postgres_db(cursor, name='postgres_database'): cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) +def drop_postgres_schema(cursor, schema_name): + cursor.execute('DROP SCHEMA IF EXISTS {} CASCADE'.format(schema_name)) + +def create_postgres_schema(cursor, schema_name): + drop_postgres_schema(cursor, schema_name) + cursor.execute('CREATE SCHEMA {}'.format(schema_name)) + def create_clickhouse_postgres_db(ip, port, name='postgres_database', database_name='postgres_database', schema_name=''): drop_clickhouse_postgres_db(name) if len(schema_name) == 0: @@ -1162,67 +1169,60 @@ def test_predefined_connection_configuration(started_cluster): drop_materialized_db() -def test_schema_1(started_cluster): - conn = get_postgres_conn(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True) +insert_counter = 0 + +def test_database_with_multiple_non_default_schemas_1(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() + NUM_TABLES = 5 schema_name = 'test_schema' - cursor.execute('DROP SCHEMA IF EXISTS {} CASCADE'.format(schema_name)) - cursor.execute('CREATE SCHEMA {}'.format(schema_name)) - clickhouse_postgres_db = 'postgres_database_with_schema' - create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, - name=clickhouse_postgres_db, schema_name=schema_name) - - NUM_TABLES=5 publication_tables = '' + insert_counter = 0 + + def insert_into_tables(): + global insert_counter + clickhouse_postgres_db = 'postgres_database_with_schema' + for i in range(NUM_TABLES): + table_name = f'postgresql_replica_{i}' + instance.query(f"INSERT INTO {clickhouse_postgres_db}.{table_name} SELECT number, number from numbers(1000 * {insert_counter}, 1000)") + insert_counter += 1 + + def assert_show_tables(expected): + result = instance.query('SHOW TABLES FROM test_database') + assert(result == expected) + print('assert show tables Ok') + + def check_all_tables_are_synchronized(): + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + print('synchronization Ok') + + create_postgres_schema(cursor, schema_name) + create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, name=clickhouse_postgres_db, schema_name=schema_name) for i in range(NUM_TABLES): table_name = 'postgresql_replica_{}'.format(i) create_postgres_table_with_schema(cursor, schema_name, table_name); - instance.query("INSERT INTO {}.{} SELECT number, number from numbers(1000)".format(clickhouse_postgres_db, table_name)) - if publication_tables != '': publication_tables += ', ' publication_tables += schema_name + '.' + table_name - create_materialized_db(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - settings=["materialized_postgresql_tables_list = '{}'".format(publication_tables), - "materialized_postgresql_allow_automatic_update = 1"]) + insert_into_tables() + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + settings=[f"materialized_postgresql_tables_list = '{publication_tables}'", "materialized_postgresql_allow_automatic_update = 1"]) - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - instance.query("INSERT INTO {}.{} SELECT number, number from numbers(1000, 1000)".format(clickhouse_postgres_db, table_name)) - - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - - result = instance.query('SHOW TABLES FROM test_database') - assert(result == "test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") - print('Ok') + check_all_tables_are_synchronized() + assert_show_tables("test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") instance.restart_clickhouse() + check_all_tables_are_synchronized() + assert_show_tables("test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - - result = instance.query('SHOW TABLES FROM test_database') - assert(result == "test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") - print('Ok') - - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - instance.query("INSERT INTO {}.{} SELECT number, number from numbers(2000, 1000)".format(clickhouse_postgres_db, table_name)) - - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - print('Ok') + insert_into_tables() + check_all_tables_are_synchronized() #altered_table = random.randint(0, NUM_TABLES-1) #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) @@ -1236,61 +1236,122 @@ def test_schema_1(started_cluster): drop_materialized_db() -def test_schema_2(started_cluster): - conn = get_postgres_conn(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True) +def test_database_with_multiple_non_default_schemas_2(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() - schema_name = 'test_schema' - cursor.execute('DROP SCHEMA IF EXISTS {} CASCADE'.format(schema_name)) - cursor.execute('CREATE SCHEMA {}'.format(schema_name)) + NUM_TABLES = 2 + schemas_num = 2 + schema_list = 'schema0, schema1' + insert_counter = 0 - clickhouse_postgres_db = 'postgres_database_with_schema' - create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, - name=clickhouse_postgres_db, schema_name=schema_name) + def check_all_tables_are_synchronized(): + for i in range(schemas_num): + schema_name = f'schema{i}' + clickhouse_postgres_db = f'clickhouse_postgres_db{i}' + for ti in range(NUM_TABLES): + table_name = f'postgresql_replica_{ti}' + print(f'checking table {schema_name}.{table_name}') + check_tables_are_synchronized(f'{table_name}', schema_name=schema_name, postgres_database=clickhouse_postgres_db); + print('synchronized Ok') + + def insert_into_tables(): + global insert_counter + for i in range(schemas_num): + clickhouse_postgres_db = f'clickhouse_postgres_db{i}' + for ti in range(NUM_TABLES): + table_name = f'postgresql_replica_{ti}' + instance.query(f'INSERT INTO {clickhouse_postgres_db}.{table_name} SELECT number, number from numbers(1000 * {insert_counter}, 1000)') + insert_counter += 1 + + def assert_show_tables(expected): + result = instance.query('SHOW TABLES FROM test_database') + assert(result == expected) + print('assert show tables Ok') + + for i in range(schemas_num): + schema_name = f'schema{i}' + clickhouse_postgres_db = f'clickhouse_postgres_db{i}' + create_postgres_schema(cursor, schema_name) + create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, name=clickhouse_postgres_db, schema_name=schema_name) + for ti in range(NUM_TABLES): + table_name = f'postgresql_replica_{ti}' + create_postgres_table_with_schema(cursor, schema_name, table_name); + + insert_into_tables() + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + settings=[f"materialized_postgresql_schema_list = '{schema_list}'", "materialized_postgresql_allow_automatic_update = 1"]) + + check_all_tables_are_synchronized() + insert_into_tables() + assert_show_tables("schema0.postgresql_replica_0\nschema0.postgresql_replica_1\nschema1.postgresql_replica_0\nschema1.postgresql_replica_1\n") + + instance.restart_clickhouse() + assert_show_tables("schema0.postgresql_replica_0\nschema0.postgresql_replica_1\nschema1.postgresql_replica_0\nschema1.postgresql_replica_1\n") + check_all_tables_are_synchronized() + insert_into_tables() + check_all_tables_are_synchronized() + + #altered_table = random.randint(0, NUM_TABLES-1) + #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + + #table_name = 'postgresql_replica_{}'.format(altered_table) + #instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) + #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); + #print('Ok') + + drop_materialized_db() + + +def test_database_with_single_non_default_schema(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) + cursor = conn.cursor() NUM_TABLES=5 + schema_name = 'test_schema' + clickhouse_postgres_db = 'postgres_database_with_schema' + insert_counter = 0 + + def insert_into_tables(): + global insert_counter + clickhouse_postgres_db = 'postgres_database_with_schema' + for i in range(NUM_TABLES): + table_name = f'postgresql_replica_{i}' + instance.query(f"INSERT INTO {clickhouse_postgres_db}.{table_name} SELECT number, number from numbers(1000 * {insert_counter}, 1000)") + insert_counter += 1 + + def assert_show_tables(expected): + result = instance.query('SHOW TABLES FROM test_database') + assert(result == expected) + print('assert show tables Ok') + + def check_all_tables_are_synchronized(): + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); + print('synchronization Ok') + + create_postgres_schema(cursor, schema_name) + create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, name=clickhouse_postgres_db, schema_name=schema_name) + for i in range(NUM_TABLES): table_name = 'postgresql_replica_{}'.format(i) create_postgres_table_with_schema(cursor, schema_name, table_name); - instance.query("INSERT INTO {}.{} SELECT number, number from numbers(1000)".format(clickhouse_postgres_db, table_name)) - create_materialized_db(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - settings=["materialized_postgresql_schema = '{}'".format(schema_name), - "materialized_postgresql_allow_automatic_update = 1"]) + insert_into_tables() + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + settings=[f"materialized_postgresql_schema = '{schema_name}'", "materialized_postgresql_allow_automatic_update = 1"]) - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - instance.query("INSERT INTO {}.{} SELECT number, number from numbers(3000, 1000)".format(clickhouse_postgres_db, table_name)) - - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); - - result = instance.query('SHOW TABLES FROM test_database') - assert(result == "postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") - print('Ok') + insert_into_tables() + check_all_tables_are_synchronized() + assert_show_tables("postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") instance.restart_clickhouse() - - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); - - result = instance.query('SHOW TABLES FROM test_database') - assert(result == "postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") - print('Ok') - - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - instance.query("INSERT INTO {}.{} SELECT number, number from numbers(4000, 1000)".format(clickhouse_postgres_db, table_name)) - - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); - print('Ok') + check_all_tables_are_synchronized() + assert_show_tables("postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") + insert_into_tables() + check_all_tables_are_synchronized() #altered_table = random.randint(0, NUM_TABLES-1) #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) From 6935cc667baa248e2038d2e33b7bd7f60b579d73 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 2 Oct 2021 12:49:20 +0000 Subject: [PATCH 112/609] Fix tests --- .../MaterializedPostgreSQLConsumer.cpp | 2 ++ .../PostgreSQLReplicationHandler.cpp | 24 +++++++++++++------ .../PostgreSQL/PostgreSQLReplicationHandler.h | 2 +- .../StorageMaterializedPostgreSQL.h | 2 +- .../test.py | 11 ++++----- 5 files changed, 25 insertions(+), 16 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 969dbdf9f43..7447170b68a 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -391,6 +391,8 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl if (storages.find(table_name) == storages.end()) { markTableAsSkipped(relation_id, table_name); + /// TODO: This can happen if we created a publication with this table but then got an exception that this + /// table has primary key or something else. LOG_ERROR(log, "Storage for table {} does not exist, but is included in replication stream. (Storages number: {})", table_name, storages.size()); diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 06540582b24..d6622f48e96 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -87,10 +87,14 @@ void PostgreSQLReplicationHandler::startup() } -String PostgreSQLReplicationHandler::doubleQuoteWithPossibleSchema(const String & table_name) const +String PostgreSQLReplicationHandler::probablyDoubleQuoteWithSchema(const String & table_name, bool quote) const { if (table_name.starts_with("\"")) + { + if (!quote) + return table_name.substr(1, table_name.size() - 1); return table_name; + } /// !schema_list.empty() -- We replicate all tables from specifies schemas. /// In this case when tables list is fetched, we append schema with dot. But without quotes. @@ -109,7 +113,13 @@ String PostgreSQLReplicationHandler::doubleQuoteWithPossibleSchema(const String } if (postgres_schema.empty()) - return doubleQuoteString(table_name); + { + /// We do no need quotes to fetch table structure in case there is no schema. (will not work) + if (quote) + return doubleQuoteString(table_name); + else + return table_name; + } return doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(table_name); } @@ -294,7 +304,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. - auto quoted_name = doubleQuoteWithPossibleSchema(table_name); + auto quoted_name = probablyDoubleQuoteWithSchema(table_name); query_str = fmt::format("SELECT * FROM {}", quoted_name); LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); @@ -403,7 +413,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio WriteBufferFromOwnString buf; for (const auto & storage_data : materialized_storages) { - buf << doubleQuoteWithPossibleSchema(storage_data.first); + buf << probablyDoubleQuoteWithSchema(storage_data.first); buf << ","; } tables_list = buf.str(); @@ -684,7 +694,7 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() for (auto & table_name : tables_names) { boost::trim(table_name); - buf << doubleQuoteWithPossibleSchema(table_name); + buf << probablyDoubleQuoteWithSchema(table_name); buf << ","; } tables_list = buf.str(); @@ -705,7 +715,7 @@ NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx std::unordered_set tables; for (const auto & [schema, table] : tx.stream(query)) - tables.insert(schema.empty() ? table : schema + '.' + table); + tables.insert((!schema.empty() && schema_as_a_part_of_table_name) ? schema + '.' + table : table); return tables; } @@ -717,7 +727,7 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( if (!is_materialized_postgresql_database) return nullptr; - return std::make_unique(fetchPostgreSQLTableStructure(tx, doubleQuoteWithPossibleSchema(table_name), true, true, true)); + return std::make_unique(fetchPostgreSQLTableStructure(tx, probablyDoubleQuoteWithSchema(table_name, false), true, true, true)); } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index dfe72361de5..8df9229a081 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -89,7 +89,7 @@ private: PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name) const; - String doubleQuoteWithPossibleSchema(const String & table_name) const; + String probablyDoubleQuoteWithSchema(const String & table_name, bool quote = true) const; Poco::Logger * log; ContextPtr context; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 181fbdf4c02..0a1bef5d737 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -26,7 +26,7 @@ namespace DB /** TODO list: * - Actually I think we can support ddl even though logical replication does not fully support it. - * But some basic ddl like adding/dropping columns, changing column type, column names -- is managable. + * But some basic ddl like adding/dropping columns, changing column type, column names -- is manageable. */ /** Case of single MaterializedPostgreSQL table engine. diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index c06a537d2fa..da27e06c193 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -463,9 +463,7 @@ def test_clickhouse_restart(started_cluster): def test_replica_identity_index(started_cluster): drop_materialized_db() - conn = get_postgres_conn(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port, - database=True) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() create_postgres_table(cursor, 'postgresql_replica', template=postgres_table_template_3); @@ -473,8 +471,7 @@ def test_replica_identity_index(started_cluster): cursor.execute("ALTER TABLE postgresql_replica REPLICA IDENTITY USING INDEX idx") instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(50, 10)") - create_materialized_db(ip=started_cluster.postgres_ip, - port=started_cluster.postgres_port) + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port) instance.query("INSERT INTO postgres_database.postgresql_replica SELECT number, number, number, number from numbers(100, 10)") check_tables_are_synchronized('postgresql_replica', order_by='key1'); @@ -718,8 +715,8 @@ def test_multiple_databases(started_cluster): cursor1 = conn1.cursor() cursor2 = conn2.cursor() - create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_1') - create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_2') + create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_1', 'postgres_database_1') + create_clickhouse_postgres_db(cluster.postgres_ip, cluster.postgres_port, 'postgres_database_2', 'postgres_database_2') cursors = [cursor1, cursor2] for cursor_id in range(len(cursors)): From e4d1d43a3454e435613d119fa5f0df915f2f39ee Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 3 Oct 2021 08:55:50 +0000 Subject: [PATCH 113/609] Fixes --- .../materialized-postgresql.md | 2 + .../DatabaseMaterializedPostgreSQL.cpp | 2 +- .../fetchPostgreSQLTableStructure.cpp | 12 +++--- .../fetchPostgreSQLTableStructure.h | 5 ++- .../MaterializedPostgreSQLConsumer.cpp | 12 ++++-- .../MaterializedPostgreSQLSettings.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 43 ++++++++++++------- .../PostgreSQL/PostgreSQLReplicationHandler.h | 12 +++--- .../StorageMaterializedPostgreSQL.cpp | 2 +- .../test.py | 2 +- 10 files changed, 56 insertions(+), 38 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 9bd626cde32..89c46f27945 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -90,12 +90,14 @@ Tables are accessed via schema name and table name at the same time: CREATE DATABASE database1 ENGINE = MaterializedPostgreSQL('postgres1:5432', 'postgres_database', 'postgres_user', 'postgres_password') SETTINGS materialized_postgresql_tables_list = 'schema1.table1,schema2.table2,schema1.table3'; + materialized_postgresql_tables_list_with_schema = 1; SELECT * FROM database1.`schema1.table1`; SELECT * FROM database1.`schema2.table2`; ``` But in this case all tables in `materialized_postgresql_tables_list` must be written with its schema name. +Requires `materialized_postgresql_tables_list_with_schema = 1`. Warning: for this case dots in table name are not allowed. diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 1b82e79400d..24cb678349f 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -66,7 +66,7 @@ void DatabaseMaterializedPostgreSQL::startSynchronization() *settings, /* is_materialized_postgresql_database = */ true); - NameSet tables_to_replicate; + std::set tables_to_replicate; try { tables_to_replicate = replication_handler->fetchRequiredTables(); diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index ca91b51be57..62a668d3359 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -29,14 +29,14 @@ namespace ErrorCodes template -std::unordered_set fetchPostgreSQLTablesList(T & tx, const String & postgres_schema) +std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schema) { Names schemas; boost::split(schemas, postgres_schema, [](char c){ return c == ','; }); for (String & key : schemas) boost::trim(key); - std::unordered_set tables; + std::set tables; if (schemas.size() <= 1) { std::string query = fmt::format("SELECT tablename FROM pg_catalog.pg_tables " @@ -294,7 +294,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure(pqxx::connection & connec } -std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection, const String & postgres_schema) +std::set fetchPostgreSQLTablesList(pqxx::connection & connection, const String & postgres_schema) { pqxx::ReadTransaction tx(connection); auto result = fetchPostgreSQLTablesList(tx, postgres_schema); @@ -319,13 +319,13 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( bool with_primary_key, bool with_replica_identity_index); template -std::unordered_set fetchPostgreSQLTablesList(pqxx::work & tx, const String & postgres_schema); +std::set fetchPostgreSQLTablesList(pqxx::work & tx, const String & postgres_schema); template -std::unordered_set fetchPostgreSQLTablesList(pqxx::ReadTransaction & tx, const String & postgres_schema); +std::set fetchPostgreSQLTablesList(pqxx::ReadTransaction & tx, const String & postgres_schema); template -std::unordered_set fetchPostgreSQLTablesList(pqxx::nontransaction & tx, const String & postgres_schema); +std::set fetchPostgreSQLTablesList(pqxx::nontransaction & tx, const String & postgres_schema); } diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h index 0097287701c..5ecb38abf17 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.h @@ -21,7 +21,8 @@ struct PostgreSQLTableStructure using PostgreSQLTableStructurePtr = std::unique_ptr; -std::unordered_set fetchPostgreSQLTablesList(pqxx::connection & connection, const String & postgres_schema); +/// We need order for materialized version. +std::set fetchPostgreSQLTablesList(pqxx::connection & connection, const String & postgres_schema); PostgreSQLTableStructure fetchPostgreSQLTableStructure( pqxx::connection & connection, const String & postgres_table_name, bool use_nulls = true); @@ -32,7 +33,7 @@ PostgreSQLTableStructure fetchPostgreSQLTableStructure( bool with_primary_key = false, bool with_replica_identity_index = false); template -std::unordered_set fetchPostgreSQLTablesList(T & tx, const String & postgres_schema); +std::set fetchPostgreSQLTablesList(T & tx, const String & postgres_schema); } diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index 7447170b68a..334d8067419 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -277,7 +277,9 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { Int32 relation_id = readInt32(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; - assert(!table_name.empty()); + /// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove? + if (table_name.empty()) + LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id); if (!isSyncAllowed(relation_id, table_name)) return; @@ -295,7 +297,9 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { Int32 relation_id = readInt32(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; - assert(!table_name.empty()); + /// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove? + if (table_name.empty()) + LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id); if (!isSyncAllowed(relation_id, table_name)) return; @@ -344,7 +348,9 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl { Int32 relation_id = readInt32(replication_message, pos, size); const auto & table_name = relation_id_to_name[relation_id]; - assert(!table_name.empty()); + /// FIXME:If table name is empty here, it means we failed to load it, but it was included in publication. Need to remove? + if (table_name.empty()) + LOG_WARNING(log, "No table mapping for relation id: {}. Probably table failed to be loaded", relation_id); if (!isSyncAllowed(relation_id, table_name)) return; diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h index 3f09f752cb0..505de5883ed 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLSettings.h @@ -21,7 +21,7 @@ namespace DB M(String, materialized_postgresql_replication_slot, "", "A user-created replication slot", 0) \ M(String, materialized_postgresql_snapshot, "", "User provided snapshot in case he manages replication slots himself", 0) \ M(String, materialized_postgresql_schema, "", "PostgreSQL schema", 0) \ - M(Bool, materialized_postgresql_tables_list_with_schema, true, \ + M(Bool, materialized_postgresql_tables_list_with_schema, false, \ "Consider by default that if there is a dot in tables list 'name.name', " \ "then the first name is postgres schema and second is postgres table. This setting is needed to allow table names with dots", 0) \ diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index d6622f48e96..960f387669e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -47,7 +47,7 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( , is_materialized_postgresql_database(is_materialized_postgresql_database_) , tables_list(replication_settings.materialized_postgresql_tables_list) , schema_list(replication_settings.materialized_postgresql_schema_list) - , schema_can_be_in_tables_list(replication_settings.materialized_postgresql_tables_list_with_schema) + , schema_as_a_part_of_table_name(!schema_list.empty() || replication_settings.materialized_postgresql_tables_list_with_schema) , user_provided_snapshot(replication_settings.materialized_postgresql_snapshot) , milliseconds_to_wait(RESCHEDULE_MS) { @@ -67,9 +67,6 @@ PostgreSQLReplicationHandler::PostgreSQLReplicationHandler( startup_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ checkConnectionAndStart(); }); consumer_task = context->getSchedulePool().createTask("PostgreSQLReplicaStartup", [this]{ consumerFunc(); }); - - if (!schema_list.empty()) - schema_as_a_part_of_table_name = true; } @@ -101,9 +98,8 @@ String PostgreSQLReplicationHandler::probablyDoubleQuoteWithSchema(const String /// If there is a setting `tables_list`, then table names can be put there along with schema, /// separated by dot and with no quotes. We add double quotes in this case. - bool schema_in_name = (schema_can_be_in_tables_list && !tables_list.empty()) || !schema_list.empty(); - if (auto pos = table_name.find('.'); schema_in_name && pos != std::string::npos) + if (auto pos = table_name.find('.'); schema_as_a_part_of_table_name && pos != std::string::npos) { schema_as_a_part_of_table_name = true; @@ -580,10 +576,10 @@ void PostgreSQLReplicationHandler::shutdownFinal() /// Used by MaterializedPostgreSQL database engine. -NameSet PostgreSQLReplicationHandler::fetchRequiredTables() +std::set PostgreSQLReplicationHandler::fetchRequiredTables() { postgres::Connection connection(connection_info); - NameSet result_tables; + std::set result_tables; bool publication_exists_before_startup; { @@ -638,8 +634,8 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() } NameSet diff; - std::set_symmetric_difference(expected_tables.begin(), expected_tables.end(), - result_tables.begin(), result_tables.end(), + std::sort(expected_tables.begin(), expected_tables.end()); + std::set_symmetric_difference(expected_tables.begin(), expected_tables.end(), result_tables.begin(), result_tables.end(), std::inserter(diff, diff.begin())); if (!diff.empty()) { @@ -650,10 +646,25 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() diff_tables += ", "; diff_tables += table_name; } + String publication_tables; + for (const auto & table_name : result_tables) + { + if (!publication_tables.empty()) + publication_tables += ", "; + publication_tables += table_name; + } + String listed_tables; + for (const auto & table_name : expected_tables) + { + if (!listed_tables.empty()) + listed_tables += ", "; + listed_tables += table_name; + } LOG_WARNING(log, - "Publication {} already exists, but specified tables list differs from publication tables list in tables: {}.", - publication_name, diff_tables); + "Publication {} already exists, but specified tables list differs from publication tables list in tables: {}." + "\nPublication tables: {}.\nTables list: {}", + publication_name, diff_tables, publication_tables, listed_tables); connection.execWithRetry([&](pqxx::nontransaction & tx_){ dropPublication(tx_); }); } @@ -675,7 +686,7 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() } else { - result_tables = NameSet(expected_tables.begin(), expected_tables.end()); + result_tables = std::set(expected_tables.begin(), expected_tables.end()); } } @@ -709,13 +720,13 @@ NameSet PostgreSQLReplicationHandler::fetchRequiredTables() } -NameSet PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx) +std::set PostgreSQLReplicationHandler::fetchTablesFromPublication(pqxx::work & tx) { std::string query = fmt::format("SELECT schemaname, tablename FROM pg_publication_tables WHERE pubname = '{}'", publication_name); - std::unordered_set tables; + std::set tables; for (const auto & [schema, table] : tx.stream(query)) - tables.insert((!schema.empty() && schema_as_a_part_of_table_name) ? schema + '.' + table : table); + tables.insert(schema_as_a_part_of_table_name ? schema + '.' + table : table); return tables; } diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 8df9229a081..8acc4c82716 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -39,7 +39,7 @@ public: void addStorage(const std::string & table_name, StorageMaterializedPostgreSQL * storage); /// Fetch list of tables which are going to be replicated. Used for database engine. - NameSet fetchRequiredTables(); + std::set fetchRequiredTables(); /// Start replication setup immediately. void startSynchronization(bool throw_on_error); @@ -61,7 +61,7 @@ private: void createPublicationIfNeeded(pqxx::nontransaction & tx); - NameSet fetchTablesFromPublication(pqxx::work & tx); + std::set fetchTablesFromPublication(pqxx::work & tx); void dropPublication(pqxx::nontransaction & ntx); @@ -110,10 +110,6 @@ private: /// max_block_size for replication stream. const size_t max_block_size; - /// Schema can be as a part of table name, i.e. as a clickhouse table it is accessed like db.`schema.table`. - /// This is possible to allow replicating tables from multiple schemas in the same MaterializedPostgreSQL database engine. - mutable bool schema_as_a_part_of_table_name = false; - /// Table structure changes are always tracked. By default, table with changed schema will get into a skip list. /// This setting allows to reloas table in the background. bool allow_automatic_update = false; @@ -126,7 +122,9 @@ private: String schema_list; - bool schema_can_be_in_tables_list; + /// Schema can be as a part of table name, i.e. as a clickhouse table it is accessed like db.`schema.table`. + /// This is possible to allow replicating tables from multiple schemas in the same MaterializedPostgreSQL database engine. + mutable bool schema_as_a_part_of_table_name = false; bool user_managed_slot = true; String user_provided_snapshot; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index 1b17e6c0c6e..1b0f472cd27 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -143,7 +143,7 @@ StoragePtr StorageMaterializedPostgreSQL::createTemporary() const } auto new_context = Context::createCopy(context); - return StorageMaterializedPostgreSQL::create(tmp_table_id, new_context, "", table_id.table_name); + return StorageMaterializedPostgreSQL::create(tmp_table_id, new_context, "temporary", table_id.table_name); } diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index da27e06c193..a14d8b9009e 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -1209,7 +1209,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): insert_into_tables() create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, - settings=[f"materialized_postgresql_tables_list = '{publication_tables}'", "materialized_postgresql_allow_automatic_update = 1"]) + settings=[f"materialized_postgresql_tables_list = '{publication_tables}'", "materialized_postgresql_tables_list_with_schema=1", "materialized_postgresql_allow_automatic_update = 1"]) check_all_tables_are_synchronized() assert_show_tables("test_schema.postgresql_replica_0\ntest_schema.postgresql_replica_1\ntest_schema.postgresql_replica_2\ntest_schema.postgresql_replica_3\ntest_schema.postgresql_replica_4\n") From 7bb26497441e5a18f5538a6bd5384f33c286d6f6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 3 Oct 2021 14:36:32 +0000 Subject: [PATCH 114/609] Fix checks --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 -- src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp | 2 -- 2 files changed, 4 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 0ea67c6a1b1..dacd2c4d2eb 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -14,8 +14,6 @@ #include #include #include -#include -#include #include diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 960f387669e..6e5e00b340e 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -101,8 +101,6 @@ String PostgreSQLReplicationHandler::probablyDoubleQuoteWithSchema(const String if (auto pos = table_name.find('.'); schema_as_a_part_of_table_name && pos != std::string::npos) { - schema_as_a_part_of_table_name = true; - auto schema = table_name.substr(0, pos); auto table = table_name.substr(pos + 1); return doubleQuoteString(schema) + '.' + doubleQuoteString(table); From c5cb4e071c1f8026850d2eb0eebedc75b656e8d2 Mon Sep 17 00:00:00 2001 From: Alexey Boykov Date: Thu, 7 Oct 2021 21:01:36 +0300 Subject: [PATCH 115/609] Creating only one binary, check compatibility --- programs/CMakeLists.txt | 4 ++ programs/local/CMakeLists.txt | 8 ++-- programs/local/LocalServer.cpp | 70 +++++++++++++++++++++++++++++++++- programs/main.cpp | 15 ++++++-- 4 files changed, 89 insertions(+), 8 deletions(-) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index 43d9f974648..4806a7fe46e 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -473,3 +473,7 @@ if (ENABLE_TESTS AND USE_GTEST) add_custom_target (clickhouse-tests ALL DEPENDS ${CLICKHOUSE_UNIT_TESTS_TARGETS}) add_dependencies(clickhouse-bundle clickhouse-tests) endif() + +if (ENABLE_FUZZING) + add_compile_definitions(FUZZING_MODE=1) +endif () diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index d965ecf05be..4ac8ad5d30d 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -21,13 +21,15 @@ endif() if (ENABLE_FUZZING) add_compile_definitions(FUZZING_MODE=1) set (WITH_COVERAGE ON) - add_executable(fuzz-clickhouse-local LocalServer.cpp ${SRCS}) - target_link_libraries(fuzz-clickhouse-local PRIVATE + target_link_libraries(clickhouse-local-lib PRIVATE ${LIB_FUZZING_ENGINE}) + #add_executable(fuzz-clickhouse-local LocalServer.cpp ${SRCS}) + #[[target_link_libraries(fuzz-clickhouse-local PRIVATE dbms ${LIB_FUZZING_ENGINE} loggers clickhouse_functions clickhouse_aggregate_functions clickhouse_storages_system - clickhouse_table_functions) + clickhouse_table_functions + readpassphrase)]] endif () diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index d93fa82d59c..9b567e60193 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -418,6 +418,11 @@ try ThreadStatus thread_status; setupSignalHandler(); +#ifdef FUZZING_MODE + static bool first_time = true; + if (first_time) + { +#endif std::cout << std::fixed << std::setprecision(3); std::cerr << std::fixed << std::setprecision(3); @@ -441,6 +446,10 @@ try processConfig(); applyCmdSettings(global_context); connect(); +#ifdef FUZZING_MODE + first_time = false; + } +#endif if (is_interactive) { @@ -455,7 +464,9 @@ try runNonInteractive(); } +#ifndef FUZZING_MODE cleanup(); +#endif return Application::EXIT_OK; } catch (...) @@ -728,4 +739,61 @@ int mainEntryClickHouseLocal(int argc, char ** argv) auto code = DB::getCurrentExceptionCode(); return code ? code : 1; } -} \ No newline at end of file +} + +#ifdef FUZZING_MODE +#include + +std::optional fuzz_app; + +extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) +{ + int & argc = *pargc; + char ** argv = *pargv; + + // position of delimiter "--" that separates arguments + // of clickhouse-local and fuzzer + int pos_delim = argc; + for (int i = 0; i < argc; ++i) + { + if (strcmp(argv[i], "--") == 0) + { + pos_delim = i; + break; + } + } + + fuzz_app.emplace(); + fuzz_app->init(pos_delim, argv); + for (int i = pos_delim + 1; i < argc; ++i) + std::swap(argv[i], argv[i - pos_delim]); + argc -= pos_delim; + if (argc == 0) // no delimiter provided + ++argc; + return 0; +} + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +{ + try + { + // inappropriate symbol for fuzzing at the end + if (size) + --size; + auto cur_str = String(reinterpret_cast(data), size); + // to clearly see the beginning and the end + std::cerr << '>' << cur_str << '<' << std::endl; + DB::FunctionGetFuzzerData::update(cur_str); + fuzz_app->run(); + } + catch (...) + { + std::cerr << "Why here?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!" << std::endl; + std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; + return 0; + //auto code = DB::getCurrentExceptionCode(); + //return code ? code : 1; + } + return 0; +} +#endif diff --git a/programs/main.cpp b/programs/main.cpp index 8c70dcacc99..bd5dabbd100 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -90,7 +90,7 @@ using MainFunc = int (*)(int, char**); /// Add an item here to register new application -std::pair clickhouse_applications[] = +[[maybe_unused]]std::pair clickhouse_applications[] = { #if ENABLE_CLICKHOUSE_LOCAL {"local", mainEntryClickHouseLocal}, @@ -141,7 +141,7 @@ std::pair clickhouse_applications[] = {"hash-binary", mainEntryClickHouseHashBinary}, }; - +#ifndef FUZZING_MODE int printHelp(int, char **) { std::cerr << "Use one of the following commands:" << std::endl; @@ -149,8 +149,9 @@ int printHelp(int, char **) std::cerr << "clickhouse " << application.first << " [args] " << std::endl; return -1; } +#endif - +#ifndef FUZZING_MODE bool isClickhouseApp(const std::string & app_suffix, std::vector & argv) { /// Use app if the first arg 'app' is passed (the arg should be quietly removed) @@ -170,6 +171,7 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector & argv) std::string app_name = "clickhouse-" + app_suffix; return !argv.empty() && (app_name == argv[0] || endsWith(argv[0], "/" + app_name)); } +#endif enum class InstructionFail @@ -338,9 +340,13 @@ struct Checker /// /// extern bool inside_main; /// class C { C() { assert(inside_main); } }; +#ifndef FUZZING_MODE bool inside_main = false; +#else +bool inside_main = true; +#endif - +#ifndef FUZZING_MODE int main(int argc_, char ** argv_) { inside_main = true; @@ -371,3 +377,4 @@ int main(int argc_, char ** argv_) return main_func(static_cast(argv.size()), argv.data()); } +#endif From 1ce331a3c54251b1bc6a59bb75c8d6752b1a887c Mon Sep 17 00:00:00 2001 From: qianmoQ Date: Tue, 26 Oct 2021 15:00:01 +0800 Subject: [PATCH 116/609] Add architecture --- docs/zh/development/architecture.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/development/architecture.md b/docs/zh/development/architecture.md index dfc66b36730..882728a6e2b 100644 --- a/docs/zh/development/architecture.md +++ b/docs/zh/development/architecture.md @@ -191,4 +191,4 @@ ClickHouse 中的复制是基于表实现的。你可以在同一个服务器上 > ClickHouse 集群由独立的分片组成,每一个分片由多个副本组成。集群不是弹性的,因此在添加新的分片后,数据不会自动在分片之间重新平衡。相反,集群负载将变得不均衡。该实现为你提供了更多控制,对于相对较小的集群,例如只有数十个节点的集群来说是很好的。但是对于我们在生产中使用的具有数百个节点的集群来说,这种方法成为一个重大缺陷。我们应该实现一个表引擎,使得该引擎能够跨集群扩展数据,同时具有动态复制的区域,这些区域能够在集群之间自动拆分和平衡。 -[来源文章](https://clickhouse.com/docs/en/development/architecture/) +[来源文章](https://clickhouse.tech/docs/en/development/architecture/) From 40713194207856c551b6528c74ee2190e6567065 Mon Sep 17 00:00:00 2001 From: qianmoQ Date: Tue, 26 Oct 2021 15:32:58 +0800 Subject: [PATCH 117/609] Architecture Overview --- docs/zh/development/architecture.md | 205 ++++++++++++++-------------- 1 file changed, 105 insertions(+), 100 deletions(-) diff --git a/docs/zh/development/architecture.md b/docs/zh/development/architecture.md index 882728a6e2b..1bd1a8b04f9 100644 --- a/docs/zh/development/architecture.md +++ b/docs/zh/development/architecture.md @@ -1,194 +1,199 @@ -# ClickHouse 架构概述 {#clickhouse-jia-gou-gai-shu} +--- +toc_priority: 62 +toc_title: 架构概述 +--- -ClickHouse 是一个真正的列式数据库管理系统(DBMS)。在 ClickHouse 中,数据始终是按列存储的,包括矢量(向量或列块)执行的过程。只要有可能,操作都是基于矢量进行分派的,而不是单个的值,这被称为«矢量化查询执行»,它有利于降低实际的数据处理开销。 +# ClickHouse 架构概述 {#overview-of-clickhouse-architecture} -> 这个想法并不新鲜,其可以追溯到 `APL` 编程语言及其后代:`A +`、`J`、`K` 和 `Q`。矢量编程被大量用于科学数据处理中。即使在关系型数据库中,这个想法也不是什么新的东西:比如,矢量编程也被大量用于 `Vectorwise` 系统中。 +ClickHouse是一个真正面向列的DBMS。数据按列存储,并在执行数组(向量或列块)期间存储。 只要有可能,操作都是在数组上调度的,而不是在单个值上调度。它被称为“向量化查询执行”,它有助于降低实际数据处理的成本。 -通常有两种不同的加速查询处理的方法:矢量化查询执行和运行时代码生成。在后者中,动态地为每一类查询生成代码,消除了间接分派和动态分派。这两种方法中,并没有哪一种严格地比另一种好。运行时代码生成可以更好地将多个操作融合在一起,从而充分利用 CPU 执行单元和流水线。矢量化查询执行不是特别实用,因为它涉及必须写到缓存并读回的临时向量。如果 L2 缓存容纳不下临时数据,那么这将成为一个问题。但矢量化查询执行更容易利用 CPU 的 SIMD 功能。朋友写的一篇[研究论文](http://15721.courses.cs.cmu.edu/spring2016/papers/p5-sompolski.pdf)表明,将两种方法结合起来是更好的选择。ClickHouse 使用了矢量化查询执行,同时初步提供了有限的运行时动态代码生成。 +> 这个想法并不新。它可以追溯到`APL`(一种编程语言,1957年)及其后代:`A +`(APL方言)、`J`(1990年)、`K`(1993年)和`Q`(来自Kx系统,2003年)。数组编程用于科学数据处理。这个想法在关系数据库中也不是什么新鲜事物:例如,它被用于“VectorWise”系统(也被Actian Corporation称为 Actian Vector Analytic Database)。 -## 列(Columns) {#lie-columns} +有两种不同的方法可以加速查询处理:向量化查询执行和运行时代码生成。后者删除了所有间接和动态调度。这两种方法都没有严格意义上的优于另一种。运行时代码生成可以更好地融合许多操作,从而充分利用 CPU 执行单元和管道。向量化查询执行可能不太实用,因为它涉及必须写入缓存并读回的临时向量。如果临时数据不适合L2缓存,这就会成为一个问题。但是矢量化查询执行更容易利用CPU的SIMD功能。我们朋友写的一篇[research paper](http://15721.courses.cs.cmu.edu/spring2016/papers/p5-sompolski.pdf)表明最好将两种方法结合起来。ClickHouse使用矢量化查询执行,并且对运行时代码生成的初始支持有限。 -要表示内存中的列(实际上是列块),需使用 `IColumn` 接口。该接口提供了用于实现各种关系操作符的辅助方法。几乎所有的操作都是不可变的:这些操作不会更改原始列,但是会创建一个新的修改后的列。比如,`IColumn::filter` 方法接受过滤字节掩码,用于 `WHERE` 和 `HAVING` 关系操作符中。另外的例子:`IColumn::permute` 方法支持 `ORDER BY` 实现,`IColumn::cut` 方法支持 `LIMIT` 实现等等。 +## 列 {#columns} -不同的 `IColumn` 实现(`ColumnUInt8`、`ColumnString` 等)负责不同的列内存布局。内存布局通常是一个连续的数组。对于数据类型为整型的列,只是一个连续的数组,比如 `std::vector`。对于 `String` 列和 `Array` 列,则由两个向量组成:其中一个向量连续存储所有的 `String` 或数组元素,另一个存储每一个 `String` 或 `Array` 的起始元素在第一个向量中的偏移。而 `ColumnConst` 则仅在内存中存储一个值,但是看起来像一个列。 +`IColumn`接口用于表示内存中的列(实际上是列的块)。该接口为各种关系运算符的实现提供了辅助方法。几乎所有操作都是不可变的:它们不会修改原始列,而是创建一个新的修改过的列。例如,`IColumn :: filter`方法接受过滤器字节掩码。 它用于`WHERE`和`HAVING`关系运算符。 其他示例:支持`ORDER BY`的`IColumn :: permute`方法,支持`LIMIT`的`IColumn :: cut`方法。 -## 字段 {#field} +各种`IColumn`实现(`ColumnUInt8`、`ColumnString`等)负责列的内存布局。 内存布局通常是一个连续的数组。对于整数类型的列,它只是一个连续的数组,就像`std :: vector`。对于`String`和`Array`列,它是两个向量:一个用于所有数组元素,连续放置,第二个用于每个数组开头的偏移量。还有`ColumnConst`只在内存中存储一个值,但看起来像一列。 -尽管如此,有时候也可能需要处理单个值。表示单个值,可以使用 `Field`。`Field` 是 `UInt64`、`Int64`、`Float64`、`String` 和 `Array` 组成的联合。`IColumn` 拥有 `operator[]` 方法来获取第 `n` 个值成为一个 `Field`,同时也拥有 `insert` 方法将一个 `Field` 追加到一个列的末尾。这些方法并不高效,因为它们需要处理表示单一值的临时 `Field` 对象,但是有更高效的方法比如 `insertFrom` 和 `insertRangeFrom` 等。 +## 字段属性 {#field} -`Field` 中并没有足够的关于一个表(table)的特定数据类型的信息。比如,`UInt8`、`UInt16`、`UInt32` 和 `UInt64` 在 `Field` 中均表示为 `UInt64`。 +尽管如此,也可以使用单个值。为了表示单个值,使用了`Field`。`Field`只是`UInt64`、`Int64`、`Float64`、`String` 和`Array`的有区别的并集。`IColumn`有`operator []`方法来获取第n个值作为`Field`,以及`insert`方法将一个`Field`附加到列的末尾。这些方法效率不高,因为它们需要处理表示单个值的临时`Field`对象。还有更高效的方法,比如`insertFrom`、`insertRangeFrom`等。 -## 抽象漏洞 {#chou-xiang-lou-dong} +`Field`没有关于表特定数据类型的足够信息。例如,`UInt8`、`UInt16`、`UInt32` 和`UInt64`在`Field`中都表示为`UInt64`。 -`IColumn` 具有用于数据的常见关系转换的方法,但这些方法并不能够满足所有需求。比如,`ColumnUInt64` 没有用于计算两列和的方法,`ColumnString` 没有用于进行子串搜索的方法。这些无法计算的例程在 `Icolumn` 之外实现。 +## Leaky Abstractions {#leaky-abstractions} -列(Columns)上的各种函数可以通过使用 `Icolumn` 的方法来提取 `Field` 值,或根据特定的 `Icolumn` 实现的数据内存布局的知识,以一种通用但不高效的方式实现。为此,函数将会转换为特定的 `IColumn` 类型并直接处理内部表示。比如,`ColumnUInt64` 具有 `getData` 方法,该方法返回一个指向列的内部数组的引用,然后一个单独的例程可以直接读写或填充该数组。实际上,«抽象漏洞(leaky abstractions)»允许我们以更高效的方式来实现各种特定的例程。 +`IColumn`有通用的数据关系转换方法,但不能满足所有需求。例如,`ColumnUInt64`没有计算两列总和的方法,而`ColumnString`没有运行子字符串搜索的方法。这些无数的例程是在`IColumn`之外实现的。 -## 数据类型 {#shu-ju-lei-xing} +列上的各种功能可以使用`IColumn`方法提取`Field`值以通用的、非有效的方式实现,或者使用特定`IColumn`实现中数据的内部内存布局知识以特殊方式实现。它是通过将函数转换为特定的`IColumn`类型并直接处理内部表示来实现的。例如,`ColumnUInt64`具有`getData`方法,该方法返回对内部数组的引用,然后一个单独的线程直接读取或填充该数组。 我们有"leaky abstractions"来允许对各种线程进行有效的专业化。 -`IDataType` 负责序列化和反序列化:读写二进制或文本形式的列或单个值构成的块。`IDataType` 直接与表的数据类型相对应。比如,有 `DataTypeUInt32`、`DataTypeDateTime`、`DataTypeString` 等数据类型。 +## 数据类型 {#data_types} -`IDataType` 与 `IColumn` 之间的关联并不大。不同的数据类型在内存中能够用相同的 `IColumn` 实现来表示。比如,`DataTypeUInt32` 和 `DataTypeDateTime` 都是用 `ColumnUInt32` 或 `ColumnConstUInt32` 来表示的。另外,相同的数据类型也可以用不同的 `IColumn` 实现来表示。比如,`DataTypeUInt8` 既可以使用 `ColumnUInt8` 来表示,也可以使用过 `ColumnConstUInt8` 来表示。 +`IDataType`负责序列化和反序列化:用于以二进制或文本形式读取和写入大块的列或单个值。`IDataType`直接对应表中的数据类型。比如有`DataTypeUInt32`、`DataTypeDateTime`、`DataTypeString`等等。 -`IDataType` 仅存储元数据。比如,`DataTypeUInt8` 不存储任何东西(除了 vptr);`DataTypeFixedString` 仅存储 `N`(固定长度字符串的串长度)。 +`IDataType`和`IColumn`只是彼此松散的关联。不同的数据类型可以通过相同的`IColumn`实现在内存中表示。例如,`DataTypeUInt32`和`DataTypeDateTime`都由`ColumnUInt32`或`ColumnConstUInt32`表示。此外,相同的数据类型可以由不同的`IColumn`实现来表示。例如,`DataTypeUInt8`可以用`ColumnUInt8`或`ColumnConstUInt8`表示。 -`IDataType` 具有针对各种数据格式的辅助函数。比如如下一些辅助函数:序列化一个值并加上可能的引号;序列化一个值用于 JSON 格式;序列化一个值作为 XML 格式的一部分。辅助函数与数据格式并没有直接的对应。比如,两种不同的数据格式 `Pretty` 和 `TabSeparated` 均可以使用 `IDataType` 接口提供的 `serializeTextEscaped` 这一辅助函数。 +`IDataType`只存储元数据。例如,`DataTypeUInt8`根本不存储任何东西(除了虚拟指针`vptr`),而`DataTypeFixedString`只存储`N`(固定大小字符串的大小)。 -## 块(Block) {#kuai-block} +`IDataType`有各种数据格式的辅助方法。示例是使用引用序列化值、序列化JSON以及将值序列化为XML格式的一部分的方法。与数据格式没有直接对应关系。例如,不同的数据格式`Pretty`和`TabSeparated`可以使用来自`IDataType`接口的相同`serializeTextEscaped`辅助方法。 -`Block` 是表示内存中表的子集(chunk)的容器,是由三元组:`(IColumn, IDataType, 列名)` 构成的集合。在查询执行期间,数据是按 `Block` 进行处理的。如果我们有一个 `Block`,那么就有了数据(在 `IColumn` 对象中),有了数据的类型信息告诉我们如何处理该列,同时也有了列名(来自表的原始列名,或人为指定的用于临时计算结果的名字)。 +## Block {#block} -当我们遍历一个块中的列进行某些函数计算时,会把结果列加入到块中,但不会更改函数参数中的列,因为操作是不可变的。之后,不需要的列可以从块中删除,但不是修改。这对于消除公共子表达式非常方便。 +`Block`是表示内存中表的子集(块)的容器。它只是一组三元组:`(IColumn, IDataType, column name)`。在查询执行期间,数据由`Block`处理。如果我们有一个`Block`,我们就有数据(在`IColumn`对象中),我们有关于它的类型的信息(在`IDataType`中),告诉我们如何处理该列,我们有列名。它可以是表中的原始列名,也可以是为获取临时计算结果而分配的一些人为名称。 -`Block` 用于处理数据块。注意,对于相同类型的计算,列名和类型对不同的块保持相同,仅列数据不同。最好把块数据(block data)和块头(block header)分离开来,因为小块大小会因复制共享指针和列名而带来很高的临时字符串开销。 +当我们在块中的列上计算某个函数时,我们将另一列及其结果添加到块中,并且我们不接触函数参数的列,因为操作是不可变的。稍后,可以从块中删除不需要的列,但不能修改。 便于消除公共子表达式。 -## 块流(Block Streams) {#kuai-liu-block-streams} +为每个处理过的数据创建块。请注意,对于相同类型的计算,不同块的列名和类型保持不变,仅列数据发生变化。最好从块头拆分块数据,因为小块大小具有用于复制shared_ptrs和列名的临时字符串的高开销。 -块流用于处理数据。我们可以使用块流从某个地方读取数据,执行数据转换,或将数据写到某个地方。`IBlockInputStream` 具有 `read` 方法,其能够在数据可用时获取下一个块。`IBlockOutputStream` 具有 `write` 方法,其能够将块写到某处。 +## Block Streams {#block-streams} -块流负责: +Block Streams用于处理数据。我们使用Block Streams从某处读取数据、执行数据转换或将数据写入某处。`IBlockInputStream`具有`read`方法来在可用时获取下一个块。`IBlockOutputStream`有`write`方法可以将块推送到某处。 -1. 读或写一个表。表仅返回一个流用于读写块。 -2. 完成数据格式化。比如,如果你打算将数据以 `Pretty` 格式输出到终端,你可以创建一个块输出流,将块写入该流中,然后进行格式化。 -3. 执行数据转换。假设你现在有 `IBlockInputStream` 并且打算创建一个过滤流,那么你可以创建一个 `FilterBlockInputStream` 并用 `IBlockInputStream` 进行初始化。之后,当你从 `FilterBlockInputStream` 中拉取块时,会从你的流中提取一个块,对其进行过滤,然后将过滤后的块返回给你。查询执行流水线就是以这种方式表示的。 +Streams主要负责: -还有一些更复杂的转换。比如,当你从 `AggregatingBlockInputStream` 拉取数据时,会从数据源读取全部数据进行聚集,然后将聚集后的数据流返回给你。另一个例子:`UnionBlockInputStream` 的构造函数接受多个输入源和多个线程,其能够启动多线程从多个输入源并行读取数据。 +1. 读取或写入表。该表仅返回用于读取或写入块的流。 +2. 实现数据格式。 例如,如果您想以`Pretty`格式将数据输出到终端,您可以创建一个块输出流,在其中推送块,并对其进行格式化。 +3. 执行数据转换。假设你有`IBlockInputStream`并且想要创建一个过滤流。您创建`FilterBlockInputStream`并使用您的流对其进行初始化。然后,当您从`FilterBlockInputStream`中拉出一个块时,它会从您的流中拉出一个块,对其进行过滤,并将过滤后的块返回给您。查询执行管道以这种方式表示。 -> 块流使用«pull»方法来控制流:当你从第一个流中拉取块时,它会接着从嵌套的流中拉取所需的块,然后整个执行流水线开始工作。»pull«和«push»都不是最好的方案,因为控制流不是明确的,这限制了各种功能的实现,比如多个查询同步执行(多个流水线合并到一起)。这个限制可以通过协程或直接运行互相等待的线程来解决。如果控制流明确,那么我们会有更多的可能性:如果我们定位了数据从一个计算单元传递到那些外部的计算单元中其中一个计算单元的逻辑。阅读这篇[文章](http://journal.stuffwithstuff.com/2013/01/13/iteration-inside-and-out/)来获取更多的想法。 +还有更复杂的转换。例如,当您从`AggregatingBlockInputStream`中提取时,它会从其源读取所有数据,对其进行聚合,然后为您返回聚合数据流。另一个例子:`UnionBlockInputStream`在构造函数中接受许多输入源以及许多线程。它启动多个线程并并行读取多个源。 -我们需要注意,查询执行流水线在每一步都会创建临时数据。我们要尽量使块的大小足够小,从而 CPU 缓存能够容纳下临时数据。在这个假设下,与其他计算相比,读写临时数据几乎是没有任何开销的。我们也可以考虑一种替代方案:将流水线中的多个操作融合在一起,使流水线尽可能短,并删除大量临时数据。这可能是一个优点,但同时也有缺点。比如,拆分流水线使得中间数据缓存、获取同时运行的类似查询的中间数据以及相似查询的流水线合并等功能很容易实现。 +> Block Streams使用“pull”方法来控制流:当您从第一个流中拉出一个块时,它从嵌套流中拉出所需的块,并且整个执行管道都将工作。“pull”和“push”都不是最好的解决方案,因为控制流是隐式的,这限制了各种功能的实现,比如同时执行多个查询(将许多管道合并在一起)。可以通过线程或仅运行相互等待的额外线程来克服此限制。如果我们明确控制流,我们可能会有更多的可控性:如果我们找到将数据从一个计算单元传递到这些计算单元之外的另一个计算单元的逻辑. 阅读[文章](http://journal.stuffwithstuff.com/2013/01/13/iteration-inside-and-out/). -## 格式(Formats) {#ge-shi-formats} +我们应该注意到查询执行管道在每一步都会创建临时数据。我们尽量保持块大小足够小,以便临时数据适配CPU缓存。在这种假设下,与其他计算相比,临时数据的写入和读取几乎是免费的。我们可以考虑另一种选择,那就是将许多操作融合在一起。它可以使管道尽可能短,并删除许多临时数据,这可能是一个优点,但也有缺点。例如,拆分管道可以轻松实现缓存中间数据、同时运行的类似查询中窃取中间数据以及为类似查询合并管道。 -数据格式同块流一起实现。既有仅用于向客户端输出数据的»展示«格式,如 `IBlockOutputStream` 提供的 `Pretty` 格式,也有其它输入输出格式,比如 `TabSeparated` 或 `JSONEachRow`。 +## Formats {#formats} -此外还有行流:`IRowInputStream` 和 `IRowOutputStream`。它们允许你按行 pull/push 数据,而不是按块。行流只需要简单地面向行格式实现。包装器 `BlockInputStreamFromRowInputStream` 和 `BlockOutputStreamFromRowOutputStream` 允许你将面向行的流转换为正常的面向块的流。 +数据格式通过Block Streams实现。有一些“表示”格式仅适用于向客户端输出数据,例如`Pretty`格式,它只提供`IBlockOutputStream`。还有输入/输出格式,如`TabSeparated`或`JSONEachRow`。 + +还有行流:`IRowInputStream`和`IRowOutputStream`。它们允许您按单行而不是按块拉/推数据。它们仅用于简化面向行格式的实现。包装器`BlockInputStreamFromRowInputStream`和`BlockOutputStreamFromRowOutputStream`允许您将面向行的流转换为常规的面向块的流。 ## I/O {#io} -对于面向字节的输入输出,有 `ReadBuffer` 和 `WriteBuffer` 这两个抽象类。它们用来替代 C++ 的 `iostream`。不用担心:每个成熟的 C++ 项目都会有充分的理由使用某些东西来代替 `iostream`。 +对于面向字节的输入/输出,有`ReadBuffer`和`WriteBuffer`抽象类。它们被用来代替C++的`iostream`。别担心:每个成熟的C++项目都有充分的理由使用`iostream`以外的东西。 -`ReadBuffer` 和 `WriteBuffer` 由一个连续的缓冲区和指向缓冲区中某个位置的一个指针组成。实现中,缓冲区可能拥有内存,也可能不拥有内存。有一个虚方法会使用随后的数据来填充缓冲区(针对 `ReadBuffer`)或刷新缓冲区(针对 `WriteBuffer`),该虚方法很少被调用。 +`ReadBuffer`和`WriteBuffer`只是一个连续的缓冲区和一个指向该缓冲区中位置的游标。实现可能拥有或不拥有缓冲区的内存。有一种虚拟方法可以用以下数据填充缓冲区(对于`ReadBuffer`)或在某处刷新缓冲区(对于`WriteBuffer`)。虚拟方法很少被调用。 -`ReadBuffer` 和 `WriteBuffer` 的实现用于处理文件、文件描述符和网络套接字(socket),也用于实现压缩(`CompressedWriteBuffer` 在写入数据前需要先用一个 `WriteBuffer` 进行初始化并进行压缩)和其它用途。`ConcatReadBuffer`、`LimitReadBuffer` 和 `HashingWriteBuffer` 的用途正如其名字所描述的一样。 +`ReadBuffer`/`WriteBuffer`的实现用于处理文件和文件描述符和sockets,用于实现压缩(`CompressedWriteBuffer`用另一个`WriteBuffer`初始化并在向其写入数据之前执行压缩),以及用于其他的——`ConcatReadBuffer`、`LimitReadBuffer`和`HashingWriteBuffer`。 -`ReadBuffer` 和 `WriteBuffer` 仅处理字节。为了实现格式化输入和输出(比如以十进制格式写一个数字),`ReadHelpers` 和 `WriteHelpers` 头文件中有一些辅助函数可用。 +Read/WriteBuffers只处理字节。`ReadHelpers`和`WriteHelpers`头文件中的函数可以帮助格式化输入/输出。例如,有一些助手可以用十进制格式写一个数字。 -让我们来看一下,当你把一个结果集以 `JSON` 格式写到标准输出(stdout)时会发生什么。你已经准备好从 `IBlockInputStream` 获取结果集,然后创建 `WriteBufferFromFileDescriptor(STDOUT_FILENO)` 用于写字节到标准输出,创建 `JSONRowOutputStream` 并用 `WriteBuffer` 初始化,用于将行以 `JSON` 格式写到标准输出,你还可以在其上创建 `BlockOutputStreamFromRowOutputStream`,将其表示为 `IBlockOutputStream`。然后调用 `copyData` 将数据从 `IBlockInputStream` 传输到 `IBlockOutputStream`,一切工作正常。在内部,`JSONRowOutputStream` 会写入 JSON 分隔符,并以指向 `IColumn` 的引用和行数作为参数调用 `IDataType::serializeTextJSON` 函数。随后,`IDataType::serializeTextJSON` 将会调用 `WriteHelpers.h` 中的一个方法:比如,`writeText` 用于数值类型,`writeJSONString` 用于 `DataTypeString` 。 +让我们看看当您想将`JSON`格式的结果集写入`stdout`时会发生什么。您已准备好从`IBlockInputStream`中获取结果集。您创建`WriteBufferFromFileDescriptor(STDOUT_FILENO)`以将字节写入标准输出。你创建了`JSONRowOutputStream`,用`WriteBuffer`初始化,将`JSON`中的行写入标准输出。你在它上面创建了`BlockOutputStreamFromRowOutputStream`,将它表示为`IBlockOutputStream`。然后你调用`copyData`将数据从`IBlockInputStream`传输到`IBlockOutputStream`。在内部,`JSONRowOutputStream`将编写各种JSON分隔符,并使用对`IColumn`的引用和行号作为参数调用`IDataType::serializeTextJSON`方法。因此,`IDataType::serializeTextJSON`将从`WriteHelpers.h`调用一个方法:例如,`writeText`用于数字类型,`writeJSONString`用于`DataTypeString`。 -## 表(Tables) {#biao-tables} +## 数据表 {#tables} -表由 `IStorage` 接口表示。该接口的不同实现对应不同的表引擎。比如 `StorageMergeTree`、`StorageMemory` 等。这些类的实例就是表。 +`IStorage`接口代表表。该接口的不同实现是不同的表引擎。例如`StorageMergeTree`、`StorageMemory`等。这些类的实例只是表。 -`IStorage` 中最重要的方法是 `read` 和 `write`,除此之外还有 `alter`、`rename` 和 `drop` 等方法。`read` 方法接受如下参数:需要从表中读取的列集,需要执行的 `AST` 查询,以及所需返回的流的数量。`read` 方法的返回值是一个或多个 `IBlockInputStream` 对象,以及在查询执行期间在一个表引擎内完成的关于数据处理阶段的信息。 +`IStorage`的关键方法是`read`, `write`。还有`alter`、`rename`、`drop`等等。`read`方法接受以下参数:要从表中读取的列集、要考虑的`AST`查询以及要返回的所需流数。它返回一个或多个`IBlockInputStream`对象和关于在查询执行期间在表引擎中完成的数据处理阶段的信息。 -在大多数情况下,`read` 方法仅负责从表中读取指定的列,而不会进行进一步的数据处理。进一步的数据处理均由查询解释器完成,不由 `IStorage` 负责。 +在大多数情况下,read方法只负责从表中读取指定的列,而不负责任何进一步的数据处理。所有进一步的数据处理都是由查询解释器完成的,并且不在`IStorage`的职责范围内。 -但是也有值得注意的例外: +但也有明显的例外: -- AST 查询被传递给 `read` 方法,表引擎可以使用它来判断是否能够使用索引,从而从表中读取更少的数据。 -- 有时候,表引擎能够将数据处理到一个特定阶段。比如,`StorageDistributed` 可以向远程服务器发送查询,要求它们将来自不同的远程服务器能够合并的数据处理到某个阶段,并返回预处理后的数据,然后查询解释器完成后续的数据处理。 +- AST查询被传递给`read`方法,表引擎可以使用它得出索引使用情况,并从表中读取更少的数据。 +- 有时表引擎可以自己处理数据到特定的阶段。例如,`StorageDistributed`可以向远程服务器发送一个查询,要求它们将数据处理到一个可以合并来自不同远程服务器的数据的阶段,并返回预处理的数据。然后查询解释器完成数据的处理。 -表的 `read` 方法能够返回多个 `IBlockInputStream` 对象以允许并行处理数据。多个块输入流能够从一个表中并行读取。然后你可以通过不同的转换对这些流进行装饰(比如表达式求值或过滤),转换过程能够独立计算,并在其上创建一个 `UnionBlockInputStream`,以并行读取多个流。 +`read`方法可以返回多个`IBlockInputStream`对象以允许并行数据处理。这些多个块输入流可以并行地从表中读取数据。然后你可以用各种可以独立计算的转换(比如表达式求值或过滤)包装这些流,并在它们上面创建一个`UnionBlockInputStream`,以并行地从多个流读取。 -另外也有 `TableFunction`。`TableFunction` 能够在查询的 `FROM` 字句中返回一个临时的 `IStorage` 以供使用。 +还有`TableFunction`。这些函数返回一个临时的`IStorage`对象,用于查询的`FROM`子句。 -要快速了解如何实现自己的表引擎,可以查看一些简单的表引擎,比如 `StorageMemory` 或 `StorageTinyLog`。 +要快速了解如何实现表引擎,可以看一些简单的东西,比如`StorageMemory`或`StorageTinyLog`。 -> 作为 `read` 方法的结果,`IStorage` 返回 `QueryProcessingStage` - 关于 storage 里哪部分查询已经被计算的信息。当前我们仅有非常粗粒度的信息。Storage 无法告诉我们«对于这个范围的数据,我已经处理完了 WHERE 字句里的这部分表达式»。我们需要在这个地方继续努力。 +> 作为`read`方法的结果,`IStorage`返回`QueryProcessingStage` —— 关于查询的哪些部分已经在存储中计算过的信息。 -## 解析器(Parsers) {#jie-xi-qi-parsers} +## 解析 {#parsers} -查询由一个手写递归下降解析器解析。比如, `ParserSelectQuery` 只是针对查询的不同部分递归地调用下层解析器。解析器创建 `AST`。`AST` 由节点表示,节点是 `IAST` 的实例。 +手写的递归解析器用于解析一个查询。例如,`ParserSelectQuery`只是为查询的各个部分递归调用底层解析器。解析器创建一个`AST`。 `AST`由节点表示,节点是`IAST`的实例。 -> 由于历史原因,未使用解析器生成器。 +> 由于历史原因,不使用解析生成器。 -## 解释器(Interpreters) {#jie-shi-qi-interpreters} +## 解释器 {#interpreters} -解释器负责从 `AST` 创建查询执行流水线。既有一些简单的解释器,如 `InterpreterExistsQuery` 和 `InterpreterDropQuery`,也有更复杂的解释器,如 `InterpreterSelectQuery`。查询执行流水线由块输入或输出流组成。比如,`SELECT` 查询的解释结果是从 `FROM` 字句的结果集中读取数据的 `IBlockInputStream`;`INSERT` 查询的结果是写入需要插入的数据的 `IBlockOutputStream`;`SELECT INSERT` 查询的解释结果是 `IBlockInputStream`,它在第一次读取时返回一个空结果集,同时将数据从 `SELECT` 复制到 `INSERT`。 +解析器负责从`AST`创建查询执行管道。简单的解析器,例如`InterpreterExistsQuery`和`InterpreterDropQuery`,或者更复杂的`InterpreterSelectQuery`。查询执行管道是块输入或输出流的组合。例如,解析`SELECT`查询的结果是从`IBlockInputStream`读取结果集; INSERT查询的结果是`IBlockOutputStream`写入数据以进行插入,解析`INSERT SELECT`查询的结果是`IBlockInputStream`,它在第一次读取时返回一个空结果集,但从`SELECT`到`INSERT`同时进行。 -`InterpreterSelectQuery` 使用 `ExpressionAnalyzer` 和 `ExpressionActions` 机制来进行查询分析和转换。这是大多数基于规则的查询优化完成的地方。`ExpressionAnalyzer` 非常混乱,应该进行重写:不同的查询转换和优化应该被提取出来并划分成不同的类,从而允许模块化转换或查询。 +`InterpreterSelectQuery`使用`ExpressionAnalyzer`和`ExpressionActions`机制进行查询分析和转换。这是大多数基于规则的查询优化完成的地方。`ExpressionAnalyzer`非常混乱,应该重写:应该将各种查询转换和优化提取到单独的类中,以允许对查询进行模块化转换。 -## 函数(Functions) {#han-shu-functions} +## 函数 {#functions} -函数既有普通函数,也有聚合函数。对于聚合函数,请看下一节。 +有普通函数和聚合函数。有关聚合函数,请参见下一节。 -普通函数不会改变行数 - 它们的执行看起来就像是独立地处理每一行数据。实际上,函数不会作用于一个单独的行上,而是作用在以 `Block` 为单位的数据上,以实现向量查询执行。 +普通函数不会改变行数——它们的工作方式就像独立处理每一行一样。事实上,函数不是针对单个行调用的,而是针对数据的`Block`来实现向量化的查询执行。 -还有一些杂项函数,比如 [块大小](../sql-reference/functions/other-functions.md#function-blocksize)、[rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock),以及 [跑累积](../sql-reference/functions/other-functions.md#function-runningaccumulate),它们对块进行处理,并且不遵从行的独立性。 +还有一些其他的函数,如[blockSize](../sql-reference/functions/other-functions.md#function-blocksize), [rowNumberInBlock](../sql-reference/functions/other-functions.md#function-rownumberinblock), [runningAccumulate](../sql-reference/functions/other-functions.md#runningaccumulate), 利用块处理并违反行独立性。 -ClickHouse 具有强类型,因此隐式类型转换不会发生。如果函数不支持某个特定的类型组合,则会抛出异常。但函数可以通过重载以支持许多不同的类型组合。比如,`plus` 函数(用于实现 `+` 运算符)支持任意数字类型的组合:`UInt8` + `Float32`,`UInt16` + `Int8` 等。同时,一些可变参数的函数能够级接收任意数目的参数,比如 `concat` 函数。 +ClickHouse具有强类型,因此不存在隐式类型转换。如果函数不支持特定类型的组合,它会抛出异常。但是对于许多不同类型的组合,函数可以工作(重载)。例如,`plus`函数(实现`+`操作符)适用于任何数字类型的组合:`UInt8` + `Float32`, `UInt16` + `Int8`,等等。此外,一些可变参数函数可以接受任意数量的参数,例如`concat`函数。 -实现函数可能有些不方便,因为函数的实现需要包含所有支持该操作的数据类型和 `IColumn` 类型。比如,`plus` 函数能够利用 C++ 模板针对不同的数字类型组合、常量以及非常量的左值和右值进行代码生成。 +实现函数可能有点不方便,因为函数显式地分派支持的数据类型和支持的`IColumns`。例如,`plus`函数的代码是通过实例化一个c++模板生成的,该模板对应于数字类型的每个组合,以及常量或非常量左右参数。 -> 这是一个实现动态代码生成的好地方,从而能够避免模板代码膨胀。同样,运行时代码生成也使得实现融合函数成为可能,比如融合«乘-加»,或者在单层循环迭代中进行多重比较。 +它是实现运行时代码生成以避免模板代码膨胀的绝佳场所。此外,它还可以添加融合函数(如融合乘法-加法)或在一个循环迭代中进行多次比较。 -由于向量查询执行,函数不会«短路»。比如,如果你写 `WHERE f(x) AND g(y)`,两边都会进行计算,即使是对于 `f(x)` 为 0 的行(除非 `f(x)` 是零常量表达式)。但是如果 `f(x)` 的选择条件很高,并且计算 `f(x)` 比计算 `g(y)` 要划算得多,那么最好进行多遍计算:首先计算 `f(x)`,根据计算结果对列数据进行过滤,然后计算 `g(y)`,之后只需对较小数量的数据进行过滤。 +由于执行向量化查询,函数不会发生短路。例如,如果你写`WHERE f(x) AND g(y)`,当`f(x)`为零时(除了`f(x)`是零常量表达式),即使是行,两边都要计算。但如果`f(x)`条件的选择性很高,而`f(x)`的计算比`g(y)`要便宜得多,那么最好实现多遍计算。它首先计算`f(x)`,然后根据结果过滤列,然后只对较小的、过滤过的数据块计算`g(y)`。 -## 聚合函数 {#ju-he-han-shu} +## 聚合函数 {#aggregate-functions} -聚合函数是状态函数。它们将传入的值激活到某个状态,并允许你从该状态获取结果。聚合函数使用 `IAggregateFunction` 接口进行管理。状态可以非常简单(`AggregateFunctionCount` 的状态只是一个单一的`UInt64` 值),也可以非常复杂(`AggregateFunctionUniqCombined` 的状态是由一个线性数组、一个散列表和一个 `HyperLogLog` 概率数据结构组合而成的)。 +聚合函数是有状态函数。它们将传递的值积累到某个状态,并允许您从该状态获得结果。它们是通过`IAggregateFunction`接口管理的。状态可以很简单(`AggregateFunctionCount`的状态只是一个`UInt64`值),也可以很复杂(`AggregateFunctionUniqCombined`的状态是线性数组、哈希表和`HyperLogLog`概率数据结构的组合)。 -为了能够在执行一个基数很大的 `GROUP BY` 查询时处理多个聚合状态,需要在 `Arena`(一个内存池)或任何合适的内存块中分配状态。状态可以有一个非平凡的构造器和析构器:比如,复杂的聚合状态能够自己分配额外的内存。这需要注意状态的创建和销毁并恰当地传递状态的所有权,以跟踪谁将何时销毁状态。 +状态在`Arena`(内存池)中分配,以在执行高基数的`GROUP BY`查询时处理多个状态。状态可以有一个非常重要的构造函数和析构函数:例如,复杂的聚合状态可以自己分配额外的内存。它需要注意创建和销毁状态并正确传递它们的所有权和销毁顺序。 -聚合状态可以被序列化和反序列化,以在分布式查询执行期间通过网络传递或者在内存不够的时候将其写到硬盘。聚合状态甚至可以通过 `DataTypeAggregateFunction` 存储到一个表中,以允许数据的增量聚合。 +聚合状态可以被序列化和反序列化以在分布式查询执行期间通过网络传递或将它们写入没有足够RAM的磁盘上。它们甚至可以存储在带有`DataTypeAggregateFunction`的表中,以允许数据的增量聚合。 -> 聚合函数状态的序列化数据格式目前尚未版本化。如果只是临时存储聚合状态,这样是可以的。但是我们有 `AggregatingMergeTree` 表引擎用于增量聚合,并且人们已经在生产中使用它。这就是为什么在未来当我们更改任何聚合函数的序列化格式时需要增加向后兼容的支持。 +> 聚合函数状态的序列化数据格式现在没有版本化。如果聚合状态只是临时存储,则可以。但是我们有用于增量聚合的`AggregatingMergeTree`表引擎,并且人们已经在生产中使用它。这就是将来在更改任何聚合函数的序列化格式时需要向后兼容的原因。 -## 服务器(Server) {#fu-wu-qi-server} +## 服务 {#server} -服务器实现了多个不同的接口: +服务器实现了几个不同的接口: -- 一个用于任何外部客户端的 HTTP 接口。 -- 一个用于本机 ClickHouse 客户端以及在分布式查询执行中跨服务器通信的 TCP 接口。 -- 一个用于传输数据以进行拷贝的接口。 +- 任何外部客户端HTTP接口。 +- 本地ClickHouse客户端和分布式查询执行期间跨服务器通信的TCP接口。 +- 用于传输数据以进行复制的接口。 -在内部,它只是一个没有协程、纤程等的基础多线程服务器。服务器不是为处理高速率的简单查询设计的,而是为处理相对低速率的复杂查询设计的,每一个复杂查询能够对大量的数据进行处理分析。 +在内部,它只是一个没有协程或纤程的原始多线程服务器。由于服务器的设计目的不是处理高速率的简单查询,而是处理相对较低速率的复杂查询,因此每一个服务器都可以处理大量数据进行分析。 -服务器使用必要的查询执行需要的环境初始化 `Context` 类:可用数据库列表、用户和访问权限、设置、集群、进程列表和查询日志等。这些环境被解释器使用。 +服务器使用查询执行所需的环境初始化`Context`类:可用数据库列表、用户和访问权限、设置、集群、进程列表、查询日志,等等。解释器使用这种环境。 -我们维护了服务器 TCP 协议的完全向后向前兼容性:旧客户端可以和新服务器通信,新客户端也可以和旧服务器通信。但是我们并不想永久维护它,我们将在大约一年后删除对旧版本的支持。 +我们维护服务器TCP协议的完全向后和向前兼容性:旧客户端可以与新服务器通信,新客户端可以与旧服务器通信。但是我们不想永远维护它,并且在大约一年后我们会移除对旧版本的支持。 -> 对于所有的外部应用,我们推荐使用 HTTP 接口,因为该接口很简单,容易使用。TCP 接口与内部数据结构的联系更加紧密:它使用内部格式传递数据块,并使用自定义帧来压缩数据。我们没有发布该协议的 C 库,因为它需要链接大部分的 ClickHouse 代码库,这是不切实际的。 +!!! note "注意" +对于大多数外部应用程序,我们建议使用HTTP接口,因为它简单易用。TCP协议与内部数据结构的链接更加紧密:它使用内部格式传递数据块,并使用自定义的帧来压缩数据。我们还没有为该协议发布一个C库,因为它需要链接大多数ClickHouse代码库,这是不实际的。 -## 分布式查询执行 {#fen-bu-shi-cha-xun-zhi-xing} +## 分布式查询 {#distributed-query-execution} -集群设置中的服务器大多是独立的。你可以在一个集群中的一个或多个服务器上创建一个 `Distributed` 表。`Distributed` 表本身并不存储数据,它只为集群的多个节点上的所有本地表提供一个«视图(view)»。当从 `Distributed` 表中进行 SELECT 时,它会重写该查询,根据负载平衡设置来选择远程节点,并将查询发送给节点。`Distributed` 表请求远程服务器处理查询,直到可以合并来自不同服务器的中间结果的阶段。然后它接收中间结果并进行合并。分布式表会尝试将尽可能多的工作分配给远程服务器,并且不会通过网络发送太多的中间数据。 +集群设置中的服务器大多是独立的。您可以在集群中的一个或所有服务器上创建一个`Distributed`表。`Distributed`表本身并不存储数据——它只提供一个视图,以查看集群中多个节点上的所有本地表。当您从`Distributed`表中进行SELECT时,它会重写查询,根据负载均衡设置选择远程节点,并将查询发送给它们。`Distributed`表请求远程服务器处理一个查询,直到可以合并来自不同服务器的中间结果的阶段。然后它接收中间结果并合并它们。分布式表试图将尽可能多的工作分配到远程服务器,而不通过网络发送太多中间数据。 -> 当 `IN` 或 `JOIN` 子句中包含子查询并且每个子查询都使用分布式表时,事情会变得更加复杂。我们有不同的策略来执行这些查询。 +当在in或JOIN子句中有子查询,并且每个子查询都使用一个`Distributed`表时,情况会变得更加复杂。我们有不同的策略来执行这些查询。 -分布式查询执行没有全局查询计划。每个节点都有针对自己的工作部分的本地查询计划。我们仅有简单的一次性分布式查询执行:将查询发送给远程节点,然后合并结果。但是对于具有高基数的 `GROUP BY` 或具有大量临时数据的 `JOIN` 这样困难的查询的来说,这是不可行的:在这种情况下,我们需要在服务器之间«改组»数据,这需要额外的协调。ClickHouse 不支持这类查询执行,我们需要在这方面进行努力。 +分布式查询执行没有全局查询计划。每个节点都有其作业部分的本地查询计划。我们只有简单的一次执行分布式查询:我们向远程节点发送查询,然后合并结果。但是对于具有高基数GROUP by的复杂查询或具有大量用于JOIN的临时数据,这是不可行的。在这种情况下,我们需要在服务器之间重新清洗数据,这需要额外的协调。ClickHouse不支持这种查询执行,我们需要对它进行处理。 -## 合并树 {#merge-tree} +## Merge Tree {#merge-tree} -`MergeTree` 是一系列支持按主键索引的存储引擎。主键可以是一个任意的列或表达式的元组。`MergeTree` 表中的数据存储于«分块»中。每一个分块以主键序存储数据(数据按主键元组的字典序排序)。表的所有列都存储在这些«分块»中分离的 `column.bin` 文件中。`column.bin` 文件由压缩块组成,每一个块通常是 64 KB 到 1 MB 大小的未压缩数据,具体取决于平均值大小。这些块由一个接一个连续放置的列值组成。每一列的列值顺序相同(顺序由主键定义),因此当你按多列进行迭代时,你能够得到相应列的值。 +`MergeTree`是一组支持主键索引的存储引擎。主键可以是任意的列或表达式元组。`MergeTree`表中的数据存储在“parts”中。每个部分以主键顺序存储数据,因此数据按照字典顺序按主键元组排序。所有表列都存储在这些部分的`column.bin`文件中。文件由压缩块组成。每个块通常是64 KB到1 MB的未压缩数据,具体取决于平均值大小。块由一个接一个连续放置的列值组成。每个列的列值顺序相同(主键定义顺序),因此当您迭代多个列时,您将获得相应行的值。 -主键本身是«稀疏»的。它并不是索引单一的行,而是索引某个范围内的数据。一个单独的 `primary.idx` 文件具有每个第 N 行的主键值,其中 N 称为 `index_granularity`(通常,N = 8192)。同时,对于每一列,都有带有标记的 `column.mrk` 文件,该文件记录的是每个第 N 行在数据文件中的偏移量。每个标记是一个 pair:文件中的偏移量到压缩块的起始,以及解压缩块中的偏移量到数据的起始。通常,压缩块根据标记对齐,并且解压缩块中的偏移量为 0。`primary.idx` 的数据始终驻留在内存,同时 `column.mrk` 的数据被缓存。 +主键本身是“稀疏的”。它不处理每一行,而只处理某些范围的数据。一个单独的初选。`primary.idx`文件具有每个第N行的主键值,其中N被称为`index_granularity`(通常N = 8192)。同样,对于每一列,我们有`column.mrk`文件的“标记”,这是数据文件中每个第n行的偏移量。每个标记是一对:文件到压缩块开始的偏移量,以及解压缩块到数据开始的偏移量。通常,压缩块按标记对齐,解压缩块中的偏移量为零。`primary.idx`始终驻留在内存中,`column.mrk`列的数据文件被缓存。 -当我们要从 `MergeTree` 的一个分块中读取部分内容时,我们会查看 `primary.idx` 数据并查找可能包含所请求数据的范围,然后查看 `column.mrk` 并计算偏移量从而得知从哪里开始读取些范围的数据。由于稀疏性,可能会读取额外的数据。ClickHouse 不适用于高负载的简单点查询,因为对于每一个键,整个 `index_granularity` 范围的行的数据都需要读取,并且对于每一列需要解压缩整个压缩块。我们使索引稀疏,是因为每一个单一的服务器需要在索引没有明显内存消耗的情况下,维护数万亿行的数据。另外,由于主键是稀疏的,导致其不是唯一的:无法在 INSERT 时检查一个键在表中是否存在。你可以在一个表中使用同一个键创建多个行。 +当我们要从`MergeTree`中的一个部分读取一些东西时,我们会查看`primary.idx`数据并定位可能包含请求数据的范围,然后查看`column.mrk`数据并计算从哪里开始读取的偏移量那些范围。由于稀疏,可能会读取多余的数据。 ClickHouse不适合单点查询的高负载,因为必须为每个键读取带有`index_granularity`行的整个范围,并且必须为每一列解压整个压缩块。 我们使索引变得稀疏,因为我们必须能够在每台服务器上维护数万亿行而不会对索引造成明显的内存消耗。此外,由于主键是稀疏的,它不是唯一的:它无法在INSERT时检查表中键的存在。一个表中可以有许多具有相同键的行。 -当你向 `MergeTree` 中插入一堆数据时,数据按主键排序并形成一个新的分块。为了保证分块的数量相对较少,有后台线程定期选择一些分块并将它们合并成一个有序的分块,这就是 `MergeTree` 的名称来源。当然,合并会导致«写入放大»。所有的分块都是不可变的:它们仅会被创建和删除,不会被修改。当运行 `SELECT` 查询时,`MergeTree` 会保存一个表的快照(分块集合)。合并之后,还会保留旧的分块一段时间,以便发生故障后更容易恢复,因此如果我们发现某些合并后的分块可能已损坏,我们可以将其替换为原分块。 +当您将一组数据`INSERT`到`MergeTree`中时,该组数据按主键顺序排序并形成一个新部分。 有后台线程会定期选择一些部分并将它们合并为一个已排序的部分,以保持部分数量相对较少。 这就是它被称为`MergeTree`的原因。当然,合并会导致“写放大”。所有部分都是不可变的:它们只会被创建和删除,而不会被修改。 执行SELECT时,它保存表的快照(一组parts)。合并后,我们也会将旧部件保留一段时间,以便故障后更容易恢复,因此如果我们发现某些合并部件可能损坏了,我们可以将其替换为源parts。 -`MergeTree` 不是 LSM 树,因为它不包含»memtable«和»log«:插入的数据直接写入文件系统。这使得它仅适用于批量插入数据,而不适用于非常频繁地一行一行插入 - 大约每秒一次是没问题的,但是每秒一千次就会有问题。我们这样做是为了简单起见,因为我们已经在我们的应用中批量插入数据。 +`MergeTree`不是LSM树,因为它不包含“memtable”和“log”:插入的数据直接写入文件系统。这使得它仅适用于批量插入数据,而不是逐行插入,也不是很频繁——大约每秒一次就可以了,但每秒一千次就不行了。我们这样做是为了简单起见,因为我们已经在我们的应用程序中批量插入数据。 -> `MergeTree` 表只能有一个(主)索引:没有任何辅助索引。在一个逻辑表下,允许有多个物理表示,比如,可以以多个物理顺序存储数据,或者同时表示预聚合数据和原始数据。 +有MergeTree引擎在后台合并期间做额外的工作。例如`CollapsingMergeTree`和`AggregatingMergeTree`。这可以被视为对更新的特殊支持。 请记住,这些并不是真正的更新,因为用户通常无法控制执行后台合并的时间,并且`MergeTree`表中的数据几乎总是存储在多个part,而不是完全合并的形式。 -有些 `MergeTree` 引擎会在后台合并期间做一些额外工作,比如 `CollapsingMergeTree` 和 `AggregatingMergeTree`。这可以视为对更新的特殊支持。请记住这些不是真正的更新,因为用户通常无法控制后台合并将会执行的时间,并且 `MergeTree` 中的数据几乎总是存储在多个分块中,而不是完全合并的形式。 +## 复制 {#replication} -## 复制(Replication) {#fu-zhi-replication} +ClickHouse中的复制可以在每个表的基础上配置。在同一台服务器上可以有一些复制的表和一些非复制的表。还可以以不同的方式复制表,例如一个表具有两因素复制,另一个表具有三因素复制。 -ClickHouse 中的复制是基于表实现的。你可以在同一个服务器上有一些可复制的表和不可复制的表。你也可以以不同的方式进行表的复制,比如一个表进行双因子复制,另一个进行三因子复制。 +复制是在`ReplicatedMergeTree`存储引擎中实现的。其中`ZooKeeper`中的路径作为存储引擎的参数。所有在`ZooKeeper`中具有相同路径的表成为彼此的副本:它们同步它们的数据并保持一致性。只需创建或删除一个表,就可以动态地添加和删除副本。 -复制是在 `ReplicatedMergeTree` 存储引擎中实现的。`ZooKeeper` 中的路径被指定为存储引擎的参数。`ZooKeeper` 中所有具有相同路径的表互为副本:它们同步数据并保持一致性。只需创建或删除表,就可以实现动态添加或删除副本。 +复制使用异步多主机方案。您可以将数据插入到任何与`ZooKeeper`有会话的副本中,并将数据异步复制到所有其他副本中。因为ClickHouse不支持更新,所以复制是无冲突的。由于没有插入的仲裁确认,如果一个节点故障,刚刚插入的数据可能会丢失。 -复制使用异步多主机方案。你可以将数据插入到与 `ZooKeeper` 进行会话的任意副本中,并将数据复制到所有其它副本中。由于 ClickHouse 不支持 UPDATEs,因此复制是无冲突的。由于没有对插入的仲裁确认,如果一个节点发生故障,刚刚插入的数据可能会丢失。 +用于复制的元数据存储在ZooKeeper中。这里有一个复制日志,列出了要执行的操作。行动是:得到部分;merge parts;删除一个分区,等等。每个副本将复制日志复制到其队列,然后从队列执行操作。例如,在插入时,在日志中创建“获取part”操作,每个副本都下载该part。在副本之间协调合并以获得字节相同的结果。所有部分在所有副本上以相同的方式合并。其中一个领导者首先发起一个新的合并,并将“merge parts”操作写入日志。多个副本(或所有副本)可以同时是leader。可以使用`merge_tree`设置`replicated_can_become_leader`来阻止副本成为leader。leader负责安排后台合并。 -用于复制的元数据存储在 ZooKeeper 中。其中一个复制日志列出了要执行的操作。操作包括:获取分块、合并分块和删除分区等。每一个副本将复制日志复制到其队列中,然后执行队列中的操作。比如,在插入时,在复制日志中创建«获取分块»这一操作,然后每一个副本都会去下载该分块。所有副本之间会协调进行合并以获得相同字节的结果。所有的分块在所有的副本上以相同的方式合并。为实现该目的,其中一个副本被选为领导者,该副本首先进行合并,并把«合并分块»操作写到日志中。 +复制是物理的:只有压缩的部分在节点之间传输,而不是查询。在大多数情况下,合并是在每个副本上独立进行的,以避免网络扩大化,降低网络成本。较大的合并部分只在复制严重滞后的情况下通过网络发送。 -复制是物理的:只有压缩的分块会在节点之间传输,查询则不会。为了降低网络成本(避免网络放大),大多数情况下,会在每一个副本上独立地处理合并。只有在存在显著的合并延迟的情况下,才会通过网络发送大块的合并分块。 +此外,每个副本在ZooKeeper中以部件集及其校验和的形式存储自己的状态。当本地文件系统的状态与ZooKeeper中的引用状态偏离时,副本通过从其他副本中下载丢失和损坏的部分来恢复一致性。当本地文件系统中有一些意外或损坏的数据时,ClickHouse不会删除它,而是将它移动到一个单独的目录并忘记它。 -另外,每一个副本将其状态作为分块和校验和组成的集合存储在 ZooKeeper 中。当本地文件系统中的状态与 ZooKeeper 中引用的状态不同时,该副本会通过从其它副本下载缺失和损坏的分块来恢复其一致性。当本地文件系统中出现一些意外或损坏的数据时,ClickHouse 不会将其删除,而是将其移动到一个单独的目录下并忘记它。 +!!! note "注意" +ClickHouse集群由独立的分片组成,每个分片由副本组成。集群**无弹性**,因此添加新分片后,不会自动在分片之间重新平衡数据。相反,应该将集群负载调整为不均匀。这个实现给了你更多的控制权,对于相对较小的集群来说是可以的,比如几十个节点。但是对于我们在生产中使用的具有数百个节点的集群,这种方法成为一个明显的缺点。我们应该实现一个跨越集群的表引擎,动态复制的区域可以在集群之间自动分割和平衡。 -> ClickHouse 集群由独立的分片组成,每一个分片由多个副本组成。集群不是弹性的,因此在添加新的分片后,数据不会自动在分片之间重新平衡。相反,集群负载将变得不均衡。该实现为你提供了更多控制,对于相对较小的集群,例如只有数十个节点的集群来说是很好的。但是对于我们在生产中使用的具有数百个节点的集群来说,这种方法成为一个重大缺陷。我们应该实现一个表引擎,使得该引擎能够跨集群扩展数据,同时具有动态复制的区域,这些区域能够在集群之间自动拆分和平衡。 - -[来源文章](https://clickhouse.tech/docs/en/development/architecture/) +{## [原始文章](https://clickhouse.tech/docs/en/development/architecture/) ##} From a4100cc980f025825b45e05b2a9184bc828f7033 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 26 Oct 2021 17:45:41 +0000 Subject: [PATCH 118/609] Build local in CI in fuzz mode --- docker/packager/other/fuzzer.sh | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh index 5eec51f9625..708356f9b0a 100755 --- a/docker/packager/other/fuzzer.sh +++ b/docker/packager/other/fuzzer.sh @@ -31,5 +31,10 @@ do mv "$FUZZER_PATH" /output/fuzzers done +ninja clickhouse-local +LOCAL_PATH=$(find ./programs -name clickhouse) +strip --strip-unneeded "$LOCAL_PATH" +mv "$LOCAL_PATH" /output/fuzzers + tar -zcvf /output/fuzzers.tar.gz /output/fuzzers rm -rf /output/fuzzers From 538d89a7eb34a438a82d2a7a3f0e00c8caa44cae Mon Sep 17 00:00:00 2001 From: qianmoQ Date: Wed, 27 Oct 2021 20:27:28 +0800 Subject: [PATCH 119/609] pr continuous integration --- docs/en/development/continuous-integration.md | 247 ++++++++---------- 1 file changed, 115 insertions(+), 132 deletions(-) diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 0176a2dcb76..718433381ad 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -1,104 +1,92 @@ --- toc_priority: 62 -toc_title: Continuous Integration Checks +toc_title: 持续集成检查 --- -# Continuous Integration Checks +# 持续集成检查 -When you submit a pull request, some automated checks are ran for your code by -the ClickHouse [continuous integration (CI) system](tests.md#test-automation). -This happens after a repository maintainer (someone from ClickHouse team) has -screened your code and added the `can be tested` label to your pull request. -The results of the checks are listed on the GitHub pull request page as -described in the [GitHub checks -documentation](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/about-status-checks). -If a check is failing, you might be required to fix it. This page gives an -overview of checks you may encounter, and what you can do to fix them. +当您提交拉取请求时,ClickHouse 会为您的代码运行一些自动检查[continuous integration (CI) system](tests.md#test-automation). +这发生在存储库维护者(来自 ClickHouse 团队的某个人)筛选了您的代码并将`can be tested`标签添加到您的拉取请求之后。 +检查结果列在 GitHub 拉取请求页面上,如[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/about-status-checks)中所述. +如果检查失败,您可能需要修复它。 此页面概述了您可能遇到的检查,以及您可以采取哪些措施来解决这些检查。 + +如果检查失败看起来与您的更改无关,则可能是一些暂时性故障或基础架构问题。 +将空提交推送到拉取请求以重新启动 CI 检查: -If it looks like the check failure is not related to your changes, it may be -some transient failure or an infrastructure problem. Push an empty commit to -the pull request to restart the CI checks: ``` git reset git commit --allow-empty git push ``` -If you are not sure what to do, ask a maintainer for help. +如果您不确定该怎么做,请向维护人员寻求帮助。 + +## 合并到Master + +验证 PR 可以合并到 master。 如果没有,它将失败并显示消息'Cannot fetch mergecommit'。 +要修复此检查,请按照[GitHub文档](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/resolving-a-merge-conflict-on-github)中的说明解决冲突, +或者使用 git 将 `master` 分支合并到你的拉取请求分支。 + +## 文档检查 + +尝试构建 ClickHouse 文档网站。 如果您更改了文档中的某些内容,它可能会失败。 +最可能的原因是文档中的某些交叉链接是错误的。 +转到检查报告并查找`ERROR`和`WARNING`消息。 + +### 报告详情 + +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt` 包含构建日志。[成功结果示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) -## Merge With Master +## 描述检查 -Verifies that the PR can be merged to master. If not, it will fail with the -message 'Cannot fetch mergecommit'. To fix this check, resolve the conflict as -described in the [GitHub -documentation](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/resolving-a-merge-conflict-on-github), -or merge the `master` branch to your pull request branch using git. +检查您的拉取请求的描述是否符合模板 [PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md). +您必须为您的更改指定一个更改日志类别(例如,错误修复),并编写一条用户可读的消息来描述[CHANGELOG.md](../whats-new/changelog/index.md)的更改 -## Docs check +## 推送到 Dockerhub -Tries to build the ClickHouse documentation website. It can fail if you changed -something in the documentation. Most probable reason is that some cross-link in -the documentation is wrong. Go to the check report and look for `ERROR` and `WARNING` messages. - -### Report Details - -- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt` contains the building log. [Successful result example](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) +构建用于构建和测试的 docker 镜像,然后将它们推送到 DockerHub。 -## Description Check +## 标记检查 -Check that the description of your pull request conforms to the template -[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md). -You have to specify a changelog category for your change (e.g., Bug Fix), and -write a user-readable message describing the change for [CHANGELOG.md](../whats-new/changelog/index.md) +这个检查意味着 CI 系统开始处理拉取请求。 当它处于“待处理”状态时,意味着并非所有检查都已开始。 启动所有检查后,状态更改为“成功”。 -## Push To Dockerhub +## 格式检查 -Builds docker images used for build and tests, then pushes them to DockerHub. +使用 [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check -style) 二进制(注意它可以在本地运行)。 +如果失败,请按照 [code style guide](style.md) 修复样式错误。 -## Marker Check - -This check means that the CI system started to process the pull request. When it has 'pending' status, it means that not all checks have been started yet. After all checks have been started, it changes status to 'success'. +### 报告详情 +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) +- `output.txt` 包含检查结果错误(无效制表等),空白页表示没有错误。 [成功结果示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt). -## Style Check +## PVS检查 +使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码。 查看报告以查看确切的错误。 如果可以,请修复它们,如果不能,请向 ClickHouse 维护人员寻求帮助。 -Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). -If it fails, fix the style errors following the [code style guide](style.md). - -### Report Details -- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) -- `output.txt` contains the check resulting errors (invalid tabulation etc), blank page means no errors. [Successful result example](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt). +### 报告详情 +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) +- `test_run.txt.out.log` 包含构建和分析日志文件。 它仅包括解析或未找到的错误。 +- `HTML report` 包含分析结果。 有关其描述,请访问 PVS 的 [官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD). -## PVS Check -Check the code with [PVS-studio](https://www.viva64.com/en/pvs-studio/), a static analysis tool. Look at the report to see the exact errors. Fix them if you can, if not -- ask a ClickHouse maintainer for help. +## 快速测试 +通常这是为 PR 运行的第一次检查。 它构建 ClickHouse 并运行大部分 [无状态功能测试](tests.md#functional-tests),省略了一些额外操作。 +如果失败,则在修复之前不会开始进一步检查。 +查看报告以了解哪些测试失败,然后按照[此处](tests.md#functional-test-locally)所述在本地重现失败. -### Report Details -- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) -- `test_run.txt.out.log` contains the building and analyzing log file. It includes only parsing or not-found errors. -- `HTML report` contains the analysis results. For its description visit PVS's [official site](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD). +### 报告详情 +[状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) - -## Fast Test -Normally this is the first check that is ran for a PR. It builds ClickHouse and -runs most of [stateless functional tests](tests.md#functional-tests), omitting -some. If it fails, further checks are not started until it is fixed. Look at -the report to see which tests fail, then reproduce the failure locally as -described [here](tests.md#functional-test-locally). - -### Report Details -[Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) - -#### Status Page Files -- `runlog.out.log` is the general log that includes all other logs. +#### 状态页面文件 +- `runlog.out.log` 包含所有的通用日志。 - `test_log.txt` -- `submodule_log.txt` contains the messages about cloning and checkouting needed submodules. +- `submodule_log.txt` 包含有关克隆和检出所需子模块的消息。 - `stderr.log` - `stdout.log` - `clickhouse-server.log` @@ -106,110 +94,105 @@ described [here](tests.md#functional-test-locally). - `install_log.txt` - `clickhouse-server.err.log` - `build_log.txt` -- `cmake_log.txt` contains messages about the C/C++ and Linux flags check. +- `cmake_log.txt` 包含有关 C/C++ 和 Linux 标志检查的消息。 -#### Status Page Columns +#### 状态页面列 -- *Test name* contains the name of the test (without the path e.g. all types of tests will be stripped to the name). -- *Test status* -- one of _Skipped_, _Success_, or _Fail_. -- *Test time, sec.* -- empty on this test. +- *Test name* 包含测试的名称(没有路径,例如所有类型的测试都将被剥离为名称)。 +- *测试状态* -- _Skipped_、_Success_ 或_Fail_ 之一。 +- *测试时间,秒* - 本次测试为空。 -## Build Check {#build-check} +## 构建检查 {#build-check} -Builds ClickHouse in various configurations for use in further steps. You have to fix the builds that fail. Build logs often has enough information to fix the error, but you might have to reproduce the failure locally. The `cmake` options can be found in the build log, grepping for `cmake`. Use these options and follow the [general build process](../development/build.md). +各种配置构建 ClickHouse,以用于进一步的步骤。 您必须修复失败的构建。 构建日志通常有足够的信息来修复错误,但您可能必须在本地重现故障。 `cmake` 选项可以在构建日志中找到,搜索 `cmake`。 使用这些选项并遵循 [构建过程](../development/build.md)。 -### Report Details +### 报告详情 -[Status page example](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html). +[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html). -- **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`). +- **Compiler**: `gcc-9` 或 `clang-10`(或其他架构的 `clang-10-xx`,例如 `clang-10-freebsd`)。 - **Build type**: `Debug` or `RelWithDebInfo` (cmake). -- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). -- **Bundled**: `bundled` build uses libraries from `contrib` folder, and `unbundled` build uses system libraries. -- **Splitted** `splitted` is a [split build](../development/build.md#split-build) -- **Status**: `success` or `fail` -- **Build log**: link to the building and files copying log, useful when build failed. +- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), 或 `thread` (TSan). +- **Bundled**: `bundled` 构建使用来自 `contrib` 文件夹的库,`unbundled` 构建使用系统库。 +- **Splitted** `splitted` [split build](../development/build.md#split-build) +- **Status**: `success` 或 `fail` +- **Build log**: 构建和文件复制日志,在构建失败时很有用。 - **Build time**. -- **Artifacts**: build result files (with `XXX` being the server version e.g. `20.8.1.4344`). - - `clickhouse-client_XXX_all.deb` - - `clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` - - `clickhouse-common-staticXXX_amd64.deb` - - `clickhouse-server_XXX_all.deb` - - `clickhouse-test_XXX_all.deb` - - `clickhouse_XXX_amd64.buildinfo` - - `clickhouse_XXX_amd64.changes` - - `clickhouse`: Main built binary. - - `clickhouse-odbc-bridge` - - `unit_tests_dbms`: GoogleTest binary with ClickHouse unit tests. - - `shared_build.tgz`: build with shared libraries. - - `performance.tgz`: Special package for performance tests. +- **Artifacts**: 构建结果文件(`XXX`是服务器版本,例如`20.8.1.4344`)。 + - `clickhouse-client_XXX_all.deb` + - `clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` + - `clickhouse-common-staticXXX_amd64.deb` + - `clickhouse-server_XXX_all.deb` + - `clickhouse-test_XXX_all.deb` + - `clickhouse_XXX_amd64.buildinfo` + - `clickhouse_XXX_amd64.changes` + - `clickhouse`: 主要构建的二进制文件。 + - `clickhouse-odbc-bridge` + - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件。 + - `shared_build.tgz`: 使用共享库构建。 + - `performance.tgz`: 用于性能测试的特殊包。 -## Special Build Check -Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. +## 特殊构建检查 +使用 `clang-tidy` 执行静态分析和代码样式检查。 该报告类似于[构建检查](#build-check)。 修复构建日志中发现的错误。 -## Functional Stateless Tests -Runs [stateless functional tests](tests.md#functional-tests) for ClickHouse -binaries built in various configurations -- release, debug, with sanitizers, -etc. Look at the report to see which tests fail, then reproduce the failure -locally as described [here](tests.md#functional-test-locally). Note that you -have to use the correct build configuration to reproduce -- a test might fail -under AddressSanitizer but pass in Debug. Download the binary from [CI build -checks page](../development/build.md#you-dont-have-to-build-clickhouse), or build it locally. +## 功能无状态测试 +在各种配置中构建的 ClickHouse 二进制文件运行 [无状态功能测试](tests.md#functional-tests) -- 发布、调试、使用等。 +查看报告以查看哪些测试失败,然后按照 [此处](tests.md#functional-test-locally) 的描述在本地重现失败。 +请注意,您必须使用正确的构建配置来重现 -- 在 AddressSanitizer 下测试可能会失败,但在 Debug 中通过。 +从 [CI 构建检查页面](../development/build.md#you-dont-have-to-build-clickhouse) 下载二进制文件,或在本地构建它。 -## Functional Stateful Tests -Runs [stateful functional tests](tests.md#functional-tests). Treat them in the same way as the functional stateless tests. The difference is that they require `hits` and `visits` tables from the [Yandex.Metrica dataset](../getting-started/example-datasets/metrica.md) to run. +## 功能状态测试 +运行 [状态功能测试](tests.md#functional-tests)。 与功能无状态测试相同的方式。不同之处在于它们需要来自 [Yandex.Metrica 数据集](../getting-started/example-datasets/metrica.md) 的 `hits` 和 `visits` 表才能运行。 -## Integration Tests -Runs [integration tests](tests.md#integration-tests). +## 集成测试 +运行 [integration tests](tests.md#integration-tests). -## Testflows Check -Runs some tests using Testflows test system. See [here](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally) how to run them locally. +## 测试流程检查 +使用 Testflows 系统运行一些测试。 请参阅[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们。 -## Stress Test -Runs stateless functional tests concurrently from several clients to detect -concurrency-related errors. If it fails: +## 压力测试 +从多个客户端同时运行无状态功能测试以检测与并发相关的错误。 +如果失败: - * Fix all other test failures first; - * Look at the report to find the server logs and check them for possible causes - of error. + * 首先修复所有失败; + * 查看报告以查找服务器日志并检查它们是否可能导致错误。 -## Split Build Smoke Test +## 分布式测试 -Checks that the server build in [split build](../development/build.md#split-build) -configuration can start and run simple queries. If it fails: +检查[split build](../development/build.md#split-build)配置中的服务器构建可以启动和运行简单查询。 +如果失败: - * Fix other test errors first; - * Build the server in [split build](../development/build.md#split-build) configuration - locally and check whether it can start and run `select 1`. + * 首先修复所有失败; + * 在本地以 [split build](../development/build.md#split-build) 配置构建服务器并检查它是否可以启动并运行`select 1`。 -## Compatibility Check -Checks that `clickhouse` binary runs on distributions with old libc versions. If it fails, ask a maintainer for help. +## 兼容性检查 +检查 `clickhouse` 二进制文件是否在具有旧 libc 版本的发行版上运行。 如果失败,请向维护人员寻求帮助。 ## AST Fuzzer -Runs randomly generated queries to catch program errors. If it fails, ask a maintainer for help. +运行随机生成的查询以捕获程序错误。 如果失败,请向维护人员寻求帮助。 -## Performance Tests -Measure changes in query performance. This is the longest check that takes just below 6 hours to run. The performance test report is described in detail [here](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report). +## 性能测试 +测量查询性能的变化。 这是最长的检查,只需不到 6 小时即可运行。 性能测试报告详细描述[这里](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report)。 # QA -> What is a `Task (private network)` item on status pages? +> 什么是状态页面上的`Task (private network)`项目? -It's a link to the Yandex's internal job system. Yandex employees can see the check's start time and its more verbose status. +它是 Yandex 内部工作系统的链接。 Yandex 员工可以看到它的开始时间及其更详细的状态。 -> Where the tests are run +> 运行测试的地方 -Somewhere on Yandex internal infrastructure. +Yandex 内部基础设施的某个地方。 From c04510d86401deec6d34321b20ee63b76fbd2512 Mon Sep 17 00:00:00 2001 From: qianmoQ Date: Mon, 1 Nov 2021 13:09:11 +0800 Subject: [PATCH 120/609] Resolve conversation --- docs/en/development/continuous-integration.md | 233 ++++++++-------- docs/zh/development/continuous-integration.md | 249 ++++++++++-------- 2 files changed, 270 insertions(+), 212 deletions(-) diff --git a/docs/en/development/continuous-integration.md b/docs/en/development/continuous-integration.md index 718433381ad..ad383fb8888 100644 --- a/docs/en/development/continuous-integration.md +++ b/docs/en/development/continuous-integration.md @@ -1,92 +1,104 @@ --- toc_priority: 62 -toc_title: 持续集成检查 +toc_title: Continuous Integration Checks --- -# 持续集成检查 +# Continuous Integration Checks -当您提交拉取请求时,ClickHouse 会为您的代码运行一些自动检查[continuous integration (CI) system](tests.md#test-automation). -这发生在存储库维护者(来自 ClickHouse 团队的某个人)筛选了您的代码并将`can be tested`标签添加到您的拉取请求之后。 -检查结果列在 GitHub 拉取请求页面上,如[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/about-status-checks)中所述. -如果检查失败,您可能需要修复它。 此页面概述了您可能遇到的检查,以及您可以采取哪些措施来解决这些检查。 - -如果检查失败看起来与您的更改无关,则可能是一些暂时性故障或基础架构问题。 -将空提交推送到拉取请求以重新启动 CI 检查: +When you submit a pull request, some automated checks are ran for your code by +the ClickHouse [continuous integration (CI) system](tests.md#test-automation). +This happens after a repository maintainer (someone from ClickHouse team) has +screened your code and added the `can be tested` label to your pull request. +The results of the checks are listed on the GitHub pull request page as +described in the [GitHub checks +documentation](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/about-status-checks). +If a check is failing, you might be required to fix it. This page gives an +overview of checks you may encounter, and what you can do to fix them. +If it looks like the check failure is not related to your changes, it may be +some transient failure or an infrastructure problem. Push an empty commit to +the pull request to restart the CI checks: ``` git reset git commit --allow-empty git push ``` -如果您不确定该怎么做,请向维护人员寻求帮助。 - -## 合并到Master - -验证 PR 可以合并到 master。 如果没有,它将失败并显示消息'Cannot fetch mergecommit'。 -要修复此检查,请按照[GitHub文档](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/resolving-a-merge-conflict-on-github)中的说明解决冲突, -或者使用 git 将 `master` 分支合并到你的拉取请求分支。 - -## 文档检查 - -尝试构建 ClickHouse 文档网站。 如果您更改了文档中的某些内容,它可能会失败。 -最可能的原因是文档中的某些交叉链接是错误的。 -转到检查报告并查找`ERROR`和`WARNING`消息。 - -### 报告详情 - -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt` 包含构建日志。[成功结果示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) +If you are not sure what to do, ask a maintainer for help. -## 描述检查 +## Merge With Master -检查您的拉取请求的描述是否符合模板 [PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md). -您必须为您的更改指定一个更改日志类别(例如,错误修复),并编写一条用户可读的消息来描述[CHANGELOG.md](../whats-new/changelog/index.md)的更改 +Verifies that the PR can be merged to master. If not, it will fail with the +message 'Cannot fetch mergecommit'. To fix this check, resolve the conflict as +described in the [GitHub +documentation](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/resolving-a-merge-conflict-on-github), +or merge the `master` branch to your pull request branch using git. -## 推送到 Dockerhub +## Docs check -构建用于构建和测试的 docker 镜像,然后将它们推送到 DockerHub。 +Tries to build the ClickHouse documentation website. It can fail if you changed +something in the documentation. Most probable reason is that some cross-link in +the documentation is wrong. Go to the check report and look for `ERROR` and `WARNING` messages. + +### Report Details + +- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt` contains the building log. [Successful result example](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) -## 标记检查 +## Description Check -这个检查意味着 CI 系统开始处理拉取请求。 当它处于“待处理”状态时,意味着并非所有检查都已开始。 启动所有检查后,状态更改为“成功”。 +Check that the description of your pull request conforms to the template +[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md). +You have to specify a changelog category for your change (e.g., Bug Fix), and +write a user-readable message describing the change for [CHANGELOG.md](../whats-new/changelog/index.md) -## 格式检查 +## Push To Dockerhub -使用 [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check -style) 二进制(注意它可以在本地运行)。 -如果失败,请按照 [code style guide](style.md) 修复样式错误。 +Builds docker images used for build and tests, then pushes them to DockerHub. -### 报告详情 -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) -- `output.txt` 包含检查结果错误(无效制表等),空白页表示没有错误。 [成功结果示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt). +## Marker Check + +This check means that the CI system started to process the pull request. When it has 'pending' status, it means that not all checks have been started yet. After all checks have been started, it changes status to 'success'. -## PVS检查 -使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码。 查看报告以查看确切的错误。 如果可以,请修复它们,如果不能,请向 ClickHouse 维护人员寻求帮助。 +## Style Check -### 报告详情 -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) -- `test_run.txt.out.log` 包含构建和分析日志文件。 它仅包括解析或未找到的错误。 -- `HTML report` 包含分析结果。 有关其描述,请访问 PVS 的 [官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD). +Performs some simple regex-based checks of code style, using the [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check-style) binary (note that it can be run locally). +If it fails, fix the style errors following the [code style guide](style.md). + +### Report Details +- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) +- `output.txt` contains the check resulting errors (invalid tabulation etc), blank page means no errors. [Successful result example](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt). -## 快速测试 -通常这是为 PR 运行的第一次检查。 它构建 ClickHouse 并运行大部分 [无状态功能测试](tests.md#functional-tests),省略了一些额外操作。 -如果失败,则在修复之前不会开始进一步检查。 -查看报告以了解哪些测试失败,然后按照[此处](tests.md#functional-test-locally)所述在本地重现失败. +## PVS Check +Check the code with [PVS-studio](https://www.viva64.com/en/pvs-studio/), a static analysis tool. Look at the report to see the exact errors. Fix them if you can, if not -- ask a ClickHouse maintainer for help. -### 报告详情 -[状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) +### Report Details +- [Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) +- `test_run.txt.out.log` contains the building and analyzing log file. It includes only parsing or not-found errors. +- `HTML report` contains the analysis results. For its description visit PVS's [official site](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD). -#### 状态页面文件 -- `runlog.out.log` 包含所有的通用日志。 + +## Fast Test +Normally this is the first check that is ran for a PR. It builds ClickHouse and +runs most of [stateless functional tests](tests.md#functional-tests), omitting +some. If it fails, further checks are not started until it is fixed. Look at +the report to see which tests fail, then reproduce the failure locally as +described [here](tests.md#functional-test-locally). + +### Report Details +[Status page example](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) + +#### Status Page Files +- `runlog.out.log` is the general log that includes all other logs. - `test_log.txt` -- `submodule_log.txt` 包含有关克隆和检出所需子模块的消息。 +- `submodule_log.txt` contains the messages about cloning and checkouting needed submodules. - `stderr.log` - `stdout.log` - `clickhouse-server.log` @@ -94,32 +106,32 @@ git push - `install_log.txt` - `clickhouse-server.err.log` - `build_log.txt` -- `cmake_log.txt` 包含有关 C/C++ 和 Linux 标志检查的消息。 +- `cmake_log.txt` contains messages about the C/C++ and Linux flags check. -#### 状态页面列 +#### Status Page Columns -- *Test name* 包含测试的名称(没有路径,例如所有类型的测试都将被剥离为名称)。 -- *测试状态* -- _Skipped_、_Success_ 或_Fail_ 之一。 -- *测试时间,秒* - 本次测试为空。 +- *Test name* contains the name of the test (without the path e.g. all types of tests will be stripped to the name). +- *Test status* -- one of _Skipped_, _Success_, or _Fail_. +- *Test time, sec.* -- empty on this test. -## 构建检查 {#build-check} +## Build Check {#build-check} -各种配置构建 ClickHouse,以用于进一步的步骤。 您必须修复失败的构建。 构建日志通常有足够的信息来修复错误,但您可能必须在本地重现故障。 `cmake` 选项可以在构建日志中找到,搜索 `cmake`。 使用这些选项并遵循 [构建过程](../development/build.md)。 +Builds ClickHouse in various configurations for use in further steps. You have to fix the builds that fail. Build logs often has enough information to fix the error, but you might have to reproduce the failure locally. The `cmake` options can be found in the build log, grepping for `cmake`. Use these options and follow the [general build process](../development/build.md). -### 报告详情 +### Report Details -[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html). +[Status page example](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html). -- **Compiler**: `gcc-9` 或 `clang-10`(或其他架构的 `clang-10-xx`,例如 `clang-10-freebsd`)。 +- **Compiler**: `gcc-9` or `clang-10` (or `clang-10-xx` for other architectures e.g. `clang-10-freebsd`). - **Build type**: `Debug` or `RelWithDebInfo` (cmake). -- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), 或 `thread` (TSan). -- **Bundled**: `bundled` 构建使用来自 `contrib` 文件夹的库,`unbundled` 构建使用系统库。 -- **Splitted** `splitted` [split build](../development/build.md#split-build) -- **Status**: `success` 或 `fail` -- **Build log**: 构建和文件复制日志,在构建失败时很有用。 +- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). +- **Bundled**: `bundled` build uses libraries from `contrib` folder, and `unbundled` build uses system libraries. +- **Splitted** `splitted` is a [split build](../development/build.md#split-build) +- **Status**: `success` or `fail` +- **Build log**: link to the building and files copying log, useful when build failed. - **Build time**. -- **Artifacts**: 构建结果文件(`XXX`是服务器版本,例如`20.8.1.4344`)。 +- **Artifacts**: build result files (with `XXX` being the server version e.g. `20.8.1.4344`). - `clickhouse-client_XXX_all.deb` - `clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` - `clickhouse-common-staticXXX_amd64.deb` @@ -127,72 +139,75 @@ git push - `clickhouse-test_XXX_all.deb` - `clickhouse_XXX_amd64.buildinfo` - `clickhouse_XXX_amd64.changes` - - `clickhouse`: 主要构建的二进制文件。 + - `clickhouse`: Main built binary. - `clickhouse-odbc-bridge` - - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件。 - - `shared_build.tgz`: 使用共享库构建。 - - `performance.tgz`: 用于性能测试的特殊包。 + - `unit_tests_dbms`: GoogleTest binary with ClickHouse unit tests. + - `shared_build.tgz`: build with shared libraries. + - `performance.tgz`: Special package for performance tests. -## 特殊构建检查 -使用 `clang-tidy` 执行静态分析和代码样式检查。 该报告类似于[构建检查](#build-check)。 修复构建日志中发现的错误。 +## Special Build Check +Performs static analysis and code style checks using `clang-tidy`. The report is similar to the [build check](#build-check). Fix the errors found in the build log. -## 功能无状态测试 -在各种配置中构建的 ClickHouse 二进制文件运行 [无状态功能测试](tests.md#functional-tests) -- 发布、调试、使用等。 -查看报告以查看哪些测试失败,然后按照 [此处](tests.md#functional-test-locally) 的描述在本地重现失败。 -请注意,您必须使用正确的构建配置来重现 -- 在 AddressSanitizer 下测试可能会失败,但在 Debug 中通过。 -从 [CI 构建检查页面](../development/build.md#you-dont-have-to-build-clickhouse) 下载二进制文件,或在本地构建它。 +## Functional Stateless Tests +Runs [stateless functional tests](tests.md#functional-tests) for ClickHouse +binaries built in various configurations -- release, debug, with sanitizers, +etc. Look at the report to see which tests fail, then reproduce the failure +locally as described [here](tests.md#functional-test-locally). Note that you +have to use the correct build configuration to reproduce -- a test might fail +under AddressSanitizer but pass in Debug. Download the binary from [CI build +checks page](../development/build.md#you-dont-have-to-build-clickhouse), or build it locally. -## 功能状态测试 -运行 [状态功能测试](tests.md#functional-tests)。 与功能无状态测试相同的方式。不同之处在于它们需要来自 [Yandex.Metrica 数据集](../getting-started/example-datasets/metrica.md) 的 `hits` 和 `visits` 表才能运行。 +## Functional Stateful Tests +Runs [stateful functional tests](tests.md#functional-tests). Treat them in the same way as the functional stateless tests. The difference is that they require `hits` and `visits` tables from the [Yandex.Metrica dataset](../getting-started/example-datasets/metrica.md) to run. -## 集成测试 -运行 [integration tests](tests.md#integration-tests). +## Integration Tests +Runs [integration tests](tests.md#integration-tests). -## 测试流程检查 -使用 Testflows 系统运行一些测试。 请参阅[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们。 +## Testflows Check +Runs some tests using Testflows test system. See [here](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally) how to run them locally. -## 压力测试 -从多个客户端同时运行无状态功能测试以检测与并发相关的错误。 -如果失败: +## Stress Test +Runs stateless functional tests concurrently from several clients to detect +concurrency-related errors. If it fails: - * 首先修复所有失败; - * 查看报告以查找服务器日志并检查它们是否可能导致错误。 + * Fix all other test failures first; + * Look at the report to find the server logs and check them for possible causes + of error. -## 分布式测试 +## Split Build Smoke Test -检查[split build](../development/build.md#split-build)配置中的服务器构建可以启动和运行简单查询。 -如果失败: +Checks that the server build in [split build](../development/build.md#split-build) +configuration can start and run simple queries. If it fails: - * 首先修复所有失败; - * 在本地以 [split build](../development/build.md#split-build) 配置构建服务器并检查它是否可以启动并运行`select 1`。 + * Fix other test errors first; + * Build the server in [split build](../development/build.md#split-build) configuration + locally and check whether it can start and run `select 1`. -## 兼容性检查 -检查 `clickhouse` 二进制文件是否在具有旧 libc 版本的发行版上运行。 如果失败,请向维护人员寻求帮助。 +## Compatibility Check +Checks that `clickhouse` binary runs on distributions with old libc versions. If it fails, ask a maintainer for help. ## AST Fuzzer -运行随机生成的查询以捕获程序错误。 如果失败,请向维护人员寻求帮助。 +Runs randomly generated queries to catch program errors. If it fails, ask a maintainer for help. -## 性能测试 -测量查询性能的变化。 这是最长的检查,只需不到 6 小时即可运行。 性能测试报告详细描述[这里](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report)。 +## Performance Tests +Measure changes in query performance. This is the longest check that takes just below 6 hours to run. The performance test report is described in detail [here](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report). # QA -> 什么是状态页面上的`Task (private network)`项目? +> What is a `Task (private network)` item on status pages? +It's a link to the Yandex's internal job system. Yandex employees can see the check's start time and its more verbose status. -它是 Yandex 内部工作系统的链接。 Yandex 员工可以看到它的开始时间及其更详细的状态。 - -> 运行测试的地方 - -Yandex 内部基础设施的某个地方。 +> Where the tests are run +Somewhere on Yandex internal infrastructure. diff --git a/docs/zh/development/continuous-integration.md b/docs/zh/development/continuous-integration.md index 6cff83067de..718433381ad 100644 --- a/docs/zh/development/continuous-integration.md +++ b/docs/zh/development/continuous-integration.md @@ -1,13 +1,17 @@ -# 持续集成检查 {#continuous-integration-checks} -当你提交一个pull请求时, ClickHouse[持续集成(CI)系统](https://clickhouse.com/docs/en/development/tests/#test-automation)会对您的代码运行一些自动检查. +--- +toc_priority: 62 +toc_title: 持续集成检查 +--- -这在存储库维护者(来自ClickHouse团队的人)筛选了您的代码并将可测试标签添加到您的pull请求之后发生. +# 持续集成检查 -检查的结果被列在[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-pull-requests/collaborating-on-repositories-with-code-quality-features/about-status-checks)中所述的GitHub pull请求页面. +当您提交拉取请求时,ClickHouse 会为您的代码运行一些自动检查[continuous integration (CI) system](tests.md#test-automation). +这发生在存储库维护者(来自 ClickHouse 团队的某个人)筛选了您的代码并将`can be tested`标签添加到您的拉取请求之后。 +检查结果列在 GitHub 拉取请求页面上,如[GitHub检查文档](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/about-status-checks)中所述. +如果检查失败,您可能需要修复它。 此页面概述了您可能遇到的检查,以及您可以采取哪些措施来解决这些检查。 -如果检查失败,您可能被要求去修复它. 该界面介绍了您可能遇到的检查,以及如何修复它们. - -如果检查失败看起来与您的更改无关, 那么它可能是一些暂时的故障或基础设施问题. 向pull请求推一个空的commit以重新启动CI检查: +如果检查失败看起来与您的更改无关,则可能是一些暂时性故障或基础架构问题。 +将空提交推送到拉取请求以重新启动 CI 检查: ``` git reset @@ -15,55 +19,74 @@ git commit --allow-empty git push ``` -如果您不确定要做什么,可以向维护人员寻求帮助. +如果您不确定该怎么做,请向维护人员寻求帮助。 -## 与Master合并 {#merge-with-master} -验证PR是否可以合并到master. 如果没有, 它将失败并显示消息'Cannot fetch mergecommit'的.请按[GitHub文档](https://docs.github.com/en/github/collaborating-with-pull-requests/addressing-merge-conflicts/resolving-a-merge-conflict-on-github)中描述的冲突解决, 或使用git将主分支合并到您的pull请求分支来修复这个检查. +## 合并到Master -## 文档检查 {#docs-check} -尝试构建ClickHouse文档网站. 如果您更改了文档中的某些内容, 它可能会失败. 最可能的原因是文档中的某些交叉链接是错误的. 转到检查报告并查找`ERROR`和`WARNING`消息. +验证 PR 可以合并到 master。 如果没有,它将失败并显示消息'Cannot fetch mergecommit'。 +要修复此检查,请按照[GitHub文档](https://docs.github.com/en/github/collaborating-with-issues-and-pull-requests/resolving-a-merge-conflict-on-github)中的说明解决冲突, +或者使用 git 将 `master` 分支合并到你的拉取请求分支。 -### 报告详情 {#report-details} -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) -- `docs_output.txt`包含构建日志信息. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) +## 文档检查 -## 描述信息检查 {#description-check} -检查pull请求的描述是否符合[PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md)模板. +尝试构建 ClickHouse 文档网站。 如果您更改了文档中的某些内容,它可能会失败。 +最可能的原因是文档中的某些交叉链接是错误的。 +转到检查报告并查找`ERROR`和`WARNING`消息。 -您必须为您的更改指定一个更改日志类别(例如,Bug修复), 并且为[CHANGELOG.md](../whats-new/changelog/)编写一条用户可读的消息用来描述更改. +### 报告详情 -## 推送到DockerHub {#push-to-dockerhub} -生成用于构建和测试的docker映像, 然后将它们推送到DockerHub. +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check.html) +- `docs_output.txt` 包含构建日志。[成功结果示例](https://clickhouse-test-reports.s3.yandex.net/12550/eabcc293eb02214caa6826b7c15f101643f67a6b/docs_check/docs_output.txt) -## 标记检查 {#marker-check} -该检查意味着CI系统已经开始处理PR.当它处于'待处理'状态时,意味着尚未开始所有检查. 启动所有检查后,状态更改为'成功'. -# 格式检查 {#style-check} -使用`utils/check-style/check-style`二进制文件执行一些简单的基于正则表达式的代码样式检查(注意, 它可以在本地运行). -如果失败, 按照[代码样式指南](./style.md)修复样式错误. +## 描述检查 -### 报告详情 {#report-details} -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) -- `docs_output.txt`记录了查结果错误(无效表格等), 空白页表示没有错误. [成功结果案例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt) +检查您的拉取请求的描述是否符合模板 [PULL_REQUEST_TEMPLATE.md](https://github.com/ClickHouse/ClickHouse/blob/master/.github/PULL_REQUEST_TEMPLATE.md). +您必须为您的更改指定一个更改日志类别(例如,错误修复),并编写一条用户可读的消息来描述[CHANGELOG.md](../whats-new/changelog/index.md)的更改 -### PVS 检查 {#pvs-check} -使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码. 查看报告以查看确切的错误.如果可以则修复它们, 如果不行, 可以向ClickHouse的维护人员寻求帮忙. -### 报告详情 {#report-details} -- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) -- `test_run.txt.out.log`包含构建和分析日志文件.它只包含解析或未找到的错误. -- `HTML report`包含分析结果.有关说明请访问PVS的[官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD) +## 推送到 Dockerhub -## 快速测试 {#fast-test} -通常情况下这是PR运行的第一个检查.它构建ClickHouse以及大多数无状态运行测试, 其中省略了一些.如果失败,在修复之前不会开始进一步的检查. 查看报告以了解哪些测试失败, 然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败. +构建用于构建和测试的 docker 镜像,然后将它们推送到 DockerHub。 -### 报告详情 {#report-details} + +## 标记检查 + +这个检查意味着 CI 系统开始处理拉取请求。 当它处于“待处理”状态时,意味着并非所有检查都已开始。 启动所有检查后,状态更改为“成功”。 + + +## 格式检查 + +使用 [`utils/check-style/check-style`](https://github.com/ClickHouse/ClickHouse/blob/master/utils/check-style/check -style) 二进制(注意它可以在本地运行)。 +如果失败,请按照 [code style guide](style.md) 修复样式错误。 + + +### 报告详情 +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check.html) +- `output.txt` 包含检查结果错误(无效制表等),空白页表示没有错误。 [成功结果示例](https://clickhouse-test-reports.s3.yandex.net/12550/659c78c7abb56141723af6a81bfae39335aa8cb2/style_check/output.txt). + + +## PVS检查 +使用静态分析工具[PVS-studio](https://www.viva64.com/en/pvs-studio/)检查代码。 查看报告以查看确切的错误。 如果可以,请修复它们,如果不能,请向 ClickHouse 维护人员寻求帮助。 + +### 报告详情 +- [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/pvs_check.html) +- `test_run.txt.out.log` 包含构建和分析日志文件。 它仅包括解析或未找到的错误。 +- `HTML report` 包含分析结果。 有关其描述,请访问 PVS 的 [官方网站](https://www.viva64.com/en/m/0036/#ID14E9A2B2CD). + + +## 快速测试 +通常这是为 PR 运行的第一次检查。 它构建 ClickHouse 并运行大部分 [无状态功能测试](tests.md#functional-tests),省略了一些额外操作。 +如果失败,则在修复之前不会开始进一步检查。 +查看报告以了解哪些测试失败,然后按照[此处](tests.md#functional-test-locally)所述在本地重现失败. + +### 报告详情 [状态页示例](https://clickhouse-test-reports.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/fast_test.html) -#### 状态页文件 {#status-page-files} -- `runlog.out.log` 是包含所有其他日志的通用日志. +#### 状态页面文件 +- `runlog.out.log` 包含所有的通用日志。 - `test_log.txt` -- `submodule_log.txt` 包含关于克隆和检查所需子模块的消息. +- `submodule_log.txt` 包含有关克隆和检出所需子模块的消息。 - `stderr.log` - `stdout.log` - `clickhouse-server.log` @@ -71,85 +94,105 @@ git push - `install_log.txt` - `clickhouse-server.err.log` - `build_log.txt` -- `cmake_log.txt` 包含关于C/C++和Linux标志检查的消息. +- `cmake_log.txt` 包含有关 C/C++ 和 Linux 标志检查的消息。 -#### 状态页列信息 {#status-page-columns} -- 测试名称 -- 包含测试的名称(不带路径, 例如, 所有类型的测试将被剥离到该名称). -- 测试状态 -- 跳过、成功或失败之一. -- 测试时间, 秒. -- 这个测试是空的. +#### 状态页面列 -## 建构检查 {#build-check} -在各种配置中构建ClickHouse, 以便在后续步骤中使用. 您必须修复失败的构建.构建日志通常有足够的信息来修复错误, 但是您可能必须在本地重现故障. `cmake`选项可以在构建日志中通过grep `cmake`操作找到.使用这些选项并遵循[一般的构建过程](./build.md). +- *Test name* 包含测试的名称(没有路径,例如所有类型的测试都将被剥离为名称)。 +- *测试状态* -- _Skipped_、_Success_ 或_Fail_ 之一。 +- *测试时间,秒* - 本次测试为空。 -### 报告详情 {#report-details} -[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html) -- **Compiler**: `gcc-9` 或 `clang-10` (或其他架构的`clang-10-xx`, 比如`clang-10-freebsd`). + +## 构建检查 {#build-check} + +各种配置构建 ClickHouse,以用于进一步的步骤。 您必须修复失败的构建。 构建日志通常有足够的信息来修复错误,但您可能必须在本地重现故障。 `cmake` 选项可以在构建日志中找到,搜索 `cmake`。 使用这些选项并遵循 [构建过程](../development/build.md)。 + +### 报告详情 + +[状态页示例](https://clickhouse-builds.s3.yandex.net/12550/67d716b5cc3987801996c31a67b31bf141bc3486/clickhouse_build_check/report.html). + +- **Compiler**: `gcc-9` 或 `clang-10`(或其他架构的 `clang-10-xx`,例如 `clang-10-freebsd`)。 - **Build type**: `Debug` or `RelWithDebInfo` (cmake). -- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), or `thread` (TSan). -- **Bundled**: `bundled` 构建使用来自 `contrib` 库, 而 `unbundled` 构建使用系统库. -- **Splitted**: `splitted` is a [split build](https://clickhouse.com/docs/en/development/build/#split-build) -- **Status**: `成功` 或 `失败` -- **Build log**: 链接到构建和文件复制日志, 当构建失败时很有用. +- **Sanitizer**: `none` (without sanitizers), `address` (ASan), `memory` (MSan), `undefined` (UBSan), 或 `thread` (TSan). +- **Bundled**: `bundled` 构建使用来自 `contrib` 文件夹的库,`unbundled` 构建使用系统库。 +- **Splitted** `splitted` [split build](../development/build.md#split-build) +- **Status**: `success` 或 `fail` +- **Build log**: 构建和文件复制日志,在构建失败时很有用。 - **Build time**. -- **Artifacts**: 构建结果文件 (`XXX`是服务器版本, 比如`20.8.1.4344`). - - `clickhouse-client_XXX_all.deb` - -` clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` - - `clickhouse-common-staticXXX_amd64.deb` - - `clickhouse-server_XXX_all.deb` - - `clickhouse-test_XXX_all.deb` - - `clickhouse_XXX_amd64.buildinfo` - - `clickhouse_XXX_amd64.changes` - - `clickhouse`: Main built binary. - - `clickhouse-odbc-bridge` - - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件. - - `shared_build.tgz`: 使用共享库构建. - - `performance.tgz`: 用于性能测试的特殊包. +- **Artifacts**: 构建结果文件(`XXX`是服务器版本,例如`20.8.1.4344`)。 + - `clickhouse-client_XXX_all.deb` + - `clickhouse-common-static-dbg_XXX[+asan, +msan, +ubsan, +tsan]_amd64.deb` + - `clickhouse-common-staticXXX_amd64.deb` + - `clickhouse-server_XXX_all.deb` + - `clickhouse-test_XXX_all.deb` + - `clickhouse_XXX_amd64.buildinfo` + - `clickhouse_XXX_amd64.changes` + - `clickhouse`: 主要构建的二进制文件。 + - `clickhouse-odbc-bridge` + - `unit_tests_dbms`: 带有 ClickHouse 单元测试的 GoogleTest 二进制文件。 + - `shared_build.tgz`: 使用共享库构建。 + - `performance.tgz`: 用于性能测试的特殊包。 -## 特殊构建检查 {#special-buildcheck} -使用clang-tidy执行静态分析和代码样式检查. 该报告类似于构建检查. 修复在构建日志中发现的错误. -## 功能无状态测试 {#functional-stateless-tests} -为构建在不同配置中的ClickHouse二进制文件运行[无状态功能测试](./tests.md#functional-tests)——发布、调试、使用杀毒软件等.通过报告查看哪些测试失败,然后按照[此处](./tests.md#functional-test-locally)描述的在本地重现失败.注意, 您必须使用正确的构建配置来重现——在AddressSanitizer下测试可能失败,但在Debug中可以通过.从[CI构建检查页面](./build.md#you-dont-have-to-build-clickhouse)下载二进制文件, 或者在本地构建它. +## 特殊构建检查 +使用 `clang-tidy` 执行静态分析和代码样式检查。 该报告类似于[构建检查](#build-check)。 修复构建日志中发现的错误。 -## 功能有状态测试 {#functional-stateful-tests} -运行[有状态功能测试](./tests.md#functional-tests).以无状态功能测试相同的方式对待它们.不同之处在于它们需要从[Yandex.Metrica数据集](https://clickhouse.com/docs/en/getting-started/example-datasets/metrica/)的`hits`和`visits`表来运行. -## 集成测试 {#integration-tests} -运行[集成测试](./tests.md#integration-tests). +## 功能无状态测试 +在各种配置中构建的 ClickHouse 二进制文件运行 [无状态功能测试](tests.md#functional-tests) -- 发布、调试、使用等。 +查看报告以查看哪些测试失败,然后按照 [此处](tests.md#functional-test-locally) 的描述在本地重现失败。 +请注意,您必须使用正确的构建配置来重现 -- 在 AddressSanitizer 下测试可能会失败,但在 Debug 中通过。 +从 [CI 构建检查页面](../development/build.md#you-dont-have-to-build-clickhouse) 下载二进制文件,或在本地构建它。 -## Testflows 检查{#testflows-check} -使用Testflows测试系统去运行一些测试, 在[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们. -## 压力测试 {#stress-test} -从多个客户端并发运行无状态功能测试, 用以检测与并发相关的错误.如果失败: -``` -* Fix all other test failures first; -* Look at the report to find the server logs and check them for possible causes - of error. -``` +## 功能状态测试 +运行 [状态功能测试](tests.md#functional-tests)。 与功能无状态测试相同的方式。不同之处在于它们需要来自 [Yandex.Metrica 数据集](../getting-started/example-datasets/metrica.md) 的 `hits` 和 `visits` 表才能运行。 -## 冒烟测试 {#split-build-smoke-test} -检查[拆分构建](./build.md#split-build)配置中的服务器构建是否可以启动并运行简单查询.如果失败: -``` -* Fix other test errors first; -* Build the server in [split build](./build.md#split-build) configuration - locally and check whether it can start and run `select 1`. -``` -## 兼容性检查 {#compatibility-check} -检查`clickhouse`二进制文件是否可以在带有旧libc版本的发行版上运行.如果失败, 请向维护人员寻求帮助. +## 集成测试 +运行 [integration tests](tests.md#integration-tests). -## AST模糊器 {#ast-fuzzer} -运行随机生成的查询来捕获程序错误.如果失败, 请向维护人员寻求帮助. -## 性能测试 {#performance-tests} -测量查询性能的变化. 这是最长的检查, 只需不到 6 小时即可运行.性能测试报告在[此处](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report)有详细描述. +## 测试流程检查 +使用 Testflows 系统运行一些测试。 请参阅[此处](https://github.com/ClickHouse/ClickHouse/tree/master/tests/testflows#running-tests-locally)查看如何在本地运行它们。 -## 质量保证 {#qa} -什么是状态页面上的任务(专用网络)项目? -它是 Yandex 内部工作系统的链接. Yandex 员工可以看到检查的开始时间及其更详细的状态. +## 压力测试 +从多个客户端同时运行无状态功能测试以检测与并发相关的错误。 +如果失败: -运行测试的地方 + * 首先修复所有失败; + * 查看报告以查找服务器日志并检查它们是否可能导致错误。 -Yandex 内部基础设施的某个地方. + +## 分布式测试 + +检查[split build](../development/build.md#split-build)配置中的服务器构建可以启动和运行简单查询。 +如果失败: + + * 首先修复所有失败; + * 在本地以 [split build](../development/build.md#split-build) 配置构建服务器并检查它是否可以启动并运行`select 1`。 + + +## 兼容性检查 +检查 `clickhouse` 二进制文件是否在具有旧 libc 版本的发行版上运行。 如果失败,请向维护人员寻求帮助。 + + +## AST Fuzzer +运行随机生成的查询以捕获程序错误。 如果失败,请向维护人员寻求帮助。 + + +## 性能测试 +测量查询性能的变化。 这是最长的检查,只需不到 6 小时即可运行。 性能测试报告详细描述[这里](https://github.com/ClickHouse/ClickHouse/tree/master/docker/test/performance-comparison#how-to-read-the-report)。 + + + +# QA + +> 什么是状态页面上的`Task (private network)`项目? + +它是 Yandex 内部工作系统的链接。 Yandex 员工可以看到它的开始时间及其更详细的状态。 + +> 运行测试的地方 + +Yandex 内部基础设施的某个地方。 From 35a4be601add303f5c7caac139e2a6d3cfeba428 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Nov 2021 15:43:23 +0300 Subject: [PATCH 121/609] Select query commit only with settings --- src/Storages/Kafka/KafkaSettings.h | 1 + src/Storages/Kafka/KafkaSource.h | 2 +- src/Storages/Kafka/StorageKafka.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQSettings.h | 1 + src/Storages/RabbitMQ/RabbitMQSource.h | 2 +- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 3 +-- tests/integration/test_storage_kafka/test.py | 23 ++++++++++++++++++- .../integration/test_storage_rabbitmq/test.py | 6 +++++ 8 files changed, 34 insertions(+), 6 deletions(-) diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 1010c486abb..51e5ee47077 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -31,6 +31,7 @@ class ASTStorage; M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ M(Bool, kafka_thread_per_consumer, false, "Provide independent thread for each consumer", 0) \ M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \ + M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ /** TODO: */ /* https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md */ diff --git a/src/Storages/Kafka/KafkaSource.h b/src/Storages/Kafka/KafkaSource.h index 1f0cbb355d1..e80edfb9ef4 100644 --- a/src/Storages/Kafka/KafkaSource.h +++ b/src/Storages/Kafka/KafkaSource.h @@ -23,7 +23,7 @@ public: const Names & columns, Poco::Logger * log_, size_t max_block_size_, - bool commit_in_suffix = true); + bool commit_in_suffix = false); ~KafkaSource() override; String getName() const override { return storage.getName(); } diff --git a/src/Storages/Kafka/StorageKafka.cpp b/src/Storages/Kafka/StorageKafka.cpp index 25384db043f..b6372860541 100644 --- a/src/Storages/Kafka/StorageKafka.cpp +++ b/src/Storages/Kafka/StorageKafka.cpp @@ -278,7 +278,7 @@ Pipe StorageKafka::read( /// Use block size of 1, otherwise LIMIT won't work properly as it will buffer excess messages in the last block /// TODO: probably that leads to awful performance. /// FIXME: seems that doesn't help with extra reading and committing unprocessed messages. - pipes.emplace_back(std::make_shared(*this, metadata_snapshot, modified_context, column_names, log, 1)); + pipes.emplace_back(std::make_shared(*this, metadata_snapshot, modified_context, column_names, log, 1, kafka_settings->kafka_commit_on_select)); } LOG_DEBUG(log, "Starting reading {} streams", pipes.size()); diff --git a/src/Storages/RabbitMQ/RabbitMQSettings.h b/src/Storages/RabbitMQ/RabbitMQSettings.h index 01109dda66a..0475a5e6163 100644 --- a/src/Storages/RabbitMQ/RabbitMQSettings.h +++ b/src/Storages/RabbitMQ/RabbitMQSettings.h @@ -28,6 +28,7 @@ namespace DB M(String, rabbitmq_vhost, "/", "RabbitMQ vhost.", 0) \ M(String, rabbitmq_queue_settings_list, "", "A list of rabbitmq queue settings", 0) \ M(Bool, rabbitmq_queue_consume, false, "Use user-defined queues and do not make any RabbitMQ setup: declaring exchanges, queues, bindings", 0) \ + M(Bool, rabbitmq_commit_on_select, false, "Commit messages when select query is made", 0) \ #define LIST_OF_RABBITMQ_SETTINGS(M) \ RABBITMQ_RELATED_SETTINGS(M) \ diff --git a/src/Storages/RabbitMQ/RabbitMQSource.h b/src/Storages/RabbitMQ/RabbitMQSource.h index f4f6c72b5c8..f3ceac8e1e5 100644 --- a/src/Storages/RabbitMQ/RabbitMQSource.h +++ b/src/Storages/RabbitMQ/RabbitMQSource.h @@ -18,7 +18,7 @@ public: ContextPtr context_, const Names & columns, size_t max_block_size_, - bool ack_in_suffix = true); + bool ack_in_suffix = false); ~RabbitMQSource() override; diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index cf9b557de25..2470fb9c0bd 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -612,7 +612,6 @@ Pipe StorageRabbitMQ::read( auto sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); auto modified_context = addSettings(local_context); - auto block_size = getMaxBlockSize(); if (!connection->isConnected()) { @@ -630,7 +629,7 @@ Pipe StorageRabbitMQ::read( for (size_t i = 0; i < num_created_consumers; ++i) { auto rabbit_source = std::make_shared( - *this, metadata_snapshot, modified_context, column_names, block_size); + *this, metadata_snapshot, modified_context, column_names, 1, rabbitmq_settings->rabbitmq_commit_on_select); auto converting_dag = ActionsDAG::makeConvertingActions( rabbit_source->getPort().getHeader().getColumnsWithTypeAndName(), diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index bf9ab4f9e27..b8536abc29f 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -236,7 +236,8 @@ kafka_topic_old old instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n') + SETTINGS kafka_commit_on_select = 1; ''') # Don't insert malformed messages since old settings syntax @@ -268,6 +269,7 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_group_name = '{kafka_group_name_new}', kafka_format = '{kafka_format_json_each_row}', kafka_row_delimiter = '\\n', + kafka_commit_on_select = 1, kafka_client_id = '{kafka_client_id} test 1234', kafka_skip_broken_messages = 1; ''') @@ -313,6 +315,7 @@ def test_kafka_json_as_string(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'kafka_json_as_string', kafka_group_name = 'kafka_json_as_string', + kafka_commit_on_select = 1, kafka_format = 'JSONAsString', kafka_flush_interval_ms=1000; ''') @@ -802,6 +805,7 @@ def test_kafka_issue4116(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'issue4116', kafka_group_name = 'issue4116', + kafka_commit_on_select = 1, kafka_format = 'CSV', kafka_row_delimiter = '\\n', format_csv_delimiter = '|'; @@ -875,6 +879,7 @@ def test_kafka_consumer_hang(kafka_cluster): ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = '{topic_name}', + kafka_commit_on_select = 1, kafka_group_name = '{topic_name}', kafka_format = 'JSONEachRow', kafka_num_consumers = 8; @@ -900,12 +905,14 @@ def test_kafka_consumer_hang2(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'consumer_hang2', kafka_group_name = 'consumer_hang2', + kafka_commit_on_select = 1, kafka_format = 'JSONEachRow'; CREATE TABLE test.kafka2 (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'consumer_hang2', + kafka_commit_on_select = 1, kafka_group_name = 'consumer_hang2', kafka_format = 'JSONEachRow'; ''') @@ -944,6 +951,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster): ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'csv', + kafka_commit_on_select = 1, kafka_group_name = 'csv', kafka_format = 'CSV'; ''') @@ -968,6 +976,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'tsv', + kafka_commit_on_select = 1, kafka_group_name = 'tsv', kafka_format = 'TSV'; ''') @@ -991,6 +1000,7 @@ def test_kafka_select_empty(kafka_cluster): ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = '{topic_name}', + kafka_commit_on_select = 1, kafka_group_name = '{topic_name}', kafka_format = 'TSV', kafka_row_delimiter = '\\n'; @@ -1017,6 +1027,7 @@ def test_kafka_json_without_delimiter(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'json', kafka_group_name = 'json', + kafka_commit_on_select = 1, kafka_format = 'JSONEachRow'; ''') @@ -1041,6 +1052,7 @@ def test_kafka_protobuf(kafka_cluster): kafka_topic_list = 'pb', kafka_group_name = 'pb', kafka_format = 'Protobuf', + kafka_commit_on_select = 1, kafka_schema = 'kafka.proto:KeyValuePair'; ''') @@ -1069,6 +1081,7 @@ SETTINGS kafka_topic_list = 'string_field_on_first_position_in_protobuf', kafka_group_name = 'string_field_on_first_position_in_protobuf', kafka_format = 'Protobuf', + kafka_commit_on_select = 1, kafka_schema = 'social:User'; ''') @@ -1135,6 +1148,7 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): kafka_topic_list = 'pb_no_delimiter', kafka_group_name = 'pb_no_delimiter', kafka_format = 'ProtobufSingle', + kafka_commit_on_select = 1, kafka_schema = 'kafka.proto:KeyValuePair'; ''') @@ -1157,6 +1171,7 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): kafka_topic_list = 'pb_no_delimiter', kafka_group_name = 'pb_no_delimiter', kafka_format = 'ProtobufSingle', + kafka_commit_on_select = 1, kafka_schema = 'kafka.proto:KeyValuePair'; ''') @@ -1487,6 +1502,7 @@ def test_kafka_virtual_columns(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'virt1', kafka_group_name = 'virt1', + kafka_commit_on_select = 1, kafka_format = 'JSONEachRow'; ''') @@ -1558,6 +1574,7 @@ def test_kafka_insert(kafka_cluster): kafka_topic_list = 'insert1', kafka_group_name = 'insert1', kafka_format = 'TSV', + kafka_commit_on_select = 1, kafka_row_delimiter = '\\n'; ''') @@ -1851,6 +1868,7 @@ def test_kafka_insert_avro(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'avro1', kafka_group_name = 'avro1', + kafka_commit_on_select = 1, kafka_format = 'Avro'; ''') @@ -2279,6 +2297,7 @@ def test_exception_from_destructor(kafka_cluster): SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'xyz', kafka_group_name = '', + kafka_commit_on_select = 1, kafka_format = 'JSONEachRow'; ''') instance.query_and_get_error(''' @@ -2577,6 +2596,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_topic_list = 'test_bad_reschedule', kafka_group_name = 'test_bad_reschedule', kafka_format = 'JSONEachRow', + kafka_commit_on_select = 1, kafka_max_block_size = 1000; CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS @@ -2646,6 +2666,7 @@ def test_kafka_csv_with_thread_per_consumer(kafka_cluster): kafka_format = 'CSV', kafka_row_delimiter = '\\n', kafka_num_consumers = 4, + kafka_commit_on_select = 1, kafka_thread_per_consumer = 1; ''') diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 1c27c95d2ab..d54b157cf8d 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -78,6 +78,7 @@ def test_rabbitmq_select(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'select', + rabbitmq_commit_on_select = 1, rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; '''.format(rabbitmq_cluster.rabbitmq_host)) @@ -113,6 +114,7 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'empty', + rabbitmq_commit_on_select = 1, rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; '''.format(rabbitmq_cluster.rabbitmq_host)) @@ -125,6 +127,7 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = '{}:5672', + rabbitmq_commit_on_select = 1, rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' '''.format(rabbitmq_cluster.rabbitmq_host)) @@ -167,6 +170,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'csv', + rabbitmq_commit_on_select = 1, rabbitmq_format = 'CSV', rabbitmq_row_delimiter = '\\n'; ''') @@ -202,6 +206,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', rabbitmq_exchange_name = 'tsv', rabbitmq_format = 'TSV', + rabbitmq_commit_on_select = 1, rabbitmq_queue_base = 'tsv', rabbitmq_row_delimiter = '\\n'; CREATE TABLE test.view (key UInt64, value UInt64) @@ -238,6 +243,7 @@ def test_rabbitmq_macros(rabbitmq_cluster): CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ SETTINGS rabbitmq_host_port = '{rabbitmq_host}:{rabbitmq_port}', + rabbitmq_commit_on_select = 1, rabbitmq_exchange_name = '{rabbitmq_exchange_name}', rabbitmq_format = '{rabbitmq_format}' ''') From 1ba71c20fdbaf08621856596b4c8b372844db911 Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Thu, 4 Nov 2021 16:39:36 +0800 Subject: [PATCH 122/609] support bool type --- src/DataTypes/DataTypeDomainBool.cpp | 21 ++++++ src/DataTypes/DataTypeFactory.cpp | 1 + src/DataTypes/DataTypeFactory.h | 1 + src/DataTypes/DataTypesNumber.cpp | 2 - .../Serializations/SerializationBool.cpp | 72 +++++++++++++++++++ .../Serializations/SerializationBool.h | 23 ++++++ 6 files changed, 118 insertions(+), 2 deletions(-) create mode 100644 src/DataTypes/DataTypeDomainBool.cpp create mode 100644 src/DataTypes/Serializations/SerializationBool.cpp create mode 100644 src/DataTypes/Serializations/SerializationBool.h diff --git a/src/DataTypes/DataTypeDomainBool.cpp b/src/DataTypes/DataTypeDomainBool.cpp new file mode 100644 index 00000000000..245c5495299 --- /dev/null +++ b/src/DataTypes/DataTypeDomainBool.cpp @@ -0,0 +1,21 @@ +#include +#include +#include + +namespace DB +{ + +void registerDataTypeDomainBool(DataTypeFactory & factory) +{ + factory.registerSimpleDataTypeCustom("Bool", [] + { + auto type = DataTypeFactory::instance().get("UInt8"); + return std::make_pair(type, std::make_unique( + std::make_unique("Bool"), std::make_unique(type->getDefaultSerialization()))); + }); + + factory.registerAlias("bool", "Bool", DataTypeFactory::CaseInsensitive); + factory.registerAlias("boolean", "Bool", DataTypeFactory::CaseInsensitive); +} + +} diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 63b7c54306d..582b42accd9 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -209,6 +209,7 @@ DataTypeFactory::DataTypeFactory() registerDataTypeInterval(*this); registerDataTypeLowCardinality(*this); registerDataTypeDomainIPv4AndIPv6(*this); + registerDataTypeDomainBool(*this); registerDataTypeDomainSimpleAggregateFunction(*this); registerDataTypeDomainGeo(*this); registerDataTypeMap(*this); diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index e44c66466a1..81d7d991bdc 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -85,6 +85,7 @@ void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory); void registerDataTypeLowCardinality(DataTypeFactory & factory); void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory); +void registerDataTypeDomainBool(DataTypeFactory & factory); void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory); void registerDataTypeDomainGeo(DataTypeFactory & factory); diff --git a/src/DataTypes/DataTypesNumber.cpp b/src/DataTypes/DataTypesNumber.cpp index 6a1aed888c9..fef4c34d8b0 100644 --- a/src/DataTypes/DataTypesNumber.cpp +++ b/src/DataTypes/DataTypesNumber.cpp @@ -57,8 +57,6 @@ void registerDataTypeNumbers(DataTypeFactory & factory) /// These synonyms are added for compatibility. factory.registerAlias("TINYINT", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BOOL", "Int8", DataTypeFactory::CaseInsensitive); - factory.registerAlias("BOOLEAN", "Int8", DataTypeFactory::CaseInsensitive); factory.registerAlias("INT1", "Int8", DataTypeFactory::CaseInsensitive); /// MySQL factory.registerAlias("BYTE", "Int8", DataTypeFactory::CaseInsensitive); /// MS Access factory.registerAlias("SMALLINT", "Int16", DataTypeFactory::CaseInsensitive); diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp new file mode 100644 index 00000000000..7d22e09f921 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -0,0 +1,72 @@ +#include + +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING; + extern const int ILLEGAL_COLUMN; +} + +SerializationBool::SerializationBool(const SerializationPtr &nested_) + : SerializationCustomSimpleText(nested_) +{ +} + +void SerializationBool::serializeText(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &) const +{ + const auto *col = checkAndGetColumn(&column); + if (!col) + throw Exception("Bool type can only serialize columns of type UInt8." + column.getName(), + ErrorCodes::ILLEGAL_COLUMN); + + if (col->getData()[row_num]) + ostr.write(str_true, sizeof(str_true) - 1); + else + ostr.write(str_false, sizeof(str_false) - 1); +} + +void SerializationBool::deserializeText(IColumn &column, ReadBuffer &istr, const FormatSettings &) const +{ + ColumnUInt8 *col = typeid_cast(&column); + if (!col) + { + throw Exception("Bool type can only deserialize columns of type UInt32." + column.getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + + if (!istr.eof()) + { + bool value = false; + + if (*istr.position() == 't' || *istr.position() == 'f') + readBoolTextWord(value, istr); + else if (*istr.position() == '1' || *istr.position() == '0') + readBoolText(value, istr); + else + throw Exception("Invalid boolean value, should be true, false, 1, or 0.", + ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); + col->insert(value); + } + else + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); +} + +void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const +{ + serializeText(column, row_num, ostr, settings); +} + +void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, const FormatSettings &settings) const +{ + deserializeText(column, istr, settings); +} + +} diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h new file mode 100644 index 00000000000..19c7d844535 --- /dev/null +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -0,0 +1,23 @@ +#pragma once + +#include + +namespace DB +{ + +class SerializationBool final : public SerializationCustomSimpleText +{ +private: + static constexpr char str_true[5] = "true"; + static constexpr char str_false[6] = "false"; +public: + SerializationBool(const SerializationPtr & nested_); + + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; + void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; +}; + +} From e264b4dcba3a0a17ceef6b814c4ea0a36166dbe5 Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Thu, 4 Nov 2021 16:46:09 +0800 Subject: [PATCH 123/609] add tests --- .../0_stateless/02114_bool_type.reference | 29 +++++++++++++++++++ tests/queries/0_stateless/02114_bool_type.sql | 16 ++++++++++ 2 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/02114_bool_type.reference create mode 100644 tests/queries/0_stateless/02114_bool_type.sql diff --git a/tests/queries/0_stateless/02114_bool_type.reference b/tests/queries/0_stateless/02114_bool_type.reference new file mode 100644 index 00000000000..43be4a86c86 --- /dev/null +++ b/tests/queries/0_stateless/02114_bool_type.reference @@ -0,0 +1,29 @@ +CREATE TABLE test.bool_test\n(\n `value` Bool\n)\nENGINE = Memory +false +true +false +true +false +true +false +true +{"value":false} +{"value":true} +{"value":false} +{"value":true} +{"value":false} +{"value":true} +{"value":false} +{"value":true} +0 +1 +0 +1 +0 +1 +0 +1 +true +true +true +true diff --git a/tests/queries/0_stateless/02114_bool_type.sql b/tests/queries/0_stateless/02114_bool_type.sql new file mode 100644 index 00000000000..ccc826748b7 --- /dev/null +++ b/tests/queries/0_stateless/02114_bool_type.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS test.bool_test; + +CREATE TABLE test.bool_test (value Bool) ENGINE = Memory; + +-- value column shoud have type 'Bool' +SHOW CREATE TABLE test.bool_test; + +INSERT INTO test.bool_test (value) VALUES ('false'), ('true'), (0), (1); +INSERT INTO test.bool_test (value) FORMAT JSONEachRow {"value":false}{"value":true}{"value":0}{"value":1} + +SELECT value FROM test.bool_test; +SELECT value FROM test.bool_test FORMAT JSONEachRow; +SELECT toUInt64(value) FROM test.bool_test; +SELECT value FROM test.bool_test where value > 0; + +DROP TABLE IF EXISTS test.bool_test; From dd54b4aa266b604809b00c51a63520db8e7d2d78 Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Sat, 6 Nov 2021 15:19:12 +0800 Subject: [PATCH 124/609] support bool literal --- src/Parsers/ExpressionElementParsers.cpp | 21 +++++++++++++++++++++ src/Parsers/ExpressionElementParsers.h | 8 ++++++++ 2 files changed, 29 insertions(+) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e16a7605afc..20b508b5d9b 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -1524,6 +1524,23 @@ bool ParserNull::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } + +bool ParserBool::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (ParserKeyword("true").parse(pos, node, expected)) + { + node = std::make_shared(true); + return true; + } + else if (ParserKeyword("false").parse(pos, node, expected)) + { + node = std::make_shared(false); + return true; + } + else + return false; +} + static bool parseNumber(char * buffer, size_t size, bool negative, int base, Field & res) { errno = 0; /// Functions strto* don't clear errno. @@ -1755,6 +1772,7 @@ bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserNull null_p; ParserNumber num_p; + ParserBool bool_p; ParserStringLiteral str_p; if (null_p.parse(pos, node, expected)) @@ -1763,6 +1781,9 @@ bool ParserLiteral::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (num_p.parse(pos, node, expected)) return true; + if (bool_p.parse(pos, node, expected)) + return true; + if (str_p.parse(pos, node, expected)) return true; diff --git a/src/Parsers/ExpressionElementParsers.h b/src/Parsers/ExpressionElementParsers.h index 91f1730f93d..5dff2e026be 100644 --- a/src/Parsers/ExpressionElementParsers.h +++ b/src/Parsers/ExpressionElementParsers.h @@ -294,6 +294,14 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; }; +/** Bool literal. + */ +class ParserBool : public IParserBase +{ +protected: + const char * getName() const override { return "Bool"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; /** Numeric literal. */ From be88441cd657d8da4bae1c5a175316f64fb98516 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 5 Nov 2021 17:25:02 +0300 Subject: [PATCH 125/609] Much better --- .../PostgreSQLReplicationHandler.cpp | 111 ++++++------ .../PostgreSQL/PostgreSQLReplicationHandler.h | 4 +- .../StorageMaterializedPostgreSQL.cpp | 9 +- .../StorageMaterializedPostgreSQL.h | 6 +- .../test.py | 159 +++++++++--------- 5 files changed, 134 insertions(+), 155 deletions(-) diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 23b45cbf83b..1d6ae48f508 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -83,38 +83,32 @@ void PostgreSQLReplicationHandler::startup() } -String PostgreSQLReplicationHandler::probablyDoubleQuoteWithSchema(const String & table_name, bool quote) const +std::pair PostgreSQLReplicationHandler::getSchemaAndTableName(const String & table_name) const { - if (table_name.starts_with("\"")) - { - if (!quote) - return table_name.substr(1, table_name.size() - 1); - return table_name; - } - /// !schema_list.empty() -- We replicate all tables from specifies schemas. /// In this case when tables list is fetched, we append schema with dot. But without quotes. /// If there is a setting `tables_list`, then table names can be put there along with schema, /// separated by dot and with no quotes. We add double quotes in this case. + if (!postgres_schema.empty()) + return std::make_pair(postgres_schema, table_name); + if (auto pos = table_name.find('.'); schema_as_a_part_of_table_name && pos != std::string::npos) - { - auto schema = table_name.substr(0, pos); - auto table = table_name.substr(pos + 1); - return doubleQuoteString(schema) + '.' + doubleQuoteString(table); - } + return std::make_pair(table_name.substr(0, pos), table_name.substr(pos + 1)); - if (postgres_schema.empty()) - { - /// We do no need quotes to fetch table structure in case there is no schema. (will not work) - if (quote) - return doubleQuoteString(table_name); - else - return table_name; - } + return std::make_pair("", table_name); +} - return doubleQuoteString(postgres_schema) + '.' + doubleQuoteString(table_name); + +String PostgreSQLReplicationHandler::doubleQuoteWithSchema(const String & table_name) const +{ + auto [schema, table] = getSchemaAndTableName(table_name); + + if (schema.empty()) + return doubleQuoteString(table); + + return doubleQuoteString(schema) + '.' + doubleQuoteString(table); } @@ -128,6 +122,8 @@ void PostgreSQLReplicationHandler::checkConnectionAndStart() } catch (const pqxx::broken_connection & pqxx_error) { + tryLogCurrentException(__PRETTY_FUNCTION__); + if (!is_attach) throw; @@ -136,10 +132,10 @@ void PostgreSQLReplicationHandler::checkConnectionAndStart() } catch (...) { + tryLogCurrentException(__PRETTY_FUNCTION__); + if (!is_attach) throw; - - tryLogCurrentException(__PRETTY_FUNCTION__); } } @@ -236,10 +232,7 @@ void PostgreSQLReplicationHandler::startSynchronization(bool throw_on_error) auto * materialized_storage = storage->as (); try { - /// FIXME: Looks like it is possible we might get here if there is no nested storage or at least nested storage id field might be empty. - /// Caught it somehow when doing something else incorrectly, but do not see any reason how it could happen. - /// Try load nested table, set materialized table metadata. - nested_storages[table_name] = materialized_storage->prepare(); + nested_storages[table_name] = materialized_storage->getNested(); } catch (Exception & e) { @@ -299,7 +292,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & /// Load from snapshot, which will show table state before creation of replication slot. /// Already connected to needed database, no need to add it to query. - auto quoted_name = probablyDoubleQuoteWithSchema(table_name); + auto quoted_name = doubleQuoteWithSchema(table_name); query_str = fmt::format("SELECT * FROM {}", quoted_name); LOG_DEBUG(log, "Loading PostgreSQL table {}.{}", postgres_database, quoted_name); @@ -324,7 +317,7 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(postgres::Connection & CompletedPipelineExecutor executor(block_io.pipeline); executor.execute(); - nested_storage = materialized_storage->prepare(); + materialized_storage->set(nested_storage); auto nested_table_id = nested_storage->getStorageID(); LOG_DEBUG(log, "Loaded table {}.{} (uuid: {})", nested_table_id.database_name, nested_table_id.table_name, toString(nested_table_id.uuid)); @@ -408,7 +401,7 @@ void PostgreSQLReplicationHandler::createPublicationIfNeeded(pqxx::nontransactio WriteBufferFromOwnString buf; for (const auto & storage_data : materialized_storages) { - buf << probablyDoubleQuoteWithSchema(storage_data.first); + buf << doubleQuoteWithSchema(storage_data.first); buf << ","; } tables_list = buf.str(); @@ -690,10 +683,6 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() result_tables = fetchPostgreSQLTablesList(tx, schema_list.empty() ? postgres_schema : schema_list); } } - else - { - result_tables = std::set(expected_tables.begin(), expected_tables.end()); - } } @@ -711,7 +700,7 @@ std::set PostgreSQLReplicationHandler::fetchRequiredTables() for (auto & table_name : tables_names) { boost::trim(table_name); - buf << probablyDoubleQuoteWithSchema(table_name); + buf << doubleQuoteWithSchema(table_name); buf << ","; } tables_list = buf.str(); @@ -747,7 +736,8 @@ PostgreSQLTableStructurePtr PostgreSQLReplicationHandler::fetchTableStructure( PostgreSQLTableStructure structure; try { - structure = fetchPostgreSQLTableStructure(tx, table_name, probablyDoubleQuoteWithSchema(table_name, false), true, true, true); + auto [schema, table] = getSchemaAndTableName(table_name); + structure = fetchPostgreSQLTableStructure(tx, table, schema, true, true, true); } catch (...) { @@ -781,13 +771,9 @@ void PostgreSQLReplicationHandler::addTableToReplication(StorageMaterializedPost if (!nested) throw Exception(ErrorCodes::LOGICAL_ERROR, "Internal table was not created"); - { - postgres::Connection tmp_connection(connection_info); - nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, postgres_table_name, materialized_storage); - } - auto nested_table_id = nested_storage->getStorageID(); - materialized_storage->setNestedStorageID(nested_table_id); - nested_storage = materialized_storage->prepare(); + postgres::Connection tmp_connection(connection_info); + nested_storage = loadFromSnapshot(tmp_connection, snapshot_name, postgres_table_name, materialized_storage); + materialized_storage->set(nested_storage); } { @@ -863,6 +849,7 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectoras (); + auto materialized_table_lock = materialized_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); /// If for some reason this temporary table already exists - also drop it. auto temp_materialized_storage = materialized_storage->createTemporary(); @@ -890,34 +877,30 @@ void PostgreSQLReplicationHandler::reloadFromSnapshot(const std::vectorlockForShare(String(), context->getSettingsRef().lock_acquire_timeout); InterpreterRenameQuery(ast_rename, nested_context).execute(); - { - auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name), - nested_context); - auto nested_table_lock = nested_storage->lockForShare(String(), context->getSettingsRef().lock_acquire_timeout); - auto nested_table_id = nested_storage->getStorageID(); + auto nested_storage = DatabaseCatalog::instance().getTable(StorageID(table_id.database_name, table_id.table_name, temp_table_id.uuid), nested_context); + materialized_storage->set(nested_storage); - materialized_storage->setNestedStorageID(nested_table_id); - nested_storage = materialized_storage->prepare(); + auto nested_sample_block = nested_storage->getInMemoryMetadataPtr()->getSampleBlock(); + auto materialized_sample_block = materialized_storage->getInMemoryMetadataPtr()->getSampleBlock(); + assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); - auto nested_storage_metadata = nested_storage->getInMemoryMetadataPtr(); - auto nested_sample_block = nested_storage_metadata->getSampleBlock(); - LOG_DEBUG(log, "Updated table {}. New structure: {}", - nested_table_id.getNameForLogs(), nested_sample_block.dumpStructure()); + LOG_INFO(log, "Updated table {}. New structure: {}", + nested_storage->getStorageID().getNameForLogs(), nested_sample_block.dumpStructure()); - auto materialized_storage_metadata = nested_storage->getInMemoryMetadataPtr(); - auto materialized_sample_block = materialized_storage_metadata->getSampleBlock(); + /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. + consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); - assertBlocksHaveEqualStructure(nested_sample_block, materialized_sample_block, "while reloading table in the background"); + auto table_to_drop = DatabaseCatalog::instance().getTable(StorageID(temp_table_id.database_name, temp_table_id.table_name, table_id.uuid), nested_context); + auto drop_table_id = table_to_drop->getStorageID(); - /// Pass pointer to new nested table into replication consumer, remove current table from skip list and set start lsn position. - consumer->updateNested(table_name, nested_storage, relation_id, start_lsn); - } + if (drop_table_id == nested_storage->getStorageID()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot drop table because is has the same uuid as new table: {}", drop_table_id.getNameForLogs()); - LOG_DEBUG(log, "Dropping table {}", temp_table_id.getNameForLogs()); - InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, temp_table_id, true); + LOG_DEBUG(log, "Dropping table {}", drop_table_id.getNameForLogs()); + InterpreterDropQuery::executeDropQuery(ASTDropQuery::Kind::Drop, nested_context, nested_context, drop_table_id, true); } catch (...) { diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h index 33f049aeea9..cf44101db76 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.h @@ -89,7 +89,9 @@ private: PostgreSQLTableStructurePtr fetchTableStructure(pqxx::ReplicationTransaction & tx, const String & table_name) const; - String probablyDoubleQuoteWithSchema(const String & table_name, bool quote = true) const; + String doubleQuoteWithSchema(const String & table_name) const; + + std::pair getSchemaAndTableName(const String & table_name) const; Poco::Logger * log; ContextPtr context; diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp index f771b2239ef..3dd9c729089 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.cpp @@ -113,7 +113,7 @@ StorageMaterializedPostgreSQL::StorageMaterializedPostgreSQL( ContextPtr context_, const String & postgres_database_name, const String & postgres_table_name) - : IStorage(nested_storage_->getStorageID()) + : IStorage(StorageID(nested_storage_->getStorageID().database_name, nested_storage_->getStorageID().table_name)) , WithContext(context_->getGlobalContext()) , log(&Poco::Logger::get("StorageMaterializedPostgreSQL(" + postgres::formatNameForLogs(postgres_database_name, postgres_table_name) + ")")) , is_materialized_postgresql_database(true) @@ -214,12 +214,11 @@ std::shared_ptr StorageMaterializedPostgreSQL::makeNestedTableContext(C } -StoragePtr StorageMaterializedPostgreSQL::prepare() +void StorageMaterializedPostgreSQL::set(StoragePtr nested_storage) { - auto nested_table = getNested(); - setInMemoryMetadata(nested_table->getInMemoryMetadata()); + nested_table_id = nested_storage->getStorageID(); + setInMemoryMetadata(nested_storage->getInMemoryMetadata()); has_nested.store(true); - return nested_table; } diff --git a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h index 56fa00cc65f..0c9e82924aa 100644 --- a/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h +++ b/src/Storages/PostgreSQL/StorageMaterializedPostgreSQL.h @@ -117,14 +117,10 @@ public: StorageID getNestedStorageID() const; - void setNestedStorageID(const StorageID & id) { nested_table_id.emplace(id); } + void set(StoragePtr nested_storage); static std::shared_ptr makeNestedTableContext(ContextPtr from_context); - /// Get nested table (or throw if it does not exist), set in-memory metadata (taken from nested table) - /// for current table, set has_nested = true. - StoragePtr prepare(); - bool supportsFinal() const override { return true; } ASTPtr getCreateNestedTableQuery(PostgreSQLTableStructurePtr table_structure); diff --git a/tests/integration/test_postgresql_replica_database_engine/test.py b/tests/integration/test_postgresql_replica_database_engine/test.py index 29ab22a3a18..20c484773f4 100644 --- a/tests/integration/test_postgresql_replica_database_engine/test.py +++ b/tests/integration/test_postgresql_replica_database_engine/test.py @@ -153,7 +153,7 @@ def assert_nested_table_is_created(table_name, materialized_database='test_datab table = table_name else: table = schema_name + "." + table_name - print('Checking table exists:', table) + print(f'Checking table {table} exists in {materialized_database}') database_tables = instance.query('SHOW TABLES FROM {}'.format(materialized_database)) while table not in database_tables: time.sleep(0.2) @@ -166,11 +166,12 @@ def assert_number_of_columns(expected, table_name, database_name='test_database' while (int(result) != expected): time.sleep(1) result = instance.query(f"select count() from system.columns where table = '{table_name}' and database = '{database_name}' and not startsWith(name, '_')") + print('Number of columns ok') @pytest.mark.timeout(320) -def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database'): - assert_nested_table_is_created(table_name, materialized_database) +def check_tables_are_synchronized(table_name, order_by='key', postgres_database='postgres_database', materialized_database='test_database', schema_name=''): + assert_nested_table_is_created(table_name, materialized_database, schema_name) print("Checking table is synchronized:", table_name) expected = instance.query('select * from {}.{} order by {};'.format(postgres_database, table_name, order_by)) @@ -1182,6 +1183,66 @@ def test_predefined_connection_configuration(started_cluster): insert_counter = 0 +def test_database_with_single_non_default_schema(started_cluster): + conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) + cursor = conn.cursor() + + NUM_TABLES=5 + schema_name = 'test_schema' + clickhouse_postgres_db = 'postgres_database_with_schema' + global insert_counter + insert_counter = 0 + + def insert_into_tables(): + global insert_counter + clickhouse_postgres_db = 'postgres_database_with_schema' + for i in range(NUM_TABLES): + table_name = f'postgresql_replica_{i}' + instance.query(f"INSERT INTO {clickhouse_postgres_db}.{table_name} SELECT number, number from numbers(1000 * {insert_counter}, 1000)") + insert_counter += 1 + + def assert_show_tables(expected): + result = instance.query('SHOW TABLES FROM test_database') + assert(result == expected) + print('assert show tables Ok') + + def check_all_tables_are_synchronized(): + for i in range(NUM_TABLES): + print('checking table', i) + check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); + print('synchronization Ok') + + create_postgres_schema(cursor, schema_name) + create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, name=clickhouse_postgres_db, schema_name=schema_name) + + for i in range(NUM_TABLES): + table_name = 'postgresql_replica_{}'.format(i) + create_postgres_table_with_schema(cursor, schema_name, table_name); + + insert_into_tables() + create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + settings=[f"materialized_postgresql_schema = '{schema_name}'", "materialized_postgresql_allow_automatic_update = 1"]) + + insert_into_tables() + check_all_tables_are_synchronized() + assert_show_tables("postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") + + instance.restart_clickhouse() + check_all_tables_are_synchronized() + assert_show_tables("postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") + insert_into_tables() + check_all_tables_are_synchronized() + + print('ALTER') + altered_table = random.randint(0, NUM_TABLES-1) + cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + + instance.query(f"INSERT INTO {clickhouse_postgres_db}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(5000, 1000)") + assert_number_of_columns(3, f'postgresql_replica_{altered_table}') + check_tables_are_synchronized(f"postgresql_replica_{altered_table}", postgres_database=clickhouse_postgres_db); + drop_materialized_db() + + def test_database_with_multiple_non_default_schemas_1(started_cluster): conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor = conn.cursor() @@ -1190,6 +1251,7 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): schema_name = 'test_schema' clickhouse_postgres_db = 'postgres_database_with_schema' publication_tables = '' + global insert_counter insert_counter = 0 def insert_into_tables(): @@ -1235,15 +1297,13 @@ def test_database_with_multiple_non_default_schemas_1(started_cluster): insert_into_tables() check_all_tables_are_synchronized() - #altered_table = random.randint(0, NUM_TABLES-1) - #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) - #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - - #table_name = 'postgresql_replica_{}'.format(altered_table) - #instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) - #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - #print('Ok') + print('ALTER') + altered_table = random.randint(0, NUM_TABLES-1) + cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) + instance.query(f"INSERT INTO {clickhouse_postgres_db}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(5000, 1000)") + assert_number_of_columns(3, f'{schema_name}.postgresql_replica_{altered_table}') + check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=schema_name, postgres_database=clickhouse_postgres_db); drop_materialized_db() @@ -1254,6 +1314,7 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): NUM_TABLES = 2 schemas_num = 2 schema_list = 'schema0, schema1' + global insert_counter insert_counter = 0 def check_all_tables_are_synchronized(): @@ -1303,76 +1364,14 @@ def test_database_with_multiple_non_default_schemas_2(started_cluster): insert_into_tables() check_all_tables_are_synchronized() - #altered_table = random.randint(0, NUM_TABLES-1) - #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) - #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - - #table_name = 'postgresql_replica_{}'.format(altered_table) - #instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) - #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), schema_name=schema_name, postgres_database=clickhouse_postgres_db); - #print('Ok') - - drop_materialized_db() - - -def test_database_with_single_non_default_schema(started_cluster): - conn = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) - cursor = conn.cursor() - - NUM_TABLES=5 - schema_name = 'test_schema' - clickhouse_postgres_db = 'postgres_database_with_schema' - insert_counter = 0 - - def insert_into_tables(): - global insert_counter - clickhouse_postgres_db = 'postgres_database_with_schema' - for i in range(NUM_TABLES): - table_name = f'postgresql_replica_{i}' - instance.query(f"INSERT INTO {clickhouse_postgres_db}.{table_name} SELECT number, number from numbers(1000 * {insert_counter}, 1000)") - insert_counter += 1 - - def assert_show_tables(expected): - result = instance.query('SHOW TABLES FROM test_database') - assert(result == expected) - print('assert show tables Ok') - - def check_all_tables_are_synchronized(): - for i in range(NUM_TABLES): - print('checking table', i) - check_tables_are_synchronized("postgresql_replica_{}".format(i), postgres_database=clickhouse_postgres_db); - print('synchronization Ok') - - create_postgres_schema(cursor, schema_name) - create_clickhouse_postgres_db(ip=cluster.postgres_ip, port=cluster.postgres_port, name=clickhouse_postgres_db, schema_name=schema_name) - - for i in range(NUM_TABLES): - table_name = 'postgresql_replica_{}'.format(i) - create_postgres_table_with_schema(cursor, schema_name, table_name); - - insert_into_tables() - create_materialized_db(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, - settings=[f"materialized_postgresql_schema = '{schema_name}'", "materialized_postgresql_allow_automatic_update = 1"]) - - insert_into_tables() - check_all_tables_are_synchronized() - assert_show_tables("postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") - - instance.restart_clickhouse() - check_all_tables_are_synchronized() - assert_show_tables("postgresql_replica_0\npostgresql_replica_1\npostgresql_replica_2\npostgresql_replica_3\npostgresql_replica_4\n") - insert_into_tables() - check_all_tables_are_synchronized() - - #altered_table = random.randint(0, NUM_TABLES-1) - #cursor.execute("ALTER TABLE test_schema.postgresql_replica_{} ADD COLUMN value2 integer".format(altered_table)) - #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); - - #table_name = 'postgresql_replica_{}'.format(altered_table) - #instance.query("INSERT INTO {}.{} SELECT number, number, number from numbers(5000, 1000)".format(clickhouse_postgres_db, table_name)) - #check_tables_are_synchronized("postgresql_replica_{}".format(altered_table), postgres_database=clickhouse_postgres_db); - #print('Ok') + print('ALTER') + altered_schema = random.randint(0, schemas_num-1) + altered_table = random.randint(0, NUM_TABLES-1) + cursor.execute(f"ALTER TABLE schema{altered_schema}.postgresql_replica_{altered_table} ADD COLUMN value2 integer") + instance.query(f"INSERT INTO clickhouse_postgres_db{altered_schema}.postgresql_replica_{altered_table} SELECT number, number, number from numbers(1000 * {insert_counter}, 1000)") + assert_number_of_columns(3, f'schema{altered_schema}.postgresql_replica_{altered_table}') + check_tables_are_synchronized(f"postgresql_replica_{altered_table}", schema_name=schema_name, postgres_database=clickhouse_postgres_db); drop_materialized_db() From 90b2a67909cfedcdbae1791a5f467a54cf10f413 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 7 Nov 2021 09:35:18 +0300 Subject: [PATCH 126/609] Skip test --- src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp index fa047744def..0ece25788b5 100644 --- a/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp +++ b/src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp @@ -388,7 +388,6 @@ void MaterializedPostgreSQLConsumer::processReplicationMessage(const char * repl else table_name = relation_name; - LOG_TRACE(log, "GOT MESSAGE FOR TABLE: {}", table_name); if (!isSyncAllowed(relation_id, relation_name)) return; @@ -602,7 +601,6 @@ void MaterializedPostgreSQLConsumer::markTableAsSkipped(Int32 relation_id, const /// Empty lsn string means - continue waiting for valid lsn. skip_list.insert({relation_id, ""}); - LOG_TRACE(log, "trying to mark table as skipped: {}", relation_name); if (storages.count(relation_name)) { /// Erase cached schema identifiers. It will be updated again once table is allowed back into replication stream @@ -791,7 +789,6 @@ bool MaterializedPostgreSQLConsumer::consume(std::vector Date: Fri, 5 Nov 2021 12:21:52 +0800 Subject: [PATCH 127/609] Alter query function normalization. --- src/Interpreters/InterpreterAlterQuery.cpp | 2 ++ src/Parsers/ParserAlterQuery.cpp | 4 ++++ ..._normalize_create_alter_function_names.reference | 2 ++ .../01705_normalize_create_alter_function_names.sql | 13 +++++++++++++ 4 files changed, 21 insertions(+) create mode 100644 tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference create mode 100644 tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index b620ddf6a1e..9be2861ad2f 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ InterpreterAlterQuery::InterpreterAlterQuery(const ASTPtr & query_ptr_, ContextP BlockIO InterpreterAlterQuery::execute() { + FunctionNameNormalizer().visit(query_ptr.get()); const auto & alter = query_ptr->as(); if (alter.alter_object == ASTAlterQuery::AlterObjectType::DATABASE) return executeToDatabase(alter); diff --git a/src/Parsers/ParserAlterQuery.cpp b/src/Parsers/ParserAlterQuery.cpp index 1ea64d94fe7..edd3549f787 100644 --- a/src/Parsers/ParserAlterQuery.cpp +++ b/src/Parsers/ParserAlterQuery.cpp @@ -790,6 +790,10 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->children.push_back(command->constraint_decl); if (command->constraint) command->children.push_back(command->constraint); + if (command->projection_decl) + command->children.push_back(command->projection_decl); + if (command->projection) + command->children.push_back(command->projection); if (command->predicate) command->children.push_back(command->predicate); if (command->update_assignments) diff --git a/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference new file mode 100644 index 00000000000..00c77f1500c --- /dev/null +++ b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.reference @@ -0,0 +1,2 @@ +CREATE TABLE default.x\n(\n `i` Int32,\n INDEX mm rand() TYPE minmax GRANULARITY 1,\n INDEX nn rand() TYPE minmax GRANULARITY 1,\n PROJECTION p\n (\n SELECT max(i)\n ),\n PROJECTION p2\n (\n SELECT min(i)\n )\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/default/x\', \'r\')\nORDER BY i\nSETTINGS index_granularity = 8192 +metadata format version: 1\ndate column: \nsampling expression: \nindex granularity: 8192\nmode: 0\nsign column: \nprimary key: i\ndata format version: 1\npartition key: \nindices: mm rand() TYPE minmax GRANULARITY 1, nn rand() TYPE minmax GRANULARITY 1\nprojections: p(SELECT max(i)), p2(SELECT min(i))\ngranularity bytes: 10485760\n diff --git a/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql new file mode 100644 index 00000000000..59993f40774 --- /dev/null +++ b/tests/queries/0_stateless/01705_normalize_create_alter_function_names.sql @@ -0,0 +1,13 @@ +-- Tags: zookeeper, no-replicated-database, no-parallel + +drop table if exists x; + +create table x(i int, index mm RAND() type minmax granularity 1, projection p (select MAX(i))) engine ReplicatedMergeTree('/clickhouse/tables/{database}/x', 'r') order by i; + +alter table x add index nn RAND() type minmax granularity 1, add projection p2 (select MIN(i)); + +show create x; + +select value from system.zookeeper WHERE name = 'metadata' and path = '/clickhouse/tables/'||currentDatabase()||'/x'; + +drop table x; From 8708e00b3a32603fd89d89c825e83b98017efc59 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 8 Nov 2021 12:40:31 +0300 Subject: [PATCH 128/609] Send incremental profile events to client --- src/Common/ProfileEvents.cpp | 14 ++++++++++++++ src/Common/ProfileEvents.h | 6 ++++++ src/Server/TCPHandler.cpp | 12 ++++++++++-- src/Server/TCPHandler.h | 6 ++++++ 4 files changed, 36 insertions(+), 2 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ee44b9eb927..67ae27b1dbd 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -317,6 +317,20 @@ Counters::Snapshot::Snapshot() : counters_holder(new Count[num_counters] {}) {} +Counters::Snapshot::Snapshot(Counters::Snapshot const & other) + : Counters::Snapshot() +{ + std::memcpy(other.counters_holder.get(), counters_holder.get(), num_counters * sizeof(Count)); +} + +Counters::Snapshot operator-(Counters::Snapshot const & lhs, Counters::Snapshot const & rhs) +{ + Counters::Snapshot result; + for (Event i = 0; i < Counters::num_counters; ++i) + result.counters_holder[i] = lhs[i] - rhs[i]; + return result; +} + Counters::Snapshot Counters::getPartiallyAtomicSnapshot() const { Snapshot res; diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index c416b49dd5c..f337d47bc4f 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -62,12 +62,18 @@ namespace ProfileEvents struct Snapshot { Snapshot(); + Snapshot(Snapshot const & other); + Snapshot(Snapshot &&) = default; const Count & operator[] (Event event) const { return counters_holder[event]; } + Snapshot & operator=(Snapshot &&) = default; + + friend Snapshot operator-(Snapshot const & lhs, Snapshot const & rhs); + private: std::unique_ptr counters_holder; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index f85bd24a275..d55c004b61c 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -923,6 +923,7 @@ void TCPHandler::sendProfileEvents() auto thread_group = CurrentThread::getGroup(); auto const current_thread_id = CurrentThread::get().thread_id; std::vector snapshots; + ThreadIdToCountersSnapshot new_snapshots; ProfileEventsSnapshot group_snapshot; { std::lock_guard guard(thread_group->mutex); @@ -935,19 +936,26 @@ void TCPHandler::sendProfileEvents() auto current_time = time(nullptr); auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); auto memory_usage = thread->memory_tracker.get(); + auto previous_snapshot = last_sent_snapshots.find(thread_id); + auto increment = previous_snapshot != last_sent_snapshots.end() ? counters - previous_snapshot->second : counters; snapshots.push_back(ProfileEventsSnapshot{ thread_id, - std::move(counters), + std::move(increment), memory_usage, current_time }); + new_snapshots[thread_id] = std::move(counters); } group_snapshot.thread_id = 0; group_snapshot.current_time = time(nullptr); group_snapshot.memory_usage = thread_group->memory_tracker.get(); - group_snapshot.counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); + auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); + auto prev_group_snapshot = last_sent_snapshots.find(0); + group_snapshot.counters = prev_group_snapshot != last_sent_snapshots.end() ? group_counters - prev_group_snapshot->second : group_counters; + new_snapshots[0] = std::move(group_counters); } + last_sent_snapshots = std::move(new_snapshots); for (auto & snapshot : snapshots) { diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 73fc88526a9..3cf3346cd72 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -3,6 +3,7 @@ #include #include +#include "Common/ProfileEvents.h" #include #include #include @@ -15,6 +16,7 @@ #include #include "IServer.h" +#include "base/types.h" namespace CurrentMetrics @@ -182,6 +184,10 @@ private: CurrentMetrics::Increment metric_increment{CurrentMetrics::TCPConnection}; + using ThreadIdToCountersSnapshot = std::unordered_map; + + ThreadIdToCountersSnapshot last_sent_snapshots; + /// It is the name of the server that will be sent to the client. String server_display_name; From a093395b9babdfe098556fd67083ca2810fcd670 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 8 Nov 2021 16:38:31 +0300 Subject: [PATCH 129/609] Process incremental profile events on client --- src/Client/ClientBase.cpp | 7 +++--- src/Client/InternalTextLogs.cpp | 4 ++-- src/Common/ProfileEvents.cpp | 36 ++++++++++++++++++------------ src/Common/ProfileEvents.h | 30 ++++++++++++++++++++----- src/Common/ProgressIndication.cpp | 37 ++++++++++--------------------- src/Common/ProgressIndication.h | 10 ++++----- src/Server/TCPHandler.cpp | 16 ++++++++----- 7 files changed, 81 insertions(+), 59 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6be126ccd26..14555d2b4e4 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -700,7 +700,7 @@ void ClientBase::onProfileEvents(Block & block) const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & names = typeid_cast(*block.getByName("name").column); const auto & host_names = typeid_cast(*block.getByName("host_name").column); - const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); + const auto & array_values = typeid_cast(*block.getByName("value").column).getData(); const auto * user_time_name = ProfileEvents::getName(ProfileEvents::UserTimeMicroseconds); const auto * system_time_name = ProfileEvents::getName(ProfileEvents::SystemTimeMicroseconds); @@ -727,7 +727,8 @@ void ClientBase::onProfileEvents(Block & block) thread_times[host_name][thread_id].memory_usage = value; } } - progress_indication.updateThreadEventData(thread_times); + auto elapsed_time = profile_events.watch.elapsedMicroseconds(); + progress_indication.updateThreadEventData(thread_times, elapsed_time); } if (profile_events.print) @@ -739,7 +740,6 @@ void ClientBase::onProfileEvents(Block & block) logs_out_stream->writeProfileEvents(block); logs_out_stream->flush(); - profile_events.watch.restart(); profile_events.last_block = {}; } else @@ -747,6 +747,7 @@ void ClientBase::onProfileEvents(Block & block) profile_events.last_block = block; } } + profile_events.watch.restart(); } diff --git a/src/Client/InternalTextLogs.cpp b/src/Client/InternalTextLogs.cpp index 430ba6daf0a..4985424030a 100644 --- a/src/Client/InternalTextLogs.cpp +++ b/src/Client/InternalTextLogs.cpp @@ -105,7 +105,7 @@ void InternalTextLogs::writeProfileEvents(const Block & block) const auto & array_thread_id = typeid_cast(*block.getByName("thread_id").column).getData(); const auto & array_type = typeid_cast(*block.getByName("type").column).getData(); const auto & column_name = typeid_cast(*block.getByName("name").column); - const auto & array_value = typeid_cast(*block.getByName("value").column).getData(); + const auto & array_value = typeid_cast(*block.getByName("value").column).getData(); for (size_t row_num = 0; row_num < block.rows(); ++row_num) { @@ -146,7 +146,7 @@ void InternalTextLogs::writeProfileEvents(const Block & block) writeCString(": ", wb); /// value - UInt64 value = array_value[row_num]; + Int64 value = array_value[row_num]; writeIntText(value, wb); //// type diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 67ae27b1dbd..ddab6ce6cd4 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -317,20 +317,6 @@ Counters::Snapshot::Snapshot() : counters_holder(new Count[num_counters] {}) {} -Counters::Snapshot::Snapshot(Counters::Snapshot const & other) - : Counters::Snapshot() -{ - std::memcpy(other.counters_holder.get(), counters_holder.get(), num_counters * sizeof(Count)); -} - -Counters::Snapshot operator-(Counters::Snapshot const & lhs, Counters::Snapshot const & rhs) -{ - Counters::Snapshot result; - for (Event i = 0; i < Counters::num_counters; ++i) - result.counters_holder[i] = lhs[i] - rhs[i]; - return result; -} - Counters::Snapshot Counters::getPartiallyAtomicSnapshot() const { Snapshot res; @@ -372,6 +358,28 @@ void increment(Event event, Count amount) DB::CurrentThread::getProfileEvents().increment(event, amount); } +CountersIncrement::CountersIncrement() noexcept + : increment_holder() +{} + +CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot) +{ + init(); + std::memcpy(increment_holder.get(), snapshot.counters_holder.get(), Counters::num_counters * sizeof(Increment)); +} + +CountersIncrement::CountersIncrement(Counters::Snapshot const & after, Counters::Snapshot const & before) +{ + init(); + for (Event i = 0; i < Counters::num_counters; ++i) + increment_holder[i] = static_cast(after[i]) - static_cast(before[i]); +} + +void CountersIncrement::init() +{ + increment_holder = std::make_unique(Counters::num_counters); +} + } #undef APPLY_FOR_EVENTS diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index f337d47bc4f..98ff709d9f8 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -1,6 +1,7 @@ #pragma once #include +#include "base/types.h" #include #include #include @@ -15,6 +16,7 @@ namespace ProfileEvents /// Event identifier (index in array). using Event = size_t; using Count = size_t; + using Increment = Int64; using Counter = std::atomic; class Counters; @@ -62,22 +64,19 @@ namespace ProfileEvents struct Snapshot { Snapshot(); - Snapshot(Snapshot const & other); Snapshot(Snapshot &&) = default; - const Count & operator[] (Event event) const + Count operator[] (Event event) const noexcept { return counters_holder[event]; } Snapshot & operator=(Snapshot &&) = default; - - friend Snapshot operator-(Snapshot const & lhs, Snapshot const & rhs); - private: std::unique_ptr counters_holder; friend class Counters; + friend struct CountersIncrement; }; /// Every single value is fetched atomically, but not all values as a whole. @@ -115,4 +114,25 @@ namespace ProfileEvents /// Get index just after last event identifier. Event end(); + + struct CountersIncrement + { + CountersIncrement() noexcept; + explicit CountersIncrement(Counters::Snapshot const & snapshot); + CountersIncrement(Counters::Snapshot const & after, Counters::Snapshot const & before); + + CountersIncrement(CountersIncrement &&) = default; + CountersIncrement & operator=(CountersIncrement &&) = default; + + Increment operator[](Event event) const noexcept + { + return increment_holder[event]; + } + private: + void init(); + + static_assert(sizeof(Count) == sizeof(Increment), "Sizes of counter and increment differ"); + + std::unique_ptr increment_holder; + }; } diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 26dae8d5b73..33508f5ad5f 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -16,28 +16,16 @@ namespace { constexpr UInt64 ALL_THREADS = 0; - UInt64 calculateNewCoresNumber(DB::ThreadIdToTimeMap const & prev, DB::ThreadIdToTimeMap const& next) + UInt64 calculateCoresNumber(DB::ThreadIdToTimeMap times, UInt64 elapsed) { - if (next.find(ALL_THREADS) == next.end()) - return 0; - - auto accumulated = std::accumulate(next.cbegin(), next.cend(), 0, - [&prev](UInt64 acc, const auto & elem) - { - if (elem.first == ALL_THREADS) - return acc; - - auto thread_time = elem.second.time(); - auto it = prev.find(elem.first); - if (it != prev.end()) - thread_time -= it->second.time(); - return acc + thread_time; - }); - - auto elapsed = next.at(ALL_THREADS).time() - (prev.contains(ALL_THREADS) ? prev.at(ALL_THREADS).time() : 0); - if (elapsed == 0) - return 0; - return (accumulated + elapsed - 1) / elapsed; + auto accumulated = std::accumulate(times.begin(), times.end(), 0, + [](Int64 acc, const auto & elem) + { + if (elem.first == ALL_THREADS) + return acc; + return acc + elem.second.time(); + }); + return (static_cast(accumulated) + elapsed - 1) / elapsed; } } @@ -89,14 +77,13 @@ void ProgressIndication::addThreadIdToList(String const & host, UInt64 thread_id thread_to_times[thread_id] = {}; } -void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data) +void ProgressIndication::updateThreadEventData(HostToThreadTimesMap & new_thread_data, UInt64 elapsed_time) { for (auto & new_host_map : new_thread_data) { - auto & host_map = thread_data[new_host_map.first]; - auto new_cores = calculateNewCoresNumber(host_map, new_host_map.second); + auto new_cores = calculateCoresNumber(new_host_map.second, elapsed_time); host_active_cores[new_host_map.first] = new_cores; - host_map = std::move(new_host_map.second); + thread_data[new_host_map.first] = std::move(new_host_map.second); } } diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index d31ed8df0ba..b775279f73b 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -16,11 +16,11 @@ namespace DB struct ThreadEventData { - UInt64 time() const noexcept { return user_ms + system_ms; } + Int64 time() const noexcept { return user_ms + system_ms; } - UInt64 user_ms = 0; - UInt64 system_ms = 0; - UInt64 memory_usage = 0; + Int64 user_ms = 0; + Int64 system_ms = 0; + Int64 memory_usage = 0; }; using ThreadIdToTimeMap = std::unordered_map; @@ -58,7 +58,7 @@ public: void addThreadIdToList(String const & host, UInt64 thread_id); - void updateThreadEventData(HostToThreadTimesMap & new_thread_data); + void updateThreadEventData(HostToThreadTimesMap & new_thread_data, UInt64 elapsed_time); bool print_hardware_utilization = false; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index d55c004b61c..8d0059593e1 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -840,7 +840,7 @@ namespace struct ProfileEventsSnapshot { UInt64 thread_id; - ProfileEvents::Counters::Snapshot counters; + ProfileEvents::CountersIncrement counters; Int64 memory_usage; time_t current_time; }; @@ -858,7 +858,7 @@ namespace auto & value_column = columns[VALUE_COLUMN_INDEX]; for (ProfileEvents::Event event = 0; event < ProfileEvents::Counters::num_counters; ++event) { - UInt64 value = snapshot.counters[event]; + Int64 value = snapshot.counters[event]; if (value == 0) continue; @@ -910,7 +910,7 @@ void TCPHandler::sendProfileEvents() { "thread_id", std::make_shared() }, { "type", ProfileEvents::TypeEnum }, { "name", std::make_shared() }, - { "value", std::make_shared() }, + { "value", std::make_shared() }, }; ColumnsWithTypeAndName temp_columns; @@ -937,7 +937,10 @@ void TCPHandler::sendProfileEvents() auto counters = thread->performance_counters.getPartiallyAtomicSnapshot(); auto memory_usage = thread->memory_tracker.get(); auto previous_snapshot = last_sent_snapshots.find(thread_id); - auto increment = previous_snapshot != last_sent_snapshots.end() ? counters - previous_snapshot->second : counters; + auto increment = + previous_snapshot != last_sent_snapshots.end() + ? CountersIncrement(counters, previous_snapshot->second) + : CountersIncrement(counters); snapshots.push_back(ProfileEventsSnapshot{ thread_id, std::move(increment), @@ -952,7 +955,10 @@ void TCPHandler::sendProfileEvents() group_snapshot.memory_usage = thread_group->memory_tracker.get(); auto group_counters = thread_group->performance_counters.getPartiallyAtomicSnapshot(); auto prev_group_snapshot = last_sent_snapshots.find(0); - group_snapshot.counters = prev_group_snapshot != last_sent_snapshots.end() ? group_counters - prev_group_snapshot->second : group_counters; + group_snapshot.counters = + prev_group_snapshot != last_sent_snapshots.end() + ? CountersIncrement(group_counters, prev_group_snapshot->second) + : CountersIncrement(group_counters); new_snapshots[0] = std::move(group_counters); } last_sent_snapshots = std::move(new_snapshots); From 0cd858263fe8e0a94295d9e65dde34adb594c04f Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 8 Nov 2021 16:43:34 +0300 Subject: [PATCH 130/609] Update protocol version --- src/Core/ProtocolDefines.h | 4 ++-- src/Server/TCPHandler.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index 8a31ffd49b1..fe49810099d 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -38,7 +38,7 @@ #define DBMS_MIN_PROTOCOL_VERSION_WITH_DISTRIBUTED_DEPTH 54448 -#define DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS 54450 +#define DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS 54451 /// Version of ClickHouse TCP protocol. /// @@ -47,6 +47,6 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54450 +#define DBMS_TCP_PROTOCOL_VERSION 54451 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 8d0059593e1..02d449f91f2 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -249,7 +249,7 @@ void TCPHandler::runImpl() sendLogs(); }); } - if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS) + if (client_tcp_protocol_version >= DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) { state.profile_queue = std::make_shared(std::numeric_limits::max()); CurrentThread::attachInternalProfileEventsQueue(state.profile_queue); @@ -901,7 +901,7 @@ namespace void TCPHandler::sendProfileEvents() { - if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_PROFILE_EVENTS) + if (client_tcp_protocol_version < DBMS_MIN_PROTOCOL_VERSION_WITH_INCREMENTAL_PROFILE_EVENTS) return; NamesAndTypesList column_names_and_types = { From f5a92c6f0b1e6cb09a03314e03c5837a1ce8a1a4 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 8 Nov 2021 23:47:57 +0300 Subject: [PATCH 131/609] cleanup code --- src/Common/ProfileEvents.cpp | 4 ---- src/Common/ProfileEvents.h | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index ddab6ce6cd4..7555cb9ba18 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -358,10 +358,6 @@ void increment(Event event, Count amount) DB::CurrentThread::getProfileEvents().increment(event, amount); } -CountersIncrement::CountersIncrement() noexcept - : increment_holder() -{} - CountersIncrement::CountersIncrement(Counters::Snapshot const & snapshot) { init(); diff --git a/src/Common/ProfileEvents.h b/src/Common/ProfileEvents.h index 98ff709d9f8..22acfa8d519 100644 --- a/src/Common/ProfileEvents.h +++ b/src/Common/ProfileEvents.h @@ -117,7 +117,7 @@ namespace ProfileEvents struct CountersIncrement { - CountersIncrement() noexcept; + CountersIncrement() noexcept = default; explicit CountersIncrement(Counters::Snapshot const & snapshot); CountersIncrement(Counters::Snapshot const & after, Counters::Snapshot const & before); From 185601690e999e80214efbcb56c59fca1e1d80af Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 10 Nov 2021 16:33:34 -0400 Subject: [PATCH 132/609] docs. alter-partition-verbose-result --- docs/en/operations/settings/settings.md | 35 +++++++++++++++++++++++++ docs/ru/operations/settings/settings.md | 35 +++++++++++++++++++++++++ 2 files changed, 70 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b1e0adf74a6..ae603a2fdb0 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4048,3 +4048,38 @@ Possible values: - 0 — Timeout disabled. Default value: `0`. + +## alter_partition_verbose_result {#alter-partition-verbose-result} + +Enables or disables the display of information about the parts to which the manipulation operations with partitions and parts have been successfully applied. +Applicable to [ATTACH PARTITION|PART](../../sql-reference/statements/alteralter/partition.md#alter_attach-partition) and to [FREEZE PARTITION](../../sql-reference/statements/alteralter/partition.md#alter_freeze-partition). + +Possible values: + +- 0 — disable verbosity. +- 1 — enable verbosity. + +Default value: `0`. + +**Example** + +```sql +CREATE TABLE test(a Int64, d Date, s String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY a; +INSERT INTO test VALUES(1, '2021-01-01', ''); +INSERT INTO test VALUES(1, '2021-01-01', ''); +ALTER TABLE test DETACH PARTITION ID '202101'; + +ALTER TABLE test ATTACH PARTITION ID '202101' SETTINGS alter_partition_verbose_result = 1; + +┌─command_type─────┬─partition_id─┬─part_name────┬─old_part_name─┐ +│ ATTACH PARTITION │ 202101 │ 202101_7_7_0 │ 202101_5_5_0 │ +│ ATTACH PARTITION │ 202101 │ 202101_8_8_0 │ 202101_6_6_0 │ +└──────────────────┴──────────────┴──────────────┴───────────────┘ + +ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; + +┌─command_type─┬─partition_id─┬─part_name────┬─backup_name─┬─backup_path───────────────────┬─part_backup_path────────────────────────────────────────────┐ +│ FREEZE ALL │ 202101 │ 202101_7_7_0 │ 8 │ /var/lib/clickhouse/shadow/8/ │ /var/lib/clickhouse/shadow/8/data/default/test/202101_7_7_0 │ +│ FREEZE ALL │ 202101 │ 202101_8_8_0 │ 8 │ /var/lib/clickhouse/shadow/8/ │ /var/lib/clickhouse/shadow/8/data/default/test/202101_8_8_0 │ +└──────────────┴──────────────┴──────────────┴─────────────┴───────────────────────────────┴─────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fdbbff1f4f3..dd6cac62f9a 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3808,3 +3808,38 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `0`. +## alter_partition_verbose_result {#alter-partition-verbose-result} + +Включает или отключает вывод информации о партах, к которым были успешно применены операции манипуляции с партициями и партами. Применимо к [ATTACH PARTITION|PART](../../sql-reference/statements/alteralter/partition.md#alter_attach-partition) и к [FREEZE PARTITION](../../sql-reference/statements/alteralter/partition.md#alter_freeze-partition) + +Возможные значения: + +- 0 — отображение отключено. +- 1 — отображение включено. + +Значение по умолчанию: `0`. + +**Пример** + +```sql +CREATE TABLE test(a Int64, d Date, s String) ENGINE = MergeTree PARTITION BY toYYYYMM(d) ORDER BY a; +INSERT INTO test VALUES(1, '2021-01-01', ''); +INSERT INTO test VALUES(1, '2021-01-01', ''); +ALTER TABLE test DETACH PARTITION ID '202101'; + +ALTER TABLE test ATTACH PARTITION ID '202101' SETTINGS alter_partition_verbose_result = 1; + +┌─command_type─────┬─partition_id─┬─part_name────┬─old_part_name─┐ +│ ATTACH PARTITION │ 202101 │ 202101_7_7_0 │ 202101_5_5_0 │ +│ ATTACH PARTITION │ 202101 │ 202101_8_8_0 │ 202101_6_6_0 │ +└──────────────────┴──────────────┴──────────────┴───────────────┘ + +ALTER TABLE test FREEZE SETTINGS alter_partition_verbose_result = 1; + +┌─command_type─┬─partition_id─┬─part_name────┬─backup_name─┬─backup_path───────────────────┬─part_backup_path────────────────────────────────────────────┐ +│ FREEZE ALL │ 202101 │ 202101_7_7_0 │ 8 │ /var/lib/clickhouse/shadow/8/ │ /var/lib/clickhouse/shadow/8/data/default/test/202101_7_7_0 │ +│ FREEZE ALL │ 202101 │ 202101_8_8_0 │ 8 │ /var/lib/clickhouse/shadow/8/ │ /var/lib/clickhouse/shadow/8/data/default/test/202101_8_8_0 │ +└──────────────┴──────────────┴──────────────┴─────────────┴───────────────────────────────┴─────────────────────────────────────────────────────────────┘ +``` + + From 6babfa5888dca489dbfd5236b862a0c90e4fbb0c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Nov 2021 10:39:41 +0300 Subject: [PATCH 133/609] Update docs/ru/operations/settings/settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index dd6cac62f9a..ef19328b9aa 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3810,7 +3810,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## alter_partition_verbose_result {#alter-partition-verbose-result} -Включает или отключает вывод информации о партах, к которым были успешно применены операции манипуляции с партициями и партами. Применимо к [ATTACH PARTITION|PART](../../sql-reference/statements/alteralter/partition.md#alter_attach-partition) и к [FREEZE PARTITION](../../sql-reference/statements/alteralter/partition.md#alter_freeze-partition) +Включает или отключает вывод информации о партах, к которым были успешно применены операции манипуляции с партициями и кусками. Применимо к [ATTACH PARTITION|PART](../../sql-reference/statements/alteralter/partition.md#alter_attach-partition) и к [FREEZE PARTITION](../../sql-reference/statements/alteralter/partition.md#alter_freeze-partition) Возможные значения: From d42a454837aaeb619654065a1b324a9332202907 Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Wed, 10 Nov 2021 16:08:24 +0800 Subject: [PATCH 134/609] support some bool format --- src/Core/Settings.h | 2 + src/Core/SettingsEnums.cpp | 9 + src/Core/SettingsEnums.h | 2 + .../Serializations/SerializationBool.cpp | 56 +++++- .../Serializations/SerializationBool.h | 11 ++ src/Formats/FormatFactory.cpp | 1 + src/Formats/FormatSettings.h | 12 ++ .../MySQL/tests/gtest_create_rewritten.cpp | 6 +- .../0_stateless/02114_bool_type.reference | 180 ++++++++++++++++++ tests/queries/0_stateless/02114_bool_type.sql | 37 ++++ 10 files changed, 312 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 18bb4efba53..8cd11a9870a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -574,6 +574,8 @@ class IColumn; M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \ \ + M(BoolFormat, bool_format, FormatSettings::BoolFormat::true_false_lower_case, "Text to represent bool value. Possible values: 'true_false', 'True_False', 'T_F', 'Yes_No', 'Y_N', 'On_Off'.", 0) \ + \ M(Bool, input_format_values_interpret_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ M(Bool, input_format_values_deduce_templates_of_expressions, true, "For Values format: if the field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ M(Bool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index f5497588891..9168052aee8 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -121,4 +121,13 @@ IMPLEMENT_SETTING_ENUM(EnumComparingMode, ErrorCodes::BAD_ARGUMENTS, {{"by_names", FormatSettings::EnumComparingMode::BY_NAMES}, {"by_values", FormatSettings::EnumComparingMode::BY_VALUES}, {"by_names_case_insensitive", FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) + +IMPLEMENT_SETTING_ENUM(BoolFormat, ErrorCodes::BAD_ARGUMENTS, + {{"true_false_lower_case", FormatSettings::BoolFormat::true_false_lower_case}, + {"true_false_camel_case", FormatSettings::BoolFormat::true_false_camel_case}, + {"T_F", FormatSettings::BoolFormat::T_F}, + {"Yes_No", FormatSettings::BoolFormat::Yes_No}, + {"Y_N", FormatSettings::BoolFormat::Y_N}, + {"On_Off", FormatSettings::BoolFormat::On_Off}}) + } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 4bdb3c83ea5..ba9e6a3960d 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -170,4 +170,6 @@ DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) +DECLARE_SETTING_ENUM_WITH_RENAME(BoolFormat, FormatSettings::BoolFormat) + } diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 7d22e09f921..4108655da5e 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -38,7 +38,7 @@ void SerializationBool::deserializeText(IColumn &column, ReadBuffer &istr, const ColumnUInt8 *col = typeid_cast(&column); if (!col) { - throw Exception("Bool type can only deserialize columns of type UInt32." + column.getName(), + throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); } @@ -59,6 +59,50 @@ void SerializationBool::deserializeText(IColumn &column, ReadBuffer &istr, const throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); } +void SerializationBool::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + const auto *col = checkAndGetColumn(&column); + if (!col) + throw Exception("Bool type can only serialize columns of type UInt8." + column.getName(), + ErrorCodes::ILLEGAL_COLUMN); + int bool_format = static_cast(settings.bool_format); + if (col->getData()[row_num]) + { + ostr.write(text_true_arr[bool_format], strlen(text_true_arr[bool_format])); + } + else + { + ostr.write(text_false_arr[bool_format], strlen(text_false_arr[bool_format])); + } +} + +void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + ColumnUInt8 *col = typeid_cast(&column); + if (!col) + { + throw Exception("Bool type can only deserialize columns of type UInt8." + column.getName(), ErrorCodes::ILLEGAL_COLUMN); + } + + if (!istr.eof()) + { + int bool_format = static_cast(settings.bool_format); + const char *formatted_true = text_true_arr[bool_format]; + const char *formatted_false = text_false_arr[bool_format]; + String input; + readString(input,istr); + if (strcmp(formatted_true, input.c_str())) + { + col->insert(true); + } else if (strcmp(formatted_false, input.c_str())) + { + col->insert(false); + } else + throw Exception("Invalid boolean value, should be " + String(formatted_true) + " or " + String(formatted_false) + ".", ErrorCodes::ILLEGAL_COLUMN); + } else + throw Exception("Expected boolean value but get EOF.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING); +} + void SerializationBool::serializeTextJSON(const IColumn &column, size_t row_num, WriteBuffer &ostr, const FormatSettings &settings) const { serializeText(column, row_num, ostr, settings); @@ -69,4 +113,14 @@ void SerializationBool::deserializeTextJSON(IColumn &column, ReadBuffer &istr, c deserializeText(column, istr, settings); } +void SerializationBool::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeTextEscaped(column, row_num, ostr, settings); +} + +void SerializationBool::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeTextEscaped(column, istr, settings); +} + } diff --git a/src/DataTypes/Serializations/SerializationBool.h b/src/DataTypes/Serializations/SerializationBool.h index 19c7d844535..2bd55106df8 100644 --- a/src/DataTypes/Serializations/SerializationBool.h +++ b/src/DataTypes/Serializations/SerializationBool.h @@ -10,14 +10,25 @@ class SerializationBool final : public SerializationCustomSimpleText private: static constexpr char str_true[5] = "true"; static constexpr char str_false[6] = "false"; + + static constexpr const char * text_true_arr[6] = {"true", "True", "T", "Yes", "Y", "On"}; + static constexpr const char * text_false_arr[6] = {"false", "False", "F", "No", "N", "Off"}; + public: SerializationBool(const SerializationPtr & nested_); void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + }; } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index b3baab75cff..28adc4566b6 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -71,6 +71,7 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings) format_settings.custom.row_between_delimiter = settings.format_custom_row_between_delimiter; format_settings.date_time_input_format = settings.date_time_input_format; format_settings.date_time_output_format = settings.date_time_output_format; + format_settings.bool_format = settings.bool_format; format_settings.enable_streaming = settings.output_format_enable_streaming; format_settings.import_nested_json = settings.input_format_import_nested_json; format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index 6d422e30bc3..1b152ef5d00 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -69,6 +69,18 @@ struct FormatSettings UInt64 output_rows_in_file = 1; } avro; + enum class BoolFormat + { + true_false_lower_case, //true or false + true_false_camel_case, //True or False + T_F, //T or F + Yes_No, //Yes or No + Y_N, //Y or N + On_Off, //On or Off + }; + + BoolFormat bool_format = BoolFormat::true_false_lower_case; + struct CSV { char delimiter = ','; diff --git a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp index 6055d53cf11..5e18b0de2e0 100644 --- a/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp +++ b/src/Interpreters/MySQL/tests/gtest_create_rewritten.cpp @@ -40,7 +40,7 @@ TEST(MySQLCreateRewritten, ColumnsDataType) {"TINYINT", "Int8"}, {"SMALLINT", "Int16"}, {"MEDIUMINT", "Int32"}, {"INT", "Int32"}, {"INTEGER", "Int32"}, {"BIGINT", "Int64"}, {"FLOAT", "Float32"}, {"DOUBLE", "Float64"}, {"VARCHAR(10)", "String"}, {"CHAR(10)", "String"}, {"Date", "Date"}, {"DateTime", "DateTime"}, - {"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Int8"} + {"TIMESTAMP", "DateTime"}, {"BOOLEAN", "Bool"} }; for (const auto & [test_type, mapped_type] : test_types) @@ -104,7 +104,7 @@ TEST(MySQLCreateRewritten, PartitionPolicy) {"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"}, {"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""}, {"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"}, - {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"} + {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Bool", " PARTITION BY key"} }; for (const auto & [test_type, mapped_type, partition_policy] : test_types) @@ -135,7 +135,7 @@ TEST(MySQLCreateRewritten, OrderbyPolicy) {"INTEGER", "Int32", " PARTITION BY intDiv(key, 4294967)"}, {"BIGINT", "Int64", " PARTITION BY intDiv(key, 18446744073709551)"}, {"FLOAT", "Float32", ""}, {"DOUBLE", "Float64", ""}, {"VARCHAR(10)", "String", ""}, {"CHAR(10)", "String", ""}, {"Date", "Date", " PARTITION BY toYYYYMM(key)"}, {"DateTime", "DateTime", " PARTITION BY toYYYYMM(key)"}, - {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Int8", " PARTITION BY key"} + {"TIMESTAMP", "DateTime", " PARTITION BY toYYYYMM(key)"}, {"BOOLEAN", "Bool", " PARTITION BY key"} }; for (const auto & [test_type, mapped_type, partition_policy] : test_types) diff --git a/tests/queries/0_stateless/02114_bool_type.reference b/tests/queries/0_stateless/02114_bool_type.reference index 43be4a86c86..ab3e1287872 100644 --- a/tests/queries/0_stateless/02114_bool_type.reference +++ b/tests/queries/0_stateless/02114_bool_type.reference @@ -27,3 +27,183 @@ true true true true +False +False +False +False +False +True +True +True +True +True +False +False +False +False +False +True +True +True +True +True +F +F +F +F +F +F +T +T +T +T +T +T +F +F +F +F +F +F +T +T +T +T +T +T +false +false +false +false +false +false +false +true +true +true +true +true +true +true +false +false +false +false +false +false +false +true +true +true +true +true +true +true +No +No +No +No +No +No +No +No +Yes +Yes +Yes +Yes +Yes +Yes +Yes +Yes +No +No +No +No +No +No +No +No +Yes +Yes +Yes +Yes +Yes +Yes +Yes +Yes +N +N +N +N +N +N +N +N +N +Y +Y +Y +Y +Y +Y +Y +Y +Y +N +N +N +N +N +N +N +N +N +Y +Y +Y +Y +Y +Y +Y +Y +Y +Off +Off +Off +Off +Off +Off +Off +Off +Off +Off +On +On +On +On +On +On +On +On +On +On +Off +Off +Off +Off +Off +Off +Off +Off +Off +Off +On +On +On +On +On +On +On +On +On +On diff --git a/tests/queries/0_stateless/02114_bool_type.sql b/tests/queries/0_stateless/02114_bool_type.sql index ccc826748b7..556d352f64f 100644 --- a/tests/queries/0_stateless/02114_bool_type.sql +++ b/tests/queries/0_stateless/02114_bool_type.sql @@ -13,4 +13,41 @@ SELECT value FROM test.bool_test FORMAT JSONEachRow; SELECT toUInt64(value) FROM test.bool_test; SELECT value FROM test.bool_test where value > 0; +set bool_format='true_false_camel_case'; +INSERT INTO test.bool_test (value) FORMAT CSV True +INSERT INTO test.bool_test (value) FORMAT TSV False +SELECT value FROM test.bool_test order by value FORMAT CSV; +SELECT value FROM test.bool_test order by value FORMAT TSV; + +set bool_format='T_F'; +INSERT INTO test.bool_test (value) FORMAT CSV T +INSERT INTO test.bool_test (value) FORMAT TSV F +SELECT value FROM test.bool_test order by value FORMAT CSV; +SELECT value FROM test.bool_test order by value FORMAT TSV; + +set bool_format='true_false_lower_case'; +INSERT INTO test.bool_test (value) FORMAT CSV true +INSERT INTO test.bool_test (value) FORMAT TSV false +SELECT value FROM test.bool_test order by value FORMAT CSV; +SELECT value FROM test.bool_test order by value FORMAT TSV; + +set bool_format='Yes_No'; +INSERT INTO test.bool_test (value) FORMAT CSV Yes +INSERT INTO test.bool_test (value) FORMAT TSV No +SELECT value FROM test.bool_test order by value FORMAT CSV; +SELECT value FROM test.bool_test order by value FORMAT TSV; + +set bool_format='Y_N'; +INSERT INTO test.bool_test (value) FORMAT CSV Y +INSERT INTO test.bool_test (value) FORMAT TSV N +SELECT value FROM test.bool_test order by value FORMAT CSV; +SELECT value FROM test.bool_test order by value FORMAT TSV; + +set bool_format='On_Off'; +INSERT INTO test.bool_test (value) FORMAT CSV On +INSERT INTO test.bool_test (value) FORMAT TSV Off +SELECT value FROM test.bool_test order by value FORMAT CSV; +SELECT value FROM test.bool_test order by value FORMAT TSV; + DROP TABLE IF EXISTS test.bool_test; + From 1b5d7d2e3fcd73f16902897ad64d9696724ed4d1 Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Thu, 11 Nov 2021 17:48:24 +0800 Subject: [PATCH 135/609] fix tests --- .../0_stateless/02013_json_function_null_column.reference | 2 +- tests/queries/0_stateless/02114_bool_type.sql | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02013_json_function_null_column.reference b/tests/queries/0_stateless/02013_json_function_null_column.reference index a4ed0f4d844..b580986f548 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.reference +++ b/tests/queries/0_stateless/02013_json_function_null_column.reference @@ -16,7 +16,7 @@ c a 1 1 -1 +true 1 a \N diff --git a/tests/queries/0_stateless/02114_bool_type.sql b/tests/queries/0_stateless/02114_bool_type.sql index 556d352f64f..a087eeccbda 100644 --- a/tests/queries/0_stateless/02114_bool_type.sql +++ b/tests/queries/0_stateless/02114_bool_type.sql @@ -1,3 +1,5 @@ +-- Tags: no-parallel +-- Tag no-parallel was added here because it will create table DROP TABLE IF EXISTS test.bool_test; CREATE TABLE test.bool_test (value Bool) ENGINE = Memory; From 656b72ad58189d0fd005f88e1cbb5d7586844b71 Mon Sep 17 00:00:00 2001 From: MaxWk <610379995@qq.com> Date: Thu, 11 Nov 2021 20:06:56 +0800 Subject: [PATCH 136/609] fix build checks --- src/DataTypes/Serializations/SerializationBool.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationBool.cpp b/src/DataTypes/Serializations/SerializationBool.cpp index 4108655da5e..004e77957d8 100644 --- a/src/DataTypes/Serializations/SerializationBool.cpp +++ b/src/DataTypes/Serializations/SerializationBool.cpp @@ -91,10 +91,10 @@ void SerializationBool::deserializeTextEscaped(IColumn & column, ReadBuffer & is const char *formatted_false = text_false_arr[bool_format]; String input; readString(input,istr); - if (strcmp(formatted_true, input.c_str())) + if (strcmp(formatted_true, input.c_str()) != 0) { col->insert(true); - } else if (strcmp(formatted_false, input.c_str())) + } else if (strcmp(formatted_false, input.c_str()) != 0) { col->insert(false); } else From 51d2328d9babc3c0d686fc9cf8f047dd73072c0c Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 11 Nov 2021 15:27:38 +0300 Subject: [PATCH 137/609] Update DDLWorker.cpp --- src/Interpreters/DDLWorker.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp index d7984af30c7..c12df26f0ef 100644 --- a/src/Interpreters/DDLWorker.cpp +++ b/src/Interpreters/DDLWorker.cpp @@ -104,14 +104,24 @@ public: void unlock() { + if (zookeeper->expired()) + { + LOG_WARNING(log, "Lock is lost, because session was expired. Path: {}, message: {}", lock_path, lock_message); + return; + } + Coordination::Stat stat; std::string dummy; + /// NOTE It will throw if session expired after we checked it above bool result = zookeeper->tryGet(lock_path, dummy, &stat); if (result && stat.ephemeralOwner == zookeeper->getClientID()) zookeeper->remove(lock_path, -1); + else if (result) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Lock is lost, it has another owner. Path: {}, message: {}, owner: {}, our id: {}", + lock_path, lock_message, stat.ephemeralOwner, zookeeper->getClientID()); else - LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: {}/{}", lock_path, lock_message); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Lock is lost, node does not exist. Path: {}, message: {}", lock_path, lock_message); } bool tryLock() From 699e3d8c26725689a3cbe81be6c6cc94b2cad273 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 11 Nov 2021 17:47:31 +0300 Subject: [PATCH 138/609] Update settings.md --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index ef19328b9aa..47404b4ef55 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3810,7 +3810,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## alter_partition_verbose_result {#alter-partition-verbose-result} -Включает или отключает вывод информации о партах, к которым были успешно применены операции манипуляции с партициями и кусками. Применимо к [ATTACH PARTITION|PART](../../sql-reference/statements/alteralter/partition.md#alter_attach-partition) и к [FREEZE PARTITION](../../sql-reference/statements/alteralter/partition.md#alter_freeze-partition) +Включает или отключает вывод информации о кусках, к которым были успешно применены операции манипуляции с партициями и кусками. Применимо к [ATTACH PARTITION|PART](../../sql-reference/statements/alteralter/partition.md#alter_attach-partition) и к [FREEZE PARTITION](../../sql-reference/statements/alteralter/partition.md#alter_freeze-partition) Возможные значения: From 9b53d5a57fbe930ac752355b395e88d7b135c07a Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Fri, 12 Nov 2021 15:30:33 -0700 Subject: [PATCH 139/609] Add support case page --- website/css/main.css | 2 +- website/src/scss/components/_form.scss | 30 ++++++++++++++++ website/support/case/index.html | 26 ++++++++++++++ website/templates/support/form.html | 46 +++++++++++++++++++++++++ website/templates/support/hero.html | 10 ++++++ website/templates/support/overview.html | 16 +++++++++ 6 files changed, 129 insertions(+), 1 deletion(-) create mode 100644 website/support/case/index.html create mode 100644 website/templates/support/form.html create mode 100644 website/templates/support/hero.html create mode 100644 website/templates/support/overview.html diff --git a/website/css/main.css b/website/css/main.css index 73ff758625f..69e99307bc7 100644 --- a/website/css/main.css +++ b/website/css/main.css @@ -1 +1 @@ -@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{color:#495057;list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file +@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-group{position:relative}form .form-group.is-select:before{border-left:6px solid transparent;border-right:6px solid transparent;border-top:8px solid #6c757d;content:"";display:block;position:absolute;right:33px;top:calc(50% - 4px);z-index:10}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none;padding-right:24px;white-space:pre-wrap}form select.form-control[name=priority]{height:84px}@media screen and (max-width:767.98px){form select.form-control[name=priority]{height:104px}}@media screen and (max-width:499.98px){form select.form-control[name=priority]{height:124px}}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{color:#495057;list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file diff --git a/website/src/scss/components/_form.scss b/website/src/scss/components/_form.scss index b37b7044ce1..144b7e7301e 100644 --- a/website/src/scss/components/_form.scss +++ b/website/src/scss/components/_form.scss @@ -1,4 +1,20 @@ form { + .form-group { + position: relative; + } + + .form-group.is-select:before { + border-left: 6px solid transparent; + border-right: 6px solid transparent; + border-top: 8px solid $gray-700; + content: ''; + display: block; + position: absolute; + right: 33px; + top: calc(50% - 4px); + z-index: 10; + } + .form-control { border: 1px solid $gray-700; border-radius: 6px; @@ -23,6 +39,20 @@ form { select.form-control { appearance: none; + padding-right: 24px; + white-space: pre-wrap; + + &[name=priority] { + height: 84px; + + @media screen and (max-width: 767.98px) { + height: 104px; + } + + @media screen and (max-width: 499.98px) { + height: 124px; + } + } } select.form-control:not([data-chosen]) { diff --git a/website/support/case/index.html b/website/support/case/index.html new file mode 100644 index 00000000000..ecb0d2f6044 --- /dev/null +++ b/website/support/case/index.html @@ -0,0 +1,26 @@ +{% set prefetch_items = [ + ('/docs/en/', 'document') +] %} + +{% extends "templates/base.html" %} + +{% block extra_meta %} +{% include "templates/common_fonts.html" %} +{% endblock %} + +{% block nav %} + +{% include "templates/global/nav.html" %} + +{% endblock %} + +{% block content %} + +{% include "templates/support/hero.html" %} +{% include "templates/support/overview.html" %} +{% include "templates/support/form.html" %} + +{% include "templates/global/newsletter.html" %} +{% include "templates/global/github_stars.html" %} + +{% endblock %} diff --git a/website/templates/support/form.html b/website/templates/support/form.html new file mode 100644 index 00000000000..a85a82af543 --- /dev/null +++ b/website/templates/support/form.html @@ -0,0 +1,46 @@ +
+
+ +
+ + + + +
+
+
+ +
+
+ +
+
+ +
+
+ +
+
+ +
+
+ +
+
+
+ +
+ +
+ +
+ +
+ +
+
diff --git a/website/templates/support/hero.html b/website/templates/support/hero.html new file mode 100644 index 00000000000..c467d8ffcd5 --- /dev/null +++ b/website/templates/support/hero.html @@ -0,0 +1,10 @@ +
+
+
+ +

+ {{ _('Support') }} +

+ +
+
diff --git a/website/templates/support/overview.html b/website/templates/support/overview.html new file mode 100644 index 00000000000..4e7257a928a --- /dev/null +++ b/website/templates/support/overview.html @@ -0,0 +1,16 @@ +
+
+ +

+ Enter Your Support Case Details +

+ +

+ Describe the problem to us. Once you submit this form, a case will be generated and you will receive an email with the case information. A Support Engineer will be assigned to work with you to resolve the case. +

+

+ Your success is at the heart of everything we do. +

+ +
+
From 054626bf352132557c621f887c5e4991b0176a4e Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Fri, 12 Nov 2021 15:35:09 -0700 Subject: [PATCH 140/609] Add noindex meta tag --- website/support/case/index.html | 1 + 1 file changed, 1 insertion(+) diff --git a/website/support/case/index.html b/website/support/case/index.html index ecb0d2f6044..07f77d65d46 100644 --- a/website/support/case/index.html +++ b/website/support/case/index.html @@ -5,6 +5,7 @@ {% extends "templates/base.html" %} {% block extra_meta %} + {% include "templates/common_fonts.html" %} {% endblock %} From 3157c61d90ad566de1c983bad75cbd198c258cb3 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sat, 13 Nov 2021 23:18:59 +0300 Subject: [PATCH 141/609] Documentation of exponentialMovingAverage --- .../reference/exponentialmovingaverage.md | 66 +++++++++++++++++++ .../reference/exponentialmovingaverage.md | 66 +++++++++++++++++++ 2 files changed, 132 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md create mode 100644 docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md new file mode 100644 index 00000000000..57cd612db61 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -0,0 +1,66 @@ +## exponentialMovingAverage {#exponential-moving-average} + +An aggregate function that calculates the exponential moving average of values for the determined time. + +**Syntax** + +```sql +exponentialMovingAverage(x)(value, timestamp) +``` + +Each `value` corresponds to the determinate `timestamp`. The half-decay period is the time interval `x` during which the previous values are taken into account. The function returns a weighted average: the older the time point, the less weight the corresponding value is considered to be. + +**Arguments** +- `value` - value must be [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md), or [Decimal](../../../sql-reference/data-types/decimal.md). +- `timestamp` - timestamp must be [Integer](../../../sql-reference/data-types/int-uint.md). + +**Parameters** +- `x` - half-decay period in seconds must be [Integer](../../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** +- Returnes an exponentially smoothed moving average of the values for the past `x` time at the latest point of time. + +Type: [Float64](../../../sql-reference/data-types/float.md#float32-float64). + +**Example** + +Input table: + +``` text +┌──temperature─┬─timestamp──┐ +│ 95 │ 1 │ +│ 95 │ 2 │ +│ 95 │ 3 │ +│ 96 │ 4 │ +│ 96 │ 5 │ +│ 96 │ 6 │ +│ 96 │ 7 │ +│ 97 │ 8 │ +│ 97 │ 9 │ +│ 97 │ 10 │ +│ 97 │ 11 │ +│ 98 │ 12 │ +│ 98 │ 13 │ +│ 98 │ 14 │ +│ 98 │ 15 │ +│ 99 │ 16 │ +│ 99 │ 17 │ +│ 99 │ 18 │ +│ 100 │ 19 │ +│ 100 │ 20 │ +└──────────────┴────────────┘ +``` + +Query: + +```sql +exponentialMovingAverage(5)(temperature, timestamp) +``` + +Result: + +``` text +┌──exponentialMovingAverage(5)(temperature, timestamp)──┐ +│ 92.25779635374204 │ +└───────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md new file mode 100644 index 00000000000..c847555095d --- /dev/null +++ b/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -0,0 +1,66 @@ +## exponentialMovingAverage {#exponential-moving-average} + +Вычисляет экспоненциальное скользящее среднее. + +**Синтакис:** + +```sql +exponentialMovingAverage(x)(value, timestamp) +``` + +Каждой точке `timestamp` на временном отрезке соответствует определенное значение `value`. Период полу-распада — временной интервал `х`, в течение которого учитываются предыдущие значения. Функция возвращает взвешенное среднее: чем старше временная точка, с тем более меньшим весом считается соответствующее ей значение. + +**Аргументы:** +- `value` - входные значения, должны быть типа [Integer](../../../sql-reference/data-types/int-uint.md), [Float](../../../sql-reference/data-types/float.md) или [Decimal](../../../sql-reference/data-types/decimal.md). +- `timestamp` - параметр для упорядочивания значений, должен быть типа [Integer](../../../sql-reference/data-types/int-uint.md). + +**Параметры** +- `x` - период полу-распада в секундах, должен быть типа [Integer](../../../sql-reference/data-types/int-uint.md). + +**Возвращаемые значения:** +- Возвращает экспоненциальное скользящее среднее. + +Тип: [Float64](../../../sql-reference/data-types/float.md#float32-float64). + +**Пример** + +Исходная таблица: + +``` text +┌──temperature─┬─timestamp──┐ +│ 95 │ 1 │ +│ 95 │ 2 │ +│ 95 │ 3 │ +│ 96 │ 4 │ +│ 96 │ 5 │ +│ 96 │ 6 │ +│ 96 │ 7 │ +│ 97 │ 8 │ +│ 97 │ 9 │ +│ 97 │ 10 │ +│ 97 │ 11 │ +│ 98 │ 12 │ +│ 98 │ 13 │ +│ 98 │ 14 │ +│ 98 │ 15 │ +│ 99 │ 16 │ +│ 99 │ 17 │ +│ 99 │ 18 │ +│ 100 │ 19 │ +│ 100 │ 20 │ +└──────────────┴────────────┘ +``` + +Запрос: + +```sql +exponentialMovingAverage(5)(temperature, timestamp) +``` + +Результат: + +``` text +┌──exponentialMovingAverage(5)(temperature, timestamp)──┐ +│ 92.25779635374204 │ +└───────────────────────────────────────────────────────┘ +``` \ No newline at end of file From ac4be9751a502b635b2de18d4f32189d8dd4a9b6 Mon Sep 17 00:00:00 2001 From: Andrei Ch Date: Sun, 14 Nov 2021 16:52:07 +0300 Subject: [PATCH 142/609] toc-priority added --- .../aggregate-functions/reference/exponentialmovingaverage.md | 4 ++++ .../aggregate-functions/reference/exponentialmovingaverage.md | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index 57cd612db61..dd62eec2166 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/en/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -1,3 +1,7 @@ +--- +toc_priority: 108 +--- + ## exponentialMovingAverage {#exponential-moving-average} An aggregate function that calculates the exponential moving average of values for the determined time. diff --git a/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md b/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md index c847555095d..e286d105797 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/exponentialmovingaverage.md @@ -1,3 +1,7 @@ +--- +toc_priority: 108 +--- + ## exponentialMovingAverage {#exponential-moving-average} Вычисляет экспоненциальное скользящее среднее. From 2253b86f1fd3d3fbbb4bfcb4ecfd104caf7d75cf Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Mon, 15 Nov 2021 12:16:54 -0700 Subject: [PATCH 143/609] Update text on support page --- website/templates/support/form.html | 4 ++-- website/templates/support/hero.html | 2 +- website/templates/support/overview.html | 7 ++----- 3 files changed, 5 insertions(+), 8 deletions(-) diff --git a/website/templates/support/form.html b/website/templates/support/form.html index a85a82af543..fe895993bc5 100644 --- a/website/templates/support/form.html +++ b/website/templates/support/form.html @@ -12,14 +12,14 @@
- +
+
+ +
+ From 72e47d9cc7f9b098d9a1644a72ee294320e56fa6 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 2 Dec 2021 10:01:54 -0700 Subject: [PATCH 456/609] Update form field class --- website/templates/support/form.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/support/form.html b/website/templates/support/form.html index 4cbba83c669..14f38543544 100644 --- a/website/templates/support/form.html +++ b/website/templates/support/form.html @@ -32,7 +32,7 @@
- +
From c9ae0aec323ba713055e8ee1d49761fc28bf8941 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 2 Dec 2021 10:02:41 -0700 Subject: [PATCH 457/609] Fix margin on form field --- website/templates/support/form.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/templates/support/form.html b/website/templates/support/form.html index 14f38543544..14c153c7fde 100644 --- a/website/templates/support/form.html +++ b/website/templates/support/form.html @@ -28,7 +28,7 @@
-
+
From be660ca25f20a37d85075c5daacb29d421bb90ce Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 2 Dec 2021 10:28:47 -0700 Subject: [PATCH 458/609] Update featured image for 21.11 release blog post --- website/blog/en/2021/clickhouse-v21.11-released.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/blog/en/2021/clickhouse-v21.11-released.md b/website/blog/en/2021/clickhouse-v21.11-released.md index a10d6ce85de..d7993c4219e 100644 --- a/website/blog/en/2021/clickhouse-v21.11-released.md +++ b/website/blog/en/2021/clickhouse-v21.11-released.md @@ -1,6 +1,6 @@ --- title: 'ClickHouse v21.11 Released' -image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-11/featured.jpg' +image: 'https://blog-images.clickhouse.com/en/2021/clickhouse-v21-11/featured-dog.jpg' date: '2021-11-11' author: '[Rich Raposa](https://github.com/rfraposa), [Alexey Milovidov](https://github.com/alexey-milovidov)' tags: ['company', 'community'] From 252192bc6da32a06ca35834748fda9244e1a8a71 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 2 Dec 2021 20:51:55 +0300 Subject: [PATCH 459/609] fix active replicas count in quorum inserts --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index e3ca902b1bd..f38b70a30a8 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -76,18 +76,24 @@ void ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zoo { quorum_info.status_path = storage.zookeeper_path + "/quorum/status"; + Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); + std::vector> replicas_status_futures; + replicas_status_futures.reserve(replicas.size()); + for (const auto & replica : replicas) + if (replica != storage.replica_name) + replicas_status_futures.emplace_back(zookeeper->asyncExists(fs::path(storage.zookeeper_path) / "replicas" / replica)); + std::future is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active"); std::future host_future = zookeeper->asyncTryGet(storage.replica_path + "/host"); - /// List of live replicas. All of them register an ephemeral node for leader_election. + size_t active_replicas = 1; /// Assume current replica is active (will check below) + for (auto & status : replicas_status_futures) + if (status.get().error == Coordination::Error::ZOK) + ++active_replicas; - Coordination::Stat leader_election_stat; - zookeeper->get(storage.zookeeper_path + "/leader_election", &leader_election_stat); - - if (leader_election_stat.numChildren < static_cast(quorum)) - throw Exception("Number of alive replicas (" - + toString(leader_election_stat.numChildren) + ") is less than requested quorum (" + toString(quorum) + ").", - ErrorCodes::TOO_FEW_LIVE_REPLICAS); + if (active_replicas < quorum) + throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}).", + active_replicas, quorum); /** Is there a quorum for the last part for which a quorum is needed? * Write of all the parts with the included quorum is linearly ordered. From 46063767345c3b978035cc58fc00cd9a88792868 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 2 Dec 2021 21:51:23 +0300 Subject: [PATCH 460/609] Update ReplicatedMergeTreeSink.cpp --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index f38b70a30a8..1ce748640dc 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -81,7 +81,7 @@ void ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zoo replicas_status_futures.reserve(replicas.size()); for (const auto & replica : replicas) if (replica != storage.replica_name) - replicas_status_futures.emplace_back(zookeeper->asyncExists(fs::path(storage.zookeeper_path) / "replicas" / replica)); + replicas_status_futures.emplace_back(zookeeper->asyncExists(fs::path(storage.zookeeper_path) / "replicas" / replica / "is_active")); std::future is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active"); std::future host_future = zookeeper->asyncTryGet(storage.replica_path + "/host"); From f2140be90b1189447108f43df3088d3cccc4c53c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 2 Dec 2021 22:03:59 +0300 Subject: [PATCH 461/609] Fix CI --- tests/ci/build_check.py | 2 +- tests/ci/metrics_lambda/app.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 292dd33bdce..28974662745 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -130,6 +130,7 @@ if __name__ == "__main__": gh = Github(get_best_robot_token()) s3_helper = S3Helper('https://s3.amazonaws.com') + version = get_version_from_repo(repo_path) release_or_pr = None if 'release' in pr_info.labels or 'release-lts' in pr_info.labels: # for release pull requests we use branch names prefixes, not pr numbers @@ -162,7 +163,6 @@ if __name__ == "__main__": docker_image = get_image_with_version(os.getenv("IMAGES_PATH"), image_name) image_version = docker_image.version - version = get_version_from_repo(repo_path) logging.info("Got version from repo %s", version.get_version_string()) version_type = 'testing' diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index d6717f76801..fbe695c4500 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -122,7 +122,7 @@ def group_runners_by_tag(listed_runners): def push_metrics_to_cloudwatch(listed_runners, namespace): client = boto3.client('cloudwatch') metrics_data = [] - busy_runners = sum(1 for runner in listed_runners if runner.busy) + busy_runners = sum(1 for runner in listed_runners if runner.busy and not runner.offline) metrics_data.append({ 'MetricName': 'BusyRunners', 'Value': busy_runners, @@ -178,6 +178,7 @@ def main(github_secret_key, github_app_id, push_to_cloudwatch, delete_offline_ru grouped_runners = group_runners_by_tag(runners) for group, group_runners in grouped_runners.items(): if push_to_cloudwatch: + print(group) push_metrics_to_cloudwatch(group_runners, 'RunnersMetrics/' + group) else: print(group, f"({len(group_runners)})") From 06312a2df9b9256e2d89468ef7bf2afe68e2a4f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 2 Dec 2021 22:48:21 +0300 Subject: [PATCH 462/609] Fix --- .../test_postgresql_replica_database_engine_2/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 16832b8f34e..99f2facbaf6 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -360,7 +360,7 @@ def test_predefined_connection_configuration(started_cluster): cursor.execute(f'CREATE TABLE test_table (key integer PRIMARY KEY, value integer)') cursor.execute(f'INSERT INTO test_table SELECT 1, 2') - instance.query("CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1)") + instance.query("CREATE DATABASE test_database ENGINE = MaterializedPostgreSQL(postgres1) SETTINGS materialized_postgresql_tables_list='test_table'") check_tables_are_synchronized("test_table"); drop_materialized_db() cursor.execute('DROP TABLE IF EXISTS test_table') From e895c85f49182e9835e67431afe9842d08813384 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Fri, 3 Dec 2021 06:00:02 +0800 Subject: [PATCH 463/609] Add exception/exception_code to trace span log (#32040) --- src/Interpreters/executeQuery.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2ab4167176f..62964180cba 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -320,6 +320,12 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr span.attribute_names.push_back("clickhouse.query_id"); span.attribute_values.push_back(elem.client_info.current_query_id); + span.attribute_names.push_back("clickhouse.exception"); + span.attribute_values.push_back(elem.exception); + + span.attribute_names.push_back("clickhouse.exception_code"); + span.attribute_values.push_back(elem.exception_code); + if (!context->query_trace_context.tracestate.empty()) { span.attribute_names.push_back("clickhouse.tracestate"); From bfaaa79cfcb5b379bda0587107a0458cff65b42e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 01:28:32 +0300 Subject: [PATCH 464/609] tests/ci: do not compress logs twice Cc: @alesapin --- tests/ci/s3_helper.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/ci/s3_helper.py b/tests/ci/s3_helper.py index 713e3653bf2..7c1ee8ad9ee 100644 --- a/tests/ci/s3_helper.py +++ b/tests/ci/s3_helper.py @@ -49,7 +49,9 @@ class S3Helper(): else: logging.info("No content type provied for %s", file_path) else: - if s3_path.endswith("txt") or s3_path.endswith("log") or ".log." in s3_path or s3_path.endswith("err") or s3_path.endswith("out"): + is_log = s3_path.endswith("log") or ".log." in s3_path + is_text = s3_path.endswith("txt") or is_log or s3_path.endswith("err") or s3_path.endswith("out") + if not s3_path.endswith('.gz') and (is_text or is_log): logging.info("Going to compress file log file %s to %s", file_path, file_path + ".gz") compress_file_fast(file_path, file_path + ".gz") file_path += ".gz" From 927638497756f8c54b56c4864133161991f67f5d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 01:29:23 +0300 Subject: [PATCH 465/609] clickhouse-test: do not use random generator with shared state MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Recently (#32094) test database had been overlapped, and random prefix for database had been increased from 6 to 8. But actually 6 bytes for random prefix should be enough (with existing alphabet (0-9a-z) it is 36**6=2'176'782'336), and the real reason of this overlap is that random generator by default uses shared state [1]: The functions supplied by this module are actually bound methods of a hidden instance of the random.Random class. You can instantiate your own instances of Random to get generators that don’t share state. [1]: https://docs.python.org/3/library/random.html I've played a little bit with random in python, and using default random generator it generates non-unique strings pretty fast, just in a few runs, but using SystemRandom (that uses /dev/urandom) it takes ~1 minute. Test: ```sh $ while /tmp/test.py | LANG=c sort -S5G | LANG=c uniq -d | tee /dev/stderr | wc -l | fgrep -q -x -c 0; do :; done ``` ```python #!/usr/bin/env python3 import multiprocessing import string import random def random_str(length=6): alphabet = string.ascii_lowercase + string.digits return ''.join(random.SystemRandom().choice(alphabet) for _ in range(length)) def worker(_): print(random_str()) with multiprocessing.Pool(processes=2) as pool: pool.map(worker, range(0, int(10e3))) ``` So let's switch to SystemRandom and use 6-byte prefix. --- tests/clickhouse-test | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 4fca8a8163b..01d632a1f50 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -371,9 +371,10 @@ class TestCase: else: # If --database is not specified, we will create temporary database with unique name # And we will recreate and drop it for each test - def random_str(length=8): + def random_str(length=6): alphabet = string.ascii_lowercase + string.digits - return ''.join(random.choice(alphabet) for _ in range(length)) + # NOTE: it is important not to use default random generator, since it shares state. + return ''.join(random.SystemRandom().choice(alphabet) for _ in range(length)) database = 'test_{suffix}'.format(suffix=random_str()) From 96bd83c31ed5646a0acc0fa77478f6a1eb0c4c46 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 01:32:57 +0300 Subject: [PATCH 466/609] Do not reopen logs on USR1, HUP is enough USR1 is also used for query_profiler_real_time_period_ns, let's not overlap. --- base/daemon/BaseDaemon.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 524fc5bbb49..b92a68f104e 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -104,7 +104,7 @@ static void writeSignalIDtoSignalPipe(int sig) errno = saved_errno; } -/** Signal handler for HUP / USR1 */ +/** Signal handler for HUP */ static void closeLogsSignalHandler(int sig, siginfo_t *, void *) { DENY_ALLOCATIONS_IN_SCOPE; @@ -161,7 +161,7 @@ __attribute__((__weak__)) void collectCrashLog( /** The thread that read info about signal or std::terminate from pipe. - * On HUP / USR1, close log files (for new files to be opened later). + * On HUP, close log files (for new files to be opened later). * On information about std::terminate, write it to log. * On other signals, write info to log. */ @@ -201,7 +201,7 @@ public: LOG_INFO(log, "Stop SignalListener thread"); break; } - else if (sig == SIGHUP || sig == SIGUSR1) + else if (sig == SIGHUP) { LOG_DEBUG(log, "Received signal to close logs."); BaseDaemon::instance().closeLogs(BaseDaemon::instance().logger()); @@ -832,7 +832,7 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. addSignalHandler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP, SIGTRAP}, signalHandler, &handled_signals); - addSignalHandler({SIGHUP, SIGUSR1}, closeLogsSignalHandler, &handled_signals); + addSignalHandler({SIGHUP}, closeLogsSignalHandler, &handled_signals); addSignalHandler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler, &handled_signals); #if defined(SANITIZER) @@ -1006,7 +1006,7 @@ void BaseDaemon::setupWatchdog() /// Forward signals to the child process. addSignalHandler( - {SIGHUP, SIGUSR1, SIGINT, SIGQUIT, SIGTERM}, + {SIGHUP, SIGINT, SIGQUIT, SIGTERM}, [](int sig, siginfo_t *, void *) { /// Forward all signals except INT as it can be send by terminal to the process group when user press Ctrl+C, From 0b2de3222818a76b6f248fbb1e6f3d0dd7b5b11e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 01:32:58 +0300 Subject: [PATCH 467/609] Fix QueryProfiler (query_profiler_{cpu,real}_time_period_ns) reset Even after timer_delete() the signal can be fired. Reproducer: $ clickhouse-server & # with configured trace_log $ clickhouse-benchmark -c2 --query 'select * from numbers(1e6)' --query_profiler_cpu_time_period_ns=1 & ... 2021.12.02 14:28:01.320288 [ 24885 ] {} TCPHandler: Processed in 177.055205644 sec. User defined signal 2 CI failures: - https://s3.amazonaws.com/clickhouse-test-reports/32067/8dbc7a8dae17090a18778f29629d8746a1bb9b72/stateful_tests__debug__actions_.html - https://s3.amazonaws.com/clickhouse-test-reports/32064/c07450a7dce363b7a4c5ca3ab0e833c25e3d46c0/stateful_tests__debug__actions_.html Fix this by do not reset the signal back, and introduce a flag to ignore signals after disabling the timer. Fixes: #31740 --- src/Common/QueryProfiler.cpp | 29 ++++++++++++++++++++++------- src/Common/QueryProfiler.h | 3 --- 2 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 457443c7dc1..12410de6bf0 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -25,6 +25,12 @@ namespace { #if defined(OS_LINUX) thread_local size_t write_trace_iteration = 0; + /// Even after timer_delete() the signal can be delivered, + /// since it does not do anything with pending signals. + /// + /// And so to overcome this flag is exists, + /// to ignore delivered signals after timer_delete(). + thread_local bool signal_handler_disarmed = true; #endif void writeTraceInfo(TraceType trace_type, int /* sig */, siginfo_t * info, void * context) @@ -117,10 +123,8 @@ QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const if (sigaddset(&sa.sa_mask, pause_signal)) throwFromErrno("Failed to add signal to mask for query profiler", ErrorCodes::CANNOT_MANIPULATE_SIGSET); - struct sigaction local_previous_handler; - if (sigaction(pause_signal, &sa, &local_previous_handler)) + if (sigaction(pause_signal, &sa, nullptr)) throwFromErrno("Failed to setup signal handler for query profiler", ErrorCodes::CANNOT_SET_SIGNAL_HANDLER); - previous_handler.emplace(local_previous_handler); try { @@ -160,6 +164,8 @@ QueryProfilerBase::QueryProfilerBase(const UInt64 thread_id, const struct itimerspec timer_spec = {.it_interval = interval, .it_value = offset}; if (timer_settime(*timer_id, 0, &timer_spec, nullptr)) throwFromErrno("Failed to set thread timer period", ErrorCodes::CANNOT_SET_TIMER_PERIOD); + + signal_handler_disarmed = false; } catch (...) { @@ -179,11 +185,14 @@ template void QueryProfilerBase::tryCleanup() { #if USE_UNWIND - if (timer_id.has_value() && timer_delete(*timer_id)) - LOG_ERROR(log, "Failed to delete query profiler timer {}", errnoToString(ErrorCodes::CANNOT_DELETE_TIMER)); + if (timer_id.has_value()) + { + if (timer_delete(*timer_id)) + LOG_ERROR(log, "Failed to delete query profiler timer {}", errnoToString(ErrorCodes::CANNOT_DELETE_TIMER)); + timer_id.reset(); + } - if (previous_handler.has_value() && sigaction(pause_signal, &*previous_handler, nullptr)) - LOG_ERROR(log, "Failed to restore signal handler after query profiler {}", errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER)); + signal_handler_disarmed = true; #endif } @@ -196,6 +205,9 @@ QueryProfilerReal::QueryProfilerReal(const UInt64 thread_id, const UInt32 period void QueryProfilerReal::signalHandler(int sig, siginfo_t * info, void * context) { + if (signal_handler_disarmed) + return; + DENY_ALLOCATIONS_IN_SCOPE; writeTraceInfo(TraceType::Real, sig, info, context); } @@ -206,6 +218,9 @@ QueryProfilerCPU::QueryProfilerCPU(const UInt64 thread_id, const UInt32 period) void QueryProfilerCPU::signalHandler(int sig, siginfo_t * info, void * context) { + if (signal_handler_disarmed) + return; + DENY_ALLOCATIONS_IN_SCOPE; writeTraceInfo(TraceType::CPU, sig, info, context); } diff --git a/src/Common/QueryProfiler.h b/src/Common/QueryProfiler.h index 5d011c6adfc..5cd64ecb76d 100644 --- a/src/Common/QueryProfiler.h +++ b/src/Common/QueryProfiler.h @@ -46,9 +46,6 @@ private: /// Pause signal to interrupt threads to get traces int pause_signal; - - /// Previous signal handler to restore after query profiler exits - std::optional previous_handler; }; /// Query profiler with timer based on real clock From 344298f4037f88b114b8e798bb30036b24be8f16 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 1 Dec 2021 21:53:43 +0300 Subject: [PATCH 468/609] Fix CAST from Nullable with cast_keep_nullable UBsan report [1]: SELECT toUInt32OrDefault(toNullable(0)) SETTINGS cast_keep_nullable = 1 ../src/Common/assert_cast.h:50:12: runtime error: downcast of address 0x000029461170 which does not point to an object of type 'const DB::ColumnNullable' 0x000029461170: note: object is of type 'DB::ColumnVector' 00 00 00 00 a0 51 3d 09 00 00 00 00 01 00 00 00 00 00 00 00 a0 3f 4e 29 00 00 00 00 a4 3f 4e 29 ^~~~~~~~~~~~~~~~~~~~~~~ vptr for 'DB::ColumnVector' 0 0x1e3e279d in DB::ColumnNullable const& assert_cast(DB::IColumn const&) obj-x86_64-linux-gnu/../src/Common/assert_cast.h:50:12 1 0x1e3e279d in DB::ColumnNullable::insertRangeFrom(DB::IColumn const&, unsigned long, unsigned long) obj-x86_64-linux-gnu/../src/Columns/ColumnNullable.cpp:167:43 2 0x12db4507 in DB::FunctionCastOrDefault::executeImpl() const (/src/ch/tmp/32019/clickhouse-ubsan+0x12db4507) [1]: https://s3.amazonaws.com/clickhouse-test-reports/32019/9e1222a50b1805f696f4205e451365f8acdbc8a5/fuzzer_astfuzzerubsan,actions//report.html And w/o -fsanitize=undefined: SELECT toUInt32OrDefault(toNullable(1), toNullable(toUInt32(1))) SETTINGS cast_keep_nullable = 1 Received exception: Code: 12. DB::Exception: Parameters start = 0, length = 1 are out of bound in ColumnVector::insertRangeFrom method (data.size() = 0).: While processing toUInt32OrDefault(toNullable(1), toNullable(toUInt32(1))). (PARAMETER_OUT_OF_BOUND) --- src/Functions/castOrDefault.cpp | 25 ++++++++++++++++--- .../0_stateless/02128_cast_nullable.reference | 9 +++++++ .../0_stateless/02128_cast_nullable.sql | 5 ++++ 3 files changed, 36 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02128_cast_nullable.reference create mode 100644 tests/queries/0_stateless/02128_cast_nullable.sql diff --git a/src/Functions/castOrDefault.cpp b/src/Functions/castOrDefault.cpp index 7394e0f36f9..622059d0a49 100644 --- a/src/Functions/castOrDefault.cpp +++ b/src/Functions/castOrDefault.cpp @@ -110,6 +110,10 @@ public: IColumn::MutablePtr result = return_type->createColumn(); result->reserve(null_map_data_size); + ColumnNullable * result_nullable = nullptr; + if (result->isNullable()) + result_nullable = assert_cast(&*result); + size_t start_insert_index = 0; /// Created separate branch because cast and inserting field from other column is slower @@ -125,7 +129,12 @@ public: continue; if (i != start_insert_index) - result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); + { + if (result_nullable) + result_nullable->insertRangeFromNotNullable(nested_column, start_insert_index, i - start_insert_index); + else + result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); + } result->insertFrom(*default_column, i); start_insert_index = i + 1; @@ -140,7 +149,12 @@ public: continue; if (i != start_insert_index) - result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); + { + if (result_nullable) + result_nullable->insertRangeFromNotNullable(nested_column, start_insert_index, i - start_insert_index); + else + result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); + } result->insertDefault(); start_insert_index = i + 1; @@ -148,7 +162,12 @@ public: } if (null_map_data_size != start_insert_index) - result->insertRangeFrom(nested_column, start_insert_index, null_map_data_size - start_insert_index); + { + if (result_nullable) + result_nullable->insertRangeFromNotNullable(nested_column, start_insert_index, null_map_data_size - start_insert_index); + else + result->insertRangeFrom(nested_column, start_insert_index, null_map_data_size - start_insert_index); + } return result; } diff --git a/tests/queries/0_stateless/02128_cast_nullable.reference b/tests/queries/0_stateless/02128_cast_nullable.reference new file mode 100644 index 00000000000..26d85f14a3e --- /dev/null +++ b/tests/queries/0_stateless/02128_cast_nullable.reference @@ -0,0 +1,9 @@ +-- { echo } +SELECT toUInt32OrDefault(toNullable(toUInt32(1))) SETTINGS cast_keep_nullable=1; +1 +SELECT toUInt32OrDefault(toNullable(toUInt32(1)), toNullable(toUInt32(2))) SETTINGS cast_keep_nullable=1; +1 +SELECT toUInt32OrDefault(toUInt32(1)) SETTINGS cast_keep_nullable=1; +1 +SELECT toUInt32OrDefault(toUInt32(1), toUInt32(2)) SETTINGS cast_keep_nullable=1; +1 diff --git a/tests/queries/0_stateless/02128_cast_nullable.sql b/tests/queries/0_stateless/02128_cast_nullable.sql new file mode 100644 index 00000000000..fec686d791b --- /dev/null +++ b/tests/queries/0_stateless/02128_cast_nullable.sql @@ -0,0 +1,5 @@ +-- { echo } +SELECT toUInt32OrDefault(toNullable(toUInt32(1))) SETTINGS cast_keep_nullable=1; +SELECT toUInt32OrDefault(toNullable(toUInt32(1)), toNullable(toUInt32(2))) SETTINGS cast_keep_nullable=1; +SELECT toUInt32OrDefault(toUInt32(1)) SETTINGS cast_keep_nullable=1; +SELECT toUInt32OrDefault(toUInt32(1), toUInt32(2)) SETTINGS cast_keep_nullable=1; From 9093ad65969ed293481be85ffbf5f6a2bc70f130 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 Dec 2021 09:27:22 +0300 Subject: [PATCH 469/609] Update materialized-postgresql.md --- docs/en/engines/database-engines/materialized-postgresql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 902fdde88e0..d2c4dbf1f3c 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -70,7 +70,7 @@ ALTER DATABASE postgres_database MODIFY SETTING materialized_postgresql_max_bloc ## PostgreSQL schema {#schema} -PostgreSQL [schema](https://www.postgresql.org/docs/9.1/ddl-schemas.html) can be used in two ways. +PostgreSQL [schema](https://www.postgresql.org/docs/9.1/ddl-schemas.html) can be configured in 3 ways (starting from version 21.12). 1. One schema for one `MaterializedPostgreSQL` database engine. Requires to use setting `materialized_postgresql_schema`. Tables are accessed via table name only: From 4fbfc7c56f9c2e95c1498f055bc003c9bfc0e831 Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 3 Dec 2021 15:08:03 +0800 Subject: [PATCH 470/609] MaterializedMySQL support VARBINARY type --- src/DataTypes/DataTypeString.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/DataTypes/DataTypeString.cpp b/src/DataTypes/DataTypeString.cpp index 84610557d21..7fa3a394be8 100644 --- a/src/DataTypes/DataTypeString.cpp +++ b/src/DataTypes/DataTypeString.cpp @@ -91,5 +91,6 @@ void registerDataTypeString(DataTypeFactory & factory) factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive); factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive); + factory.registerAlias("VARBINARY", "String", DataTypeFactory::CaseInsensitive); } } From fb0cc625090c265c4d9d399db2688d51f44b18eb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 10:52:13 +0300 Subject: [PATCH 471/609] perf: fix waiting of the server after running tests killall requires strict match, i.e. "clickhouse-server" not "clickhouse": 2021-12-03 05:24:56 + env kill -- -21700 2021-12-03 05:24:56 kill: (-21700): No such process 2021-12-03 05:24:56 + killall clickhouse 2021-12-03 05:24:56 clickhouse: no process found 2021-12-03 05:24:56 + echo Servers stopped. 2021-12-03 05:24:56 Servers stopped. 2021-12-03 05:24:56 + analyze_queries $ tail -n1 *-server-log.log ==> left-server-log.log <== 2021.12.03 05:26:59.530647 [ 450 ] {} SystemLog (system.asynchronous_metric_log): Flushed system log up to offset 1668052 ==> right-server-log.log <== 2021.12.03 05:27:20.873136 [ 466 ] {} SystemLog (system.metric_log): Flushed system log up to offset 9605 ==> setup-server-log.log <== 2021.12.03 02:47:14.844395 [ 96 ] {} Application: Child process exited normally with code 0. As you can see killall instantly fails with no such process, while this cannot be true since it was there, and also according to logs there were messages after running analyze_queries() from compare.sh This should fix problems like in [1]. [1]: https://clickhouse-test-reports.s3.yandex.net/32080/344298f4037f88b114b8e798bb30036b24be8f16/performance_comparison/report.html#fail1 --- docker/test/performance-comparison/compare.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index b6a06be2ac7..2fefe856eea 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -1409,7 +1409,7 @@ case "$stage" in while env kill -- -$watchdog_pid ; do sleep 1; done # Stop the servers to free memory for the subsequent query analysis. - while killall clickhouse; do echo . ; sleep 1 ; done + while killall clickhouse-server; do echo . ; sleep 1 ; done echo Servers stopped. ;& "analyze_queries") From 3e96b28843e94159af35bfbd7bf0de3c3cfbc2c0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 10:56:42 +0300 Subject: [PATCH 472/609] perf: convert killall to pkill (since killall has some magic, see -e option) --- docker/test/performance-comparison/compare.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 2fefe856eea..c32b50a3cbe 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -61,7 +61,7 @@ function configure cp -rv right/config left ||: # Start a temporary server to rename the tables - while killall clickhouse-server; do echo . ; sleep 1 ; done + while pkill clickhouse-serv; do echo . ; sleep 1 ; done echo all killed set -m # Spawn temporary in its own process groups @@ -88,7 +88,7 @@ function configure clickhouse-client --port $LEFT_SERVER_PORT --query "create database test" ||: clickhouse-client --port $LEFT_SERVER_PORT --query "rename table datasets.hits_v1 to test.hits" ||: - while killall clickhouse-server; do echo . ; sleep 1 ; done + while pkill clickhouse-serv; do echo . ; sleep 1 ; done echo all killed # Make copies of the original db for both servers. Use hardlinks instead @@ -106,7 +106,7 @@ function configure function restart { - while killall clickhouse-server; do echo . ; sleep 1 ; done + while pkill clickhouse-serv; do echo . ; sleep 1 ; done echo all killed # Change the jemalloc settings here. @@ -1409,7 +1409,7 @@ case "$stage" in while env kill -- -$watchdog_pid ; do sleep 1; done # Stop the servers to free memory for the subsequent query analysis. - while killall clickhouse-server; do echo . ; sleep 1 ; done + while pkill clickhouse-serv; do echo . ; sleep 1 ; done echo Servers stopped. ;& "analyze_queries") From ec0bfa7bcf0aa07a4cc8d399e8e7b18a6435390c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 3 Dec 2021 11:33:16 +0300 Subject: [PATCH 473/609] Better output for some actions --- tests/ci/build_check.py | 5 ++-- tests/ci/docs_check.py | 22 ++++++++-------- tests/ci/fast_test_check.py | 5 ++-- tests/ci/functional_test_check.py | 14 +++++----- tests/ci/integration_test_check.py | 14 +++++----- tests/ci/metrics_lambda/app.py | 2 +- tests/ci/pr_info.py | 41 ++++++++++++++++++++++++++++++ tests/ci/pvs_check.py | 14 +++++++--- tests/ci/stress_check.py | 14 +++++----- tests/ci/tee_popen.py | 36 ++++++++++++++++++++++++++ tests/ci/unit_tests_check.py | 14 +++++----- 11 files changed, 134 insertions(+), 47 deletions(-) create mode 100644 tests/ci/tee_popen.py diff --git a/tests/ci/build_check.py b/tests/ci/build_check.py index 28974662745..36db7d596c9 100644 --- a/tests/ci/build_check.py +++ b/tests/ci/build_check.py @@ -14,6 +14,7 @@ from version_helper import get_version_from_repo, update_version_local from ccache_utils import get_ccache_if_not_exists, upload_ccache from ci_config import CI_CONFIG from docker_pull_helper import get_image_with_version +from tee_popen import TeePopen def get_build_config(build_check_name, build_name): @@ -77,8 +78,8 @@ def get_image_name(build_config): def build_clickhouse(packager_cmd, logs_path): build_log_path = os.path.join(logs_path, 'build_log.log') - with open(build_log_path, 'w') as log_file: - retcode = subprocess.Popen(packager_cmd, shell=True, stderr=log_file, stdout=log_file).wait() + with TeePopen(packager_cmd, build_log_path) as process: + retcode = process.wait() if retcode == 0: logging.info("Built successfully") else: diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index 0430d566b38..87c327f2776 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -13,6 +13,7 @@ from commit_status_helper import post_commit_status, get_commit from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen NAME = "Docs Check (actions)" @@ -55,17 +56,16 @@ if __name__ == "__main__": run_log_path = os.path.join(test_output, 'runlog.log') - with open(run_log_path, 'w', encoding='utf-8') as log: - with subprocess.Popen(cmd, shell=True, stderr=log, stdout=log) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - status = "success" - description = "Docs check passed" - else: - description = "Docs check failed (non zero exit code)" - status = "failure" - logging.info("Run failed") + with TeePopen(cmd, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + status = "success" + description = "Docs check passed" + else: + description = "Docs check failed (non zero exit code)" + status = "failure" + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) files = os.listdir(test_output) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 7953dcdf5d3..6e1f3d17580 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -16,6 +16,7 @@ from commit_status_helper import post_commit_status from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen NAME = 'Fast test (actions)' @@ -101,8 +102,8 @@ if __name__ == "__main__": os.makedirs(logs_path) run_log_path = os.path.join(logs_path, 'runlog.log') - with open(run_log_path, 'w') as log: - retcode = subprocess.Popen(run_cmd, shell=True, stderr=log, stdout=log).wait() + with TeePopen(run_cmd, run_log_path) as process: + retcode = process.wait() if retcode == 0: logging.info("Run successfully") else: diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 8ab4fc3c9c4..15b9ab44b31 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -18,6 +18,7 @@ from commit_status_helper import post_commit_status, get_commit from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen def get_additional_envs(check_name): if 'DatabaseReplicated' in check_name: @@ -160,13 +161,12 @@ if __name__ == "__main__": run_command = get_run_command(packages_path, result_path, server_log_path, kill_timeout, additional_envs, docker_image, flaky_check, tests_to_run) logging.info("Going to run func tests: %s", run_command) - with open(run_log_path, 'w', encoding='utf-8') as log: - with subprocess.Popen(run_command, shell=True, stderr=log, stdout=log) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - else: - logging.info("Run failed") + with TeePopen(run_command, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index c4950c93422..4a60d825687 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -19,6 +19,7 @@ from commit_status_helper import post_commit_status from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen DOWNLOAD_RETRIES_COUNT = 5 @@ -146,13 +147,12 @@ if __name__ == "__main__": runner_path = os.path.join(repo_path, "tests/integration", "ci-runner.py") run_command = f"sudo -E {runner_path} | tee {output_path_log}" - with open(output_path_log, 'w', encoding='utf-8') as log: - with subprocess.Popen(run_command, shell=True, stderr=log, stdout=log, env=my_env) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run tests successfully") - else: - logging.info("Some tests failed") + with TeePopen(run_command, output_path_log, my_env) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run tests successfully") + else: + logging.info("Some tests failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index fbe695c4500..dff0a7d715e 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -104,7 +104,7 @@ def list_runners(access_token): def group_runners_by_tag(listed_runners): result = {} - RUNNER_TYPE_LABELS = ['style-checker', 'builder', 'func-tester', 'stress-tester'] + RUNNER_TYPE_LABELS = ['style-checker', 'builder', 'func-tester', 'stress-tester', 'fuzzer-unit-tester'] for runner in listed_runners: for tag in runner.tags: if tag in RUNNER_TYPE_LABELS: diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a26b19f4bc9..88d4595bc66 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -145,6 +145,47 @@ class PRInfo: return True return False + def can_skip_builds_and_use_version_from_master(self): + if 'force tests' in self.labels: + return False + + if self.changed_files is None or not self.changed_files: + return False + + for f in self.changed_files: + if (not f.startswith('tests/queries') + or not f.startswith('tests/integration') + or not f.startswith('tests/performance')): + return False + + return True + + def can_skip_integration_tests(self): + if 'force tests' in self.labels: + return False + + if self.changed_files is None or not self.changed_files: + return False + + for f in self.changed_files: + if not f.startswith('tests/queries') or not f.startswith('tests/performance'): + return False + + return True + + def can_skip_functional_tests(self): + if 'force tests' in self.labels: + return False + + if self.changed_files is None or not self.changed_files: + return False + + for f in self.changed_files: + if not f.startswith('tests/integration') or not f.startswith('tests/performance'): + return False + + return True + class FakePRInfo: def __init__(self): diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 1ce1c5839f4..19218035eb8 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -16,6 +16,7 @@ from commit_status_helper import get_commit from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen NAME = 'PVS Studio (actions)' LICENCE_NAME = 'Free license: ClickHouse, Yandex' @@ -73,9 +74,16 @@ if __name__ == "__main__": cmd = f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --volume={repo_path}:/repo_folder --volume={temp_path}:/test_output -e LICENCE_NAME='{LICENCE_NAME}' -e LICENCE_KEY='{licence_key}' {docker_image}" commit = get_commit(gh, pr_info.sha) - try: - subprocess.check_output(cmd, shell=True) - except: + run_log_path = os.path.join(temp_path, 'run_log.log') + + with TeePopen(cmd, run_log_path) as process: + retcode = process.wait() + if retcode != 0: + logging.info("Run failed") + else: + logging.info("Run Ok") + + if retcode != 0: commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") sys.exit(1) diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 254f0f6c199..7ec1a978cb9 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -18,6 +18,7 @@ from commit_status_helper import post_commit_status from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen def get_run_command(build_path, result_folder, server_log_folder, image): @@ -107,13 +108,12 @@ if __name__ == "__main__": run_command = get_run_command(packages_path, result_path, server_log_path, docker_image) logging.info("Going to run func tests: %s", run_command) - with open(run_log_path, 'w', encoding='utf-8') as log: - with subprocess.Popen(run_command, shell=True, stderr=log, stdout=log) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - else: - logging.info("Run failed") + with TeePopen(run_command, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py new file mode 100644 index 00000000000..81ac498bbd0 --- /dev/null +++ b/tests/ci/tee_popen.py @@ -0,0 +1,36 @@ +#!/usr/bin/env python3 + +from subprocess import Popen, PIPE, STDOUT +import sys +import os + + +# Very simple tee logic implementation. You can specify shell command, output +# logfile and env variables. After TeePopen is created you can only wait until +# it finishes. stderr and stdout will be redirected both to specified file and +# stdout. +class TeePopen: + def __init__(self, command, log_file, env=os.environ.copy()): + self.command = command + self.log_file = log_file + self.env = env + + def __enter__(self): + self.process = Popen(self.command, shell=True, universal_newlines=True, env=self.env, stderr=STDOUT, stdout=PIPE, bufsize=1) + self.log_file = open(self.log_file, 'w', encoding='utf-8') + return self + + def __exit__(self, t, value, traceback): + for line in self.process.stdout: + sys.stdout.write(line) + self.log_file.write(line) + + self.process.wait() + self.log_file.close() + + def wait(self): + for line in self.process.stdout: + sys.stdout.write(line) + self.log_file.write(line) + + return self.process.wait() diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 0c2cff083d5..abccbcd4512 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -17,6 +17,7 @@ from commit_status_helper import post_commit_status from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from tee_popen import TeePopen IMAGE_NAME = 'clickhouse/unit-test' @@ -128,13 +129,12 @@ if __name__ == "__main__": logging.info("Going to run func tests: %s", run_command) - with open(run_log_path, 'w', encoding='utf-8') as log: - with subprocess.Popen(run_command, shell=True, stderr=log, stdout=log) as process: - retcode = process.wait() - if retcode == 0: - logging.info("Run successfully") - else: - logging.info("Run failed") + with TeePopen(run_command, run_log_path) as process: + retcode = process.wait() + if retcode == 0: + logging.info("Run successfully") + else: + logging.info("Run failed") subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) From 5d7dfc6eb9a2b7af3e354be68d072a2c2849e1bb Mon Sep 17 00:00:00 2001 From: SuperDJY Date: Fri, 3 Dec 2021 17:06:58 +0800 Subject: [PATCH 474/609] fix storage join settings with persistency (#32066) --- src/Storages/JoinSettings.cpp | 41 ------------------- src/Storages/JoinSettings.h | 30 -------------- src/Storages/StorageJoin.cpp | 4 +- src/Storages/StorageJoin.h | 1 - ...e_join_settings_with_persistency.reference | 2 + ...storage_join_settings_with_persistency.sql | 6 +++ 6 files changed, 9 insertions(+), 75 deletions(-) delete mode 100644 src/Storages/JoinSettings.cpp delete mode 100644 src/Storages/JoinSettings.h create mode 100644 tests/queries/0_stateless/02127_storage_join_settings_with_persistency.reference create mode 100644 tests/queries/0_stateless/02127_storage_join_settings_with_persistency.sql diff --git a/src/Storages/JoinSettings.cpp b/src/Storages/JoinSettings.cpp deleted file mode 100644 index 8a2699746da..00000000000 --- a/src/Storages/JoinSettings.cpp +++ /dev/null @@ -1,41 +0,0 @@ -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_SETTING; -} - -IMPLEMENT_SETTINGS_TRAITS(joinSettingsTraits, LIST_OF_JOIN_SETTINGS) - -void JoinSettings::loadFromQuery(ASTStorage & storage_def) -{ - if (storage_def.settings) - { - try - { - applyChanges(storage_def.settings->changes); - } - catch (Exception & e) - { - if (e.code() == ErrorCodes::UNKNOWN_SETTING) - e.addMessage("for storage " + storage_def.engine->name); - throw; - } - } - else - { - auto settings_ast = std::make_shared(); - settings_ast->is_standalone = false; - storage_def.set(storage_def.settings, settings_ast); - } -} - -} diff --git a/src/Storages/JoinSettings.h b/src/Storages/JoinSettings.h deleted file mode 100644 index 13be557a439..00000000000 --- a/src/Storages/JoinSettings.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ -class ASTStorage; - - -#define JOIN_RELATED_SETTINGS(M) \ - M(Bool, persistent, true, "Disable setting to avoid the overhead of writing to disk for StorageJoin", 0) - -#define LIST_OF_JOIN_SETTINGS(M) \ - JOIN_RELATED_SETTINGS(M) \ - FORMAT_FACTORY_SETTINGS(M) - -DECLARE_SETTINGS_TRAITS(joinSettingsTraits, LIST_OF_JOIN_SETTINGS) - - -/** Settings for the Join engine. - * Could be loaded from a CREATE TABLE query (SETTINGS clause). - */ -struct JoinSettings : public BaseSettings -{ - void loadFromQuery(ASTStorage & storage_def); -}; - -} diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 6c903b22846..3a9b42f7371 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -261,9 +261,7 @@ void registerStorageJoin(StorageFactory & factory) disk_name = setting.value.get(); else if (setting.name == "persistent") { - auto join_settings = std::make_unique(); - join_settings->loadFromQuery(*args.storage_def); - persistent = join_settings->persistent; + persistent = setting.value.get(); } else throw Exception("Unknown setting " + setting.name + " for storage " + args.engine_name, ErrorCodes::BAD_ARGUMENTS); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 1b9d34c46fa..ee685830a6c 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -5,7 +5,6 @@ #include #include #include -#include #include diff --git a/tests/queries/0_stateless/02127_storage_join_settings_with_persistency.reference b/tests/queries/0_stateless/02127_storage_join_settings_with_persistency.reference new file mode 100644 index 00000000000..16f7a201276 --- /dev/null +++ b/tests/queries/0_stateless/02127_storage_join_settings_with_persistency.reference @@ -0,0 +1,2 @@ +CREATE TABLE default.`02127_join_settings_with_persistency_1`\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Join(ANY, LEFT, k)\nSETTINGS persistent = 1, join_any_take_last_row = 0 +CREATE TABLE default.`02127_join_settings_with_persistency_0`\n(\n `k` UInt64,\n `s` String\n)\nENGINE = Join(ANY, LEFT, k)\nSETTINGS persistent = 0, join_any_take_last_row = 0 diff --git a/tests/queries/0_stateless/02127_storage_join_settings_with_persistency.sql b/tests/queries/0_stateless/02127_storage_join_settings_with_persistency.sql new file mode 100644 index 00000000000..1dc1529ebad --- /dev/null +++ b/tests/queries/0_stateless/02127_storage_join_settings_with_persistency.sql @@ -0,0 +1,6 @@ +DROP TABLE IF EXISTS 02127_join_settings_with_persistency_1; +CREATE TABLE 02127_join_settings_with_persistency_1 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=1, join_any_take_last_row=0; +SHOW CREATE TABLE 02127_join_settings_with_persistency_1; +DROP TABLE IF EXISTS 02127_join_settings_with_persistency_0; +CREATE TABLE 02127_join_settings_with_persistency_0 (k UInt64, s String) ENGINE = Join(ANY, LEFT, k) SETTINGS persistent=0, join_any_take_last_row=0; +SHOW CREATE TABLE 02127_join_settings_with_persistency_0; From 99a5ca2646f9adfa8478a8d36dac2ccf299451f8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 3 Dec 2021 12:19:39 +0300 Subject: [PATCH 475/609] Fix style (got tired of it) --- tests/ci/pvs_check.py | 1 - tests/ci/tee_popen.py | 2 ++ 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 19218035eb8..aa4a130902b 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -2,7 +2,6 @@ # pylint: disable=line-too-long -import subprocess import os import json import logging diff --git a/tests/ci/tee_popen.py b/tests/ci/tee_popen.py index 81ac498bbd0..cbb915e6de7 100644 --- a/tests/ci/tee_popen.py +++ b/tests/ci/tee_popen.py @@ -10,12 +10,14 @@ import os # it finishes. stderr and stdout will be redirected both to specified file and # stdout. class TeePopen: + # pylint: disable=W0102 def __init__(self, command, log_file, env=os.environ.copy()): self.command = command self.log_file = log_file self.env = env def __enter__(self): + # pylint: disable=W0201 self.process = Popen(self.command, shell=True, universal_newlines=True, env=self.env, stderr=STDOUT, stdout=PIPE, bufsize=1) self.log_file = open(self.log_file, 'w', encoding='utf-8') return self From caa481aba835910939953810a07e07914eb6cd50 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 3 Dec 2021 12:28:17 +0300 Subject: [PATCH 476/609] Use ccache in fast test --- tests/ci/fast_test_check.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 7953dcdf5d3..14375394d35 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -16,6 +16,7 @@ from commit_status_helper import post_commit_status from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch from rerun_helper import RerunHelper +from ccache_utils import get_ccache_if_not_exists, upload_ccache NAME = 'Fast test (actions)' @@ -86,7 +87,12 @@ if __name__ == "__main__": os.makedirs(output_path) cache_path = os.path.join(caches_path, "fasttest") + + logging.info("Will try to fetch cache for our build") + get_ccache_if_not_exists(cache_path, s3_helper, pr_info.number, temp_path) + if not os.path.exists(cache_path): + logging.info("cache was not fetched, will create empty dir") os.makedirs(cache_path) repo_path = os.path.join(temp_path, "fasttest-repo") @@ -137,6 +143,9 @@ if __name__ == "__main__": else: state, description, test_results, additional_logs = process_results(output_path) + logging.info("Will upload cache") + upload_ccache(cache_path, s3_helper, pr_info.number, temp_path) + ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, NAME, test_results) From 6ec559f103998cbd81061fee0a24fd162794564e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Dec 2021 12:48:28 +0300 Subject: [PATCH 477/609] Update JSONEachRowRowOutputFormat.h --- src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 12e1259d319..147169b5e91 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -26,8 +26,8 @@ public: public: /// Content-Type to set when sending HTTP response. String getContentType() const override - { - return settings.json.array_of_rows ? "application/json; charset=UTF-8" : IRowOutputFormat::getContentType(); + { + return settings.json.array_of_rows ? "application/json; charset=UTF-8" : IRowOutputFormat::getContentType(); } protected: From d8604acb8e4626033d1394c7cb1c25a2db78a020 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 3 Dec 2021 13:18:24 +0300 Subject: [PATCH 478/609] Bump From 2d3f77314703193a8e4f87527f91c0f8686e135f Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Dec 2021 13:25:55 +0300 Subject: [PATCH 479/609] Function accurateCastOrDefault remove separate branch --- src/Functions/castOrDefault.cpp | 66 +++++++++++++++------------------ 1 file changed, 30 insertions(+), 36 deletions(-) diff --git a/src/Functions/castOrDefault.cpp b/src/Functions/castOrDefault.cpp index 622059d0a49..95046d95176 100644 --- a/src/Functions/castOrDefault.cpp +++ b/src/Functions/castOrDefault.cpp @@ -107,7 +107,7 @@ public: const auto & null_map_data = cast_result_nullable.getNullMapData(); size_t null_map_data_size = null_map_data.size(); const auto & nested_column = cast_result_nullable.getNestedColumn(); - IColumn::MutablePtr result = return_type->createColumn(); + auto result = return_type->createColumn(); result->reserve(null_map_data_size); ColumnNullable * result_nullable = nullptr; @@ -116,49 +116,43 @@ public: size_t start_insert_index = 0; - /// Created separate branch because cast and inserting field from other column is slower + Field default_value; + ColumnPtr default_column; + if (arguments.size() == 3) { - const auto & default_column_with_type = arguments[2]; - auto default_column = default_column_with_type.column->convertToFullColumnIfConst(); + auto default_values_column = arguments[2].column; - for (size_t i = 0; i < null_map_data_size; ++i) - { - bool is_current_index_null = null_map_data[i]; - if (!is_current_index_null) - continue; - - if (i != start_insert_index) - { - if (result_nullable) - result_nullable->insertRangeFromNotNullable(nested_column, start_insert_index, i - start_insert_index); - else - result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); - } - - result->insertFrom(*default_column, i); - start_insert_index = i + 1; - } + if (isColumnConst(*default_values_column)) + default_value = (*default_values_column)[0]; + else + default_column = default_values_column->convertToFullColumnIfConst(); } else { - for (size_t i = 0; i < null_map_data_size; ++i) + default_value = return_type->getDefault(); + } + + for (size_t i = 0; i < null_map_data_size; ++i) + { + bool is_current_index_null = null_map_data[i]; + if (!is_current_index_null) + continue; + + if (i != start_insert_index) { - bool is_current_index_null = null_map_data[i]; - if (!is_current_index_null) - continue; - - if (i != start_insert_index) - { - if (result_nullable) - result_nullable->insertRangeFromNotNullable(nested_column, start_insert_index, i - start_insert_index); - else - result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); - } - - result->insertDefault(); - start_insert_index = i + 1; + if (result_nullable) + result_nullable->insertRangeFromNotNullable(nested_column, start_insert_index, i - start_insert_index); + else + result->insertRangeFrom(nested_column, start_insert_index, i - start_insert_index); } + + if (default_column) + result->insertFrom(*default_column, i); + else + result->insert(default_value); + + start_insert_index = i + 1; } if (null_map_data_size != start_insert_index) From 0470525ea3b6fdd29f13bee24a5bf4d60720cf54 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Dec 2021 12:48:18 +0300 Subject: [PATCH 480/609] Fix --- src/Storages/WindowView/StorageWindowView.cpp | 23 ++++++++++++++++++- src/Storages/WindowView/StorageWindowView.h | 2 ++ .../01052_window_view_proc_tumble_to_now.sql | 6 ++--- .../01053_window_view_proc_hop_to_now.sql | 6 ++--- 4 files changed, 30 insertions(+), 7 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 510140c4f1d..47e4037b2ad 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -142,6 +142,7 @@ namespace bool is_time_column_func_now = false; String window_id_name; + String now_timezone; void visit(ASTFunction & node, ASTPtr & node_ptr) { @@ -150,6 +151,16 @@ namespace if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") { + if (!t->children.empty()) + { + const auto & children = t->children[0]->as()->children; + if (!children.empty()) + { + const auto * timezone_ast = children[0]->as(); + if (timezone_ast) + now_timezone = timezone_ast->value.safeGet(); + } + } is_time_column_func_now = true; node_ptr->children[0]->children[0] = std::make_shared("____timestamp"); window_id_name = node.getColumnName(); @@ -609,7 +620,10 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( auto node = ast->clone(); /// now() -> ____timestamp if (is_time_column_func_now) + { time_now_visitor.visit(node); + function_now_timezone = time_now_data.now_timezone; + } /// TUMBLE/HOP -> WINDOW_ID func_window_visitor.visit(node); to_identifier_visitor.visit(node); @@ -640,7 +654,10 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( ASTPtr order_by_ptr = order_by; if (is_time_column_func_now) + { time_now_visitor.visit(order_by_ptr); + function_now_timezone = time_now_data.now_timezone; + } to_identifier_visitor.visit(order_by_ptr); for (auto & child : order_by->arguments->children) @@ -1165,7 +1182,11 @@ void StorageWindowView::writeIntoWindowView( { ColumnWithTypeAndName column; column.name = "____timestamp"; - column.type = std::make_shared(); + const auto & timezone = window_view.function_now_timezone; + if (timezone.empty()) + column.type = std::make_shared(); + else + column.type = std::make_shared(timezone); column.column = column.type->createColumnConst(0, Field(std::time(nullptr))); auto adding_column_dag = ActionsDAG::makeAddingColumnActions(std::move(column)); diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index f71e7a986d8..893647add79 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -206,6 +206,8 @@ private: BackgroundSchedulePool::TaskHolder clean_cache_task; BackgroundSchedulePool::TaskHolder fire_task; + String function_now_timezone; + ASTPtr innerQueryParser(ASTSelectQuery & inner_query); void eventTimeParser(const ASTCreateQuery & query); diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql index 106628cfd03..89fcb243be3 100644 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql @@ -2,17 +2,17 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS wv NO DELAY; DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now(), INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now('US/Samoa'), INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); SELECT sleep(2); SELECT count from dst; -DROP TABLE wv; +DROP TABLE wv NO DELAY; DROP TABLE mt; DROP TABLE dst; diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql index 42100ef1508..fdc448947e6 100644 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql @@ -2,17 +2,17 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv; +DROP TABLE IF EXISTS wv NO DELAY; DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(now('US/Samoa'), INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); SELECT sleep(2); SELECT count from dst; -DROP TABLE wv; +DROP TABLE wv NO DELAY; DROP TABLE mt; DROP TABLE dst; From 4bbb02bbae52f8d6807c1db7d411e5a1fc438257 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Dec 2021 14:06:58 +0300 Subject: [PATCH 481/609] RangeHashedDictionary added update_field support --- .../ClickHouseDictionarySource.cpp | 78 ++++---- src/Dictionaries/RangeHashedDictionary.cpp | 172 ++++++++++++------ src/Dictionaries/RangeHashedDictionary.h | 11 +- .../test_dictionaries_update_field/test.py | 2 +- 4 files changed, 156 insertions(+), 107 deletions(-) diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index edca02b83ad..1ddcdd96454 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -230,74 +230,64 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) std::string settings_config_prefix = config_prefix + ".clickhouse"; - std::unique_ptr configuration; + std::string host = config.getString(settings_config_prefix + ".host", "localhost"); + std::string user = config.getString(settings_config_prefix + ".user", "default"); + std::string password = config.getString(settings_config_prefix + ".password", ""); + std::string db = config.getString(settings_config_prefix + ".db", default_database); + std::string table = config.getString(settings_config_prefix + ".table", ""); + UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); + auto named_collection = created_from_ddl ? getExternalDataSourceConfiguration(config, settings_config_prefix, global_context) : std::nullopt; + if (named_collection) { - std::string host = named_collection->host; - UInt16 port = named_collection->port; - configuration = std::make_unique( - ClickHouseDictionarySource::Configuration{ - .host = host, - .user = named_collection->username, - .password = named_collection->password, - .db = named_collection->database, - .table = named_collection->table, - .query = config.getString(settings_config_prefix + ".query", ""), - .where = config.getString(settings_config_prefix + ".where", ""), - .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), - .update_field = config.getString(settings_config_prefix + ".update_field", ""), - .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), - .port = port, - .is_local = isLocalAddress({host, port}, default_port), - .secure = config.getBool(settings_config_prefix + ".secure", false) - }); - } - else - { - std::string host = config.getString(settings_config_prefix + ".host", "localhost"); - UInt16 port = static_cast(config.getUInt(settings_config_prefix + ".port", default_port)); - configuration = std::make_unique( - ClickHouseDictionarySource::Configuration{ - .host = host, - .user = config.getString(settings_config_prefix + ".user", "default"), - .password = config.getString(settings_config_prefix + ".password", ""), - .db = config.getString(settings_config_prefix + ".db", default_database), - .table = config.getString(settings_config_prefix + ".table", ""), - .query = config.getString(settings_config_prefix + ".query", ""), - .where = config.getString(settings_config_prefix + ".where", ""), - .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), - .update_field = config.getString(settings_config_prefix + ".update_field", ""), - .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), - .port = port, - .is_local = isLocalAddress({host, port}, default_port), - .secure = config.getBool(settings_config_prefix + ".secure", false) - }); + host = named_collection->host; + user = named_collection->username; + password = named_collection->password; + db = named_collection->database; + table = named_collection->table; + port = named_collection->port; } + ClickHouseDictionarySource::Configuration configuration{ + .host = host, + .user = user, + .password = password, + .db = db, + .table = table, + .query = config.getString(settings_config_prefix + ".query", ""), + .where = config.getString(settings_config_prefix + ".where", ""), + .invalidate_query = config.getString(settings_config_prefix + ".invalidate_query", ""), + .update_field = config.getString(settings_config_prefix + ".update_field", ""), + .update_lag = config.getUInt64(settings_config_prefix + ".update_lag", 1), + .port = port, + .is_local = isLocalAddress({host, port}, default_port), + .secure = config.getBool(settings_config_prefix + ".secure", false)}; + ContextMutablePtr context; - if (configuration->is_local) + if (configuration.is_local) { /// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication). Session session(global_context, ClientInfo::Interface::LOCAL); - session.authenticate(configuration->user, configuration->password, {}); + session.authenticate(configuration.user, configuration.password, {}); context = session.makeQueryContext(); } else { context = Context::createCopy(global_context); } + context->applySettingsChanges(readSettingsFromDictionaryConfig(config, config_prefix)); String dictionary_name = config.getString(".dictionary.name", ""); String dictionary_database = config.getString(".dictionary.database", ""); - if (dictionary_name == configuration->table && dictionary_database == configuration->db) + if (dictionary_name == configuration.table && dictionary_database == configuration.db) throw Exception(ErrorCodes::BAD_ARGUMENTS, "ClickHouseDictionarySource table cannot be dictionary table"); - return std::make_unique(dict_struct, *configuration, sample_block, context); + return std::make_unique(dict_struct, configuration, sample_block, context); }; factory.registerSource("clickhouse", create_table_source); diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index 42d6a0c0c03..bc871a8cdcf 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -78,12 +78,14 @@ RangeHashedDictionary::RangeHashedDictionary( const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - bool require_nonempty_) + bool require_nonempty_, + BlockPtr update_field_loaded_block_) : IDictionary(dict_id_) , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) , require_nonempty(require_nonempty_) + , update_field_loaded_block(std::move(update_field_loaded_block_)) { createAttributes(); loadData(); @@ -295,7 +297,6 @@ void RangeHashedDictionary::createAttributes() for (const auto & attribute : dict_struct.attributes) { - attribute_index_by_name.emplace(attribute.name, attributes.size()); attributes.push_back(createAttribute(attribute)); if (attribute.hierarchical) @@ -307,68 +308,21 @@ void RangeHashedDictionary::createAttributes() template void RangeHashedDictionary::loadData() { - QueryPipeline pipeline(source_ptr->loadAll()); - - PullingPipelineExecutor executor(pipeline); - Block block; - while (executor.pull(block)) + if (!source_ptr->hasUpdateField()) { - size_t skip_keys_size_offset = dict_struct.getKeysSize(); + QueryPipeline pipeline(source_ptr->loadAll()); - Columns key_columns; - key_columns.reserve(skip_keys_size_offset); - - /// Split into keys columns and attribute columns - for (size_t i = 0; i < skip_keys_size_offset; ++i) - key_columns.emplace_back(block.safeGetByPosition(i).column); - - DictionaryKeysArenaHolder arena_holder; - DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); - const size_t keys_size = keys_extractor.getKeysSize(); - - element_count += keys_size; - - // Support old behaviour, where invalid date means 'open range'. - const bool is_date = isDate(dict_struct.range_min->type); - - const auto & min_range_column = unwrapNullableColumn(*block.safeGetByPosition(skip_keys_size_offset).column); - const auto & max_range_column = unwrapNullableColumn(*block.safeGetByPosition(skip_keys_size_offset + 1).column); - - skip_keys_size_offset += 2; - - for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { - const auto & attribute_column = *block.safeGetByPosition(attribute_index + skip_keys_size_offset).column; - auto & attribute = attributes[attribute_index]; - - for (size_t key_index = 0; key_index < keys_size; ++key_index) - { - auto key = keys_extractor.extractCurrentKey(); - - RangeStorageType lower_bound; - RangeStorageType upper_bound; - - if (is_date) - { - lower_bound = getColumnIntValueOrDefault(min_range_column, key_index, is_date, 0); - upper_bound = getColumnIntValueOrDefault(max_range_column, key_index, is_date, DATE_LUT_MAX_DAY_NUM + 1); - } - else - { - lower_bound = getColumnIntValueOrDefault(min_range_column, key_index, is_date, RANGE_MIN_NULL_VALUE); - upper_bound = getColumnIntValueOrDefault(max_range_column, key_index, is_date, RANGE_MAX_NULL_VALUE); - } - - if constexpr (std::is_same_v) - key = copyKeyInArena(key); - - setAttributeValue(attribute, key, Range{lower_bound, upper_bound}, attribute_column[key_index]); - keys_extractor.rollbackCurrentKey(); - } - - keys_extractor.reset(); + blockToAttributes(block); } } + else + { + updateData(); + } if (require_nonempty && 0 == element_count) throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, @@ -497,6 +451,106 @@ void RangeHashedDictionary::getItemsImpl( found_count.fetch_add(keys_found, std::memory_order_relaxed); } +template +void RangeHashedDictionary::updateData() +{ + if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) + { + QueryPipeline pipeline(source_ptr->loadUpdatedAll()); + + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + { + /// We are using this to keep saved data if input stream consists of multiple blocks + if (!update_field_loaded_block) + update_field_loaded_block = std::make_shared(block.cloneEmpty()); + + for (size_t attribute_index = 0; attribute_index < block.columns(); ++attribute_index) + { + const IColumn & update_column = *block.getByPosition(attribute_index).column.get(); + MutableColumnPtr saved_column = update_field_loaded_block->getByPosition(attribute_index).column->assumeMutable(); + saved_column->insertRangeFrom(update_column, 0, update_column.size()); + } + } + } + else + { + static constexpr size_t range_columns_size = 2; + + auto pipe = source_ptr->loadUpdatedAll(); + mergeBlockWithPipe( + dict_struct.getKeysSize() + range_columns_size, + *update_field_loaded_block, + std::move(pipe)); + } + + if (update_field_loaded_block) + { + blockToAttributes(*update_field_loaded_block.get()); + } +} + +template +void RangeHashedDictionary::blockToAttributes(const Block & block [[maybe_unused]]) +{ + size_t skip_keys_size_offset = dict_struct.getKeysSize(); + + Columns key_columns; + key_columns.reserve(skip_keys_size_offset); + + /// Split into keys columns and attribute columns + for (size_t i = 0; i < skip_keys_size_offset; ++i) + key_columns.emplace_back(block.safeGetByPosition(i).column); + + DictionaryKeysArenaHolder arena_holder; + DictionaryKeysExtractor keys_extractor(key_columns, arena_holder.getComplexKeyArena()); + const size_t keys_size = keys_extractor.getKeysSize(); + + element_count += keys_size; + + // Support old behaviour, where invalid date means 'open range'. + const bool is_date = isDate(dict_struct.range_min->type); + + const auto & min_range_column = unwrapNullableColumn(*block.safeGetByPosition(skip_keys_size_offset).column); + const auto & max_range_column = unwrapNullableColumn(*block.safeGetByPosition(skip_keys_size_offset + 1).column); + + skip_keys_size_offset += 2; + + for (size_t attribute_index = 0; attribute_index < attributes.size(); ++attribute_index) + { + const auto & attribute_column = *block.safeGetByPosition(attribute_index + skip_keys_size_offset).column; + auto & attribute = attributes[attribute_index]; + + for (size_t key_index = 0; key_index < keys_size; ++key_index) + { + auto key = keys_extractor.extractCurrentKey(); + + RangeStorageType lower_bound; + RangeStorageType upper_bound; + + if (is_date) + { + lower_bound = getColumnIntValueOrDefault(min_range_column, key_index, is_date, 0); + upper_bound = getColumnIntValueOrDefault(max_range_column, key_index, is_date, DATE_LUT_MAX_DAY_NUM + 1); + } + else + { + lower_bound = getColumnIntValueOrDefault(min_range_column, key_index, is_date, RANGE_MIN_NULL_VALUE); + upper_bound = getColumnIntValueOrDefault(max_range_column, key_index, is_date, RANGE_MAX_NULL_VALUE); + } + + if constexpr (std::is_same_v) + key = copyKeyInArena(key); + + setAttributeValue(attribute, key, Range{lower_bound, upper_bound}, attribute_column[key_index]); + keys_extractor.rollbackCurrentKey(); + } + + keys_extractor.reset(); + } +} + template template void RangeHashedDictionary::setAttributeValueImpl(Attribute & attribute, KeyType key, const Range & range, const Field & value) diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 1ccd9708d79..1605e2bab81 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -39,7 +39,8 @@ public: const DictionaryStructure & dict_struct_, DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, - bool require_nonempty_); + bool require_nonempty_, + BlockPtr update_field_loaded_block_ = nullptr); std::string getTypeName() const override { return "RangeHashed"; } @@ -63,7 +64,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); + return std::make_shared(getDictionaryID(), dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, update_field_loaded_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -156,6 +157,10 @@ private: ValueSetter && set_value, DefaultValueExtractor & default_value_extractor) const; + void updateData(); + + void blockToAttributes(const Block & block); + template static void setAttributeValueImpl(Attribute & attribute, KeyType key, const Range & range, const Field & value); @@ -185,8 +190,8 @@ private: const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; const bool require_nonempty; + BlockPtr update_field_loaded_block; - std::map attribute_index_by_name; std::vector attributes; Arena complex_key_arena; diff --git a/tests/integration/test_dictionaries_update_field/test.py b/tests/integration/test_dictionaries_update_field/test.py index 2e46403c63b..8fb0d67e8b8 100644 --- a/tests/integration/test_dictionaries_update_field/test.py +++ b/tests/integration/test_dictionaries_update_field/test.py @@ -34,7 +34,7 @@ def started_cluster(): @pytest.mark.parametrize("dictionary_name,dictionary_type", [ ("flat_update_field_dictionary", "FLAT"), ("simple_key_hashed_update_field_dictionary", "HASHED"), - ("complex_key_hashed_update_field_dictionary", "HASHED") + ("complex_key_hashed_update_field_dictionary", "COMPLEX_KEY_HASHED") ]) def test_update_field(started_cluster, dictionary_name, dictionary_type): create_dictionary_query = """ From 8e37281a7f51b123bc83b814961332105b913378 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Dec 2021 14:10:47 +0300 Subject: [PATCH 482/609] RangeHashedDictionary fix bytes_allocated with update_field --- src/Dictionaries/RangeHashedDictionary.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index bc871a8cdcf..7dc955eb8f7 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -355,6 +355,9 @@ void RangeHashedDictionary::calculateBytesAllocated() if constexpr (dictionary_key_type == DictionaryKeyType::Complex) bytes_allocated += complex_key_arena.size(); + + if (update_field_loaded_block) + bytes_allocated += update_field_loaded_block->allocatedBytes(); } template From 4c916a0e3888cf4a958852dc091aebb95d9ca8b7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 3 Dec 2021 14:31:49 +0300 Subject: [PATCH 483/609] DictionariesLoader qualify name with database fix --- src/Interpreters/ExternalDictionariesLoader.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index e682a98114d..74bff33c914 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -95,14 +95,16 @@ QualifiedTableName ExternalDictionariesLoader::qualifyDictionaryNameWithDatabase return qualified_dictionary_name; } - if (qualified_name->database.empty() && has(dictionary_name)) + /// If dictionary was not qualified with database name, try to resolve dictionary as xml dictionary. + if (qualified_name->database.empty() && !has(qualified_name->table)) { - /// This is xml dictionary - return *qualified_name; - } + auto current_database_name = query_context->getCurrentDatabase(); + std::string resolved_name = resolveDictionaryNameFromDatabaseCatalog(dictionary_name, current_database_name); - if (qualified_name->database.empty()) - qualified_name->database = query_context->getCurrentDatabase(); + /// If after qualify dictionary_name with default_database_name we find it, add default_database to qualified name. + if (has(resolved_name)) + qualified_name->database = query_context->getCurrentDatabase(); + } return *qualified_name; } From 898db5b46846622b715c49c26e902ea3c1b17657 Mon Sep 17 00:00:00 2001 From: frank chen Date: Fri, 3 Dec 2021 19:42:46 +0800 Subject: [PATCH 484/609] Resolve review comments Signed-off-by: frank chen --- src/Formats/FormatFactory.cpp | 20 ++++++++++++++++++++ src/Formats/FormatFactory.h | 5 +++++ src/Storages/StorageURL.cpp | 25 +------------------------ 3 files changed, 26 insertions(+), 24 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 8e490fac301..898012eeaf9 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -302,6 +302,26 @@ OutputFormatPtr FormatFactory::getOutputFormat( return format; } +String FormatFactory::getContentType( + const String & name, + ContextPtr context, + const std::optional & _format_settings) const +{ + const auto & output_getter = getCreators(name).output_creator; + if (!output_getter) + throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT, "Format {} is not suitable for output (with processors)", name); + + auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); + + Block emptyBlock; + RowOutputFormatParams emptyParams; + WriteBufferFromOwnString emptyBuffer; + auto format = output_getter(emptyBuffer, emptyBlock, emptyParams, format_settings); + + return format->getContentType(); +} + + void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator) { auto & target = dict[name].input_creator; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 77ecd2c167f..ea285c47996 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -131,6 +131,11 @@ public: const Block & sample, ContextPtr context, WriteCallback callback = {}, + const std::optional & _format_settings = std::nullopt) const; + + String getContentType( + const String & name, + ContextPtr context, const std::optional & format_settings = std::nullopt) const; void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 2d975aa42f3..fe05d168c31 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -230,30 +230,7 @@ StorageURLSink::StorageURLSink( const String & http_method) : SinkToStorage(sample_block) { - // - // get the content type first - // - // The code here may look a little wired. - // The getContentType() is prodived on IOutputFormat class which relies on a WriteBuffer object, - // and this WriteBuffer object here is WriterBufferFromHTTP itself which accepts the Content-Type header. - // So, this is cyclic dependency. - // To decouple such dependency, we must be able to set header to 'WriteBufferFromHTTP' after we get the instance of output format by calling IOutputFormat::getContentType. - // But this is tricky because the 'WriteBufferFromHTTP' object may have been decorated by 'WriteBufferWithCompression' and is not acceesible due to private modifiers. - // - // So, here we first instantiate an OutputFormat object with a fake stream to get the Content-Type. - // This is not the best way but a more simpler way to understand. - // - std::string content_type; - { - WriteBufferFromOStream buffer(std::cout); - auto output = FormatFactory::instance().getOutputFormat(format, - buffer, - sample_block, - context, - {} /* write callback */, - format_settings); - content_type = output->getContentType(); - } + std::string content_type = FormatFactory::instance().getContentType(format, context, format_settings); write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(Poco::URI(uri), http_method, content_type, timeouts), From 4f136cb30c89d7e378fb4eb4ad942261c4a8a16a Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 3 Dec 2021 15:37:39 +0300 Subject: [PATCH 485/609] Fix NaN deserialization for Quoted escaping rule --- .../Serializations/SerializationNullable.cpp | 63 +++++++++++++++++-- .../Impl/CustomSeparatedRowInputFormat.cpp | 2 +- .../02130_parse_quoted_null.reference | 12 ++++ .../0_stateless/02130_parse_quoted_null.sh | 56 +++++++++++++++++ 4 files changed, 127 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/02130_parse_quoted_null.reference create mode 100755 tests/queries/0_stateless/02130_parse_quoted_null.sh diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index 5e2b31ebb9d..261d0ff3c5d 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -394,12 +394,65 @@ template ReturnType SerializationNullable::deserializeTextQuotedImpl(IColumn & column, ReadBuffer & istr, const FormatSettings & settings, const SerializationPtr & nested) { - return safeDeserialize(column, *nested, - [&istr] + if (istr.eof() || (*istr.position() != 'N' && *istr.position() != 'n')) + { + /// This is not null, surely. + return safeDeserialize(column, *nested, + [] { return false; }, + [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextQuoted(nested_column, istr, settings); }); + } + + /// Check if we have enough data in buffer to check if it's a null. + if (istr.available() >= 4) + { + auto check_for_null = [&istr]() { - return checkStringByFirstCharacterAndAssertTheRestCaseInsensitive("NULL", istr); - }, - [&nested, &istr, &settings] (IColumn & nested_column) { nested->deserializeTextQuoted(nested_column, istr, settings); }); + auto * pos = istr.position(); + if (checkStringCaseInsensitive("NULL", istr)) + return true; + istr.position() = pos; + return false; + }; + auto deserialize_nested = [&nested, &settings, &istr] (IColumn & nested_column) + { + nested->deserializeTextQuoted(nested_column, istr, settings); + }; + return safeDeserialize(column, *nested, check_for_null, deserialize_nested); + } + + /// We don't have enough data in buffer to check if it's a NULL + /// and we cannot check it just by one symbol (otherwise we won't be able + /// to differentiate for example NULL and NaN for float) + /// Use PeekableReadBuffer to make a checkpoint before checking + /// null and rollback if the check was failed. + PeekableReadBuffer buf(istr, true); + auto check_for_null = [&buf]() + { + buf.setCheckpoint(); + SCOPE_EXIT(buf.dropCheckpoint()); + if (checkStringCaseInsensitive("NULL", buf)) + return true; + + buf.rollbackToCheckpoint(); + return false; + }; + + auto deserialize_nested = [&nested, &settings, &buf] (IColumn & nested_column) + { + nested->deserializeTextQuoted(nested_column, buf, settings); + /// Check that we don't have any unread data in PeekableReadBuffer own memory. + if (likely(!buf.hasUnreadData())) + return; + + /// We have some unread data in PeekableReadBuffer own memory. + /// It can happen only if there is an unquoted string instead of a number. + throw DB::ParsingException( + ErrorCodes::CANNOT_READ_ALL_DATA, + "Error while parsing Nullable: got an unquoted string {} instead of a number", + String(buf.position(), std::min(10ul, buf.available()))); + }; + + return safeDeserialize(column, *nested, check_for_null, deserialize_nested); } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 6ff9a8cca2c..8cd9d154ae4 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -205,7 +205,7 @@ void CustomSeparatedRowInputFormat::syncAfterError() bool CustomSeparatedRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out) { - return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_before_delimiter, "delimiter before first firld", ignore_spaces); + return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_before_delimiter, "delimiter before first field", ignore_spaces); } bool CustomSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) diff --git a/tests/queries/0_stateless/02130_parse_quoted_null.reference b/tests/queries/0_stateless/02130_parse_quoted_null.reference new file mode 100644 index 00000000000..1f7989bd2ba --- /dev/null +++ b/tests/queries/0_stateless/02130_parse_quoted_null.reference @@ -0,0 +1,12 @@ +\N 1 +nan 2 +42.42 3 +\N 4 +\N 5 +\N 6 +\N 7 +nan 8 +nan 9 +nan 10 +nan 11 +OK diff --git a/tests/queries/0_stateless/02130_parse_quoted_null.sh b/tests/queries/0_stateless/02130_parse_quoted_null.sh new file mode 100755 index 00000000000..9cb6cb73e6c --- /dev/null +++ b/tests/queries/0_stateless/02130_parse_quoted_null.sh @@ -0,0 +1,56 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + + +USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}') +DATA_FILE=$USER_FILES_PATH/test_02130.data +SELECT_QUERY="select * from file('test_02130.data', 'CustomSeparated', 'x Nullable(Float64), y Nullable(UInt64)') settings input_format_parallel_parsing=0, format_custom_escaping_rule='Quoted'" + + +$CLICKHOUSE_CLIENT -q "drop table if exists test_02130" +$CLICKHOUSE_CLIENT -q "create table test_02130 (x Nullable(Float64), y Nullable(UInt64)) engine=Memory()" + +echo -e "null\t1" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" + +echo -e "nan\t2" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" + +echo -e "42.42\t3" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" + +echo -e "null\t4" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=1 + +echo -e "null\t5" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=2 + +echo -e "null\t6" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=3 + +echo -e "null\t7" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=4 + +echo -e "nan\t8" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=1 + +echo -e "nan\t9" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=2 + +echo -e "nan\t10" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=3 + +echo -e "nan\t11" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=4 + +echo -e "42\tnan" > $DATA_FILE +$CLICKHOUSE_CLIENT -q "$SELECT_QUERY" --max_read_buffer_size=4 2>&1 | grep -F -q "CANNOT_READ_ALL_DATA" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -q "select * from test_02130 order by y" +$CLICKHOUSE_CLIENT -q "drop table test_02130" + +rm $DATA_FILE From adaa32b41b0de7d916b7fdb2aae80412190e9916 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Dec 2021 15:52:57 +0300 Subject: [PATCH 486/609] Fix inner table name --- src/Storages/WindowView/StorageWindowView.cpp | 14 ++- ...7_window_view_parser_inner_table.reference | 20 ++-- .../01047_window_view_parser_inner_table.sql | 92 ++++++++------- .../01048_window_view_parser.reference | 28 ++--- .../0_stateless/01048_window_view_parser.sql | 111 ++++++++---------- .../01052_window_view_proc_tumble_to_now.sql | 5 +- .../01053_window_view_proc_hop_to_now.sql | 5 +- 7 files changed, 135 insertions(+), 140 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 47e4037b2ad..79ba2568de4 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -846,7 +846,6 @@ void StorageWindowView::threadFuncFireProc() std::unique_lock lock(fire_signal_mutex); UInt32 timestamp_now = std::time(nullptr); - LOG_TRACE(log, "Processing time. Now: {}. Next fire time: {}", timestamp_now, next_fire_signal); while (next_fire_signal <= timestamp_now) { try @@ -879,7 +878,6 @@ void StorageWindowView::threadFuncFireEvent() while (!fire_signal.empty()) { - LOG_TRACE(log, "Fire signals: {}", fire_signal.size()); fire(fire_signal.front()); fire_signal.pop_front(); } @@ -987,15 +985,21 @@ StorageWindowView::StorageWindowView( else window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); - auto generate_inner_table_name = [](const String & table_name) { return ".inner." + table_name; }; + auto generate_inner_table_name = [](const StorageID & storage_id) + { + if (storage_id.hasUUID()) + return ".inner." + toString(storage_id.uuid); + return ".inner." + storage_id.table_name; + }; + if (attach_) { - inner_table_id = StorageID(table_id_.database_name, generate_inner_table_name(table_id_.table_name)); + inner_table_id = StorageID(table_id_.database_name, generate_inner_table_name(table_id_)); } else { auto inner_create_query - = getInnerTableCreateQuery(inner_query, query.storage, table_id_.database_name, generate_inner_table_name(table_id_.table_name)); + = getInnerTableCreateQuery(inner_query, query.storage, table_id_.database_name, generate_inner_table_name(table_id_)); auto create_context = Context::createCopy(context_); InterpreterCreateQuery create_interpreter(inner_create_query, create_context); diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference index 184a6c5ba51..19ebe5e0dbc 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference @@ -1,22 +1,22 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index 6c607ba689e..f332ec57b7f 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -1,71 +1,75 @@ +-- Tags: no-parallel + SET allow_experimental_window_view = 1; +DROP DATABASE IF EXISTS test_01047; +CREATE DATABASE test_01047 ENGINE=Ordinary; -DROP TABLE IF EXISTS mt; +DROP TABLE IF EXISTS test_01047.mt; -CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test_01047.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), TUMBLE_END(wid) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), TUMBLE_END(wid) AS count FROM test_01047.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---PARTITION---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, TUMBLE(now(), INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, TUMBLE(now(), INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, HOP_END(wid) FROM test_01047.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---PARTITION---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, HOP_END(wid) FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01047.wv; +DROP TABLE IF EXISTS test_01047.`.inner.wv`; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, HOP_END(wid) FROM test_01047.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +SHOW CREATE TABLE test_01047.`.inner.wv`; -DROP TABLE wv; -DROP TABLE mt; \ No newline at end of file +DROP TABLE test_01047.wv; +DROP TABLE test_01047.mt; diff --git a/tests/queries/0_stateless/01048_window_view_parser.reference b/tests/queries/0_stateless/01048_window_view_parser.reference index 84ede9be463..47ed39fc1d8 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_parser.reference @@ -1,26 +1,26 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE default.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE default.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 -CREATE TABLE default.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql index eebb36aefdf..e7dc4b324f6 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_parser.sql @@ -1,93 +1,82 @@ +-- Tags: no-parallel + SET allow_experimental_window_view = 1; +DROP DATABASE IF EXISTS test_01048; +CREATE DATABASE test_01048 ENGINE=Ordinary; -DROP TABLE IF EXISTS mt; -DROP TABLE IF EXISTS `.inner.wv`; +DROP TABLE IF EXISTS test_01048.mt; -CREATE TABLE mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); +CREATE TABLE test_01048.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM mt GROUP BY TUMBLE(timestamp, INTERVAL 1 SECOND) as wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM test_01048.mt GROUP BY TUMBLE(timestamp, INTERVAL 1 SECOND) as wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM mt GROUP BY wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01048.mt GROUP BY wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, b; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, b; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---TimeZone---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM mt GROUP BY wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM test_01048.mt GROUP BY wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND) as wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND) as wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM mt GROUP BY wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01048.mt GROUP BY wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---IDENTIFIER---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---FUNCTION---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---TimeZone---'; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'Asia/Shanghai') as wid; -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'Asia/Shanghai') as wid; +SHOW CREATE TABLE test_01048.`.inner.wv`; -DROP TABLE IF EXISTS wv; -DROP TABLE IF EXISTS `.inner.wv`; -CREATE WINDOW VIEW wv AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); -SHOW CREATE TABLE `.inner.wv`; +DROP TABLE IF EXISTS test_01048.wv; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); +SHOW CREATE TABLE test_01048.`.inner.wv`; -DROP TABLE wv; -DROP TABLE mt; \ No newline at end of file +DROP TABLE test_01048.wv; +DROP TABLE test_01048.mt; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql index 89fcb243be3..787811ad942 100644 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql @@ -2,8 +2,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv NO DELAY; -DROP TABLE IF EXISTS `.inner.wv`; +DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); @@ -13,6 +12,6 @@ INSERT INTO mt VALUES (1); SELECT sleep(2); SELECT count from dst; -DROP TABLE wv NO DELAY; +DROP TABLE wv; DROP TABLE mt; DROP TABLE dst; diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql index fdc448947e6..4413e52025d 100644 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql @@ -2,8 +2,7 @@ SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; DROP TABLE IF EXISTS dst; -DROP TABLE IF EXISTS wv NO DELAY; -DROP TABLE IF EXISTS `.inner.wv`; +DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); @@ -13,6 +12,6 @@ INSERT INTO mt VALUES (1); SELECT sleep(2); SELECT count from dst; -DROP TABLE wv NO DELAY; +DROP TABLE wv; DROP TABLE mt; DROP TABLE dst; From 0c1e89b0439c753f4ca1ed06d0f2e90b076e4109 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 Dec 2021 14:14:06 +0100 Subject: [PATCH 487/609] Add test --- .../0_stateless/02128_wait_end_of_query_fix.reference | 0 .../0_stateless/02128_wait_end_of_query_fix.sh | 11 +++++++++++ 2 files changed, 11 insertions(+) create mode 100755 tests/queries/0_stateless/02128_wait_end_of_query_fix.reference create mode 100755 tests/queries/0_stateless/02128_wait_end_of_query_fix.sh diff --git a/tests/queries/0_stateless/02128_wait_end_of_query_fix.reference b/tests/queries/0_stateless/02128_wait_end_of_query_fix.reference new file mode 100755 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02128_wait_end_of_query_fix.sh b/tests/queries/0_stateless/02128_wait_end_of_query_fix.sh new file mode 100755 index 00000000000..c98f9980fb1 --- /dev/null +++ b/tests/queries/0_stateless/02128_wait_end_of_query_fix.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +set -eu + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# https://github.com/ClickHouse/ClickHouse/issues/32186 + +${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}/?&query=SELECT+cluster%2C+host_address%2C+port+FROM+system.clusters+FORMAT+JSON&max_result_bytes=104857600&log_queries=1&optimize_throw_if_noop=1&output_format_json_quote_64bit_integers=0&lock_acquire_timeout=10&max_execution_time=10&wait_end_of_query=1" >/dev/null From 7549619b25c8a711cc2e3522c0e6631e0307528f Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 3 Dec 2021 16:25:35 +0300 Subject: [PATCH 488/609] Improve skiping unknown fields with Quoted escaping rule in Template/CustomSeparated formats --- src/Formats/EscapingRuleUtils.cpp | 5 +- src/IO/ReadHelpers.cpp | 92 +++++++++++++++++++ src/IO/ReadHelpers.h | 11 +++ .../Impl/CustomSeparatedRowInputFormat.cpp | 2 +- .../02129_skip_quoted_fields.reference | 26 ++++++ .../0_stateless/02129_skip_quoted_fields.sh | 53 +++++++++++ 6 files changed, 184 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02129_skip_quoted_fields.reference create mode 100755 tests/queries/0_stateless/02129_skip_quoted_fields.sh diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index 2c2662a6a67..d956d9e6bfb 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -69,10 +69,7 @@ void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule esca readEscapedString(tmp, buf); break; case FormatSettings::EscapingRule::Quoted: - /// FIXME: it skips only strings, not numbers, arrays or tuples. - /// we should read until delimiter and skip all data between - /// single quotes. - readQuotedString(tmp, buf); + readQuotedFieldIntoString(tmp, buf); break; case FormatSettings::EscapingRule::CSV: readCSVString(tmp, buf, format_settings.csv); diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 675adc43ce6..b0a6838b81e 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1212,4 +1212,96 @@ void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delim } } + +template +static void readQuotedFieldInBrackets(String & s, ReadBuffer & buf) +{ + assertChar(opening_bracket, buf); + s.push_back(opening_bracket); + + size_t balance = 1; + + while (!buf.eof() && balance) + { + char * next_pos = find_first_symbols<'\'', opening_bracket, closing_bracket>(buf.position(), buf.buffer().end()); + appendToStringOrVector(s, buf, next_pos); + buf.position() = next_pos; + + if (!buf.hasPendingData()) + continue; + + s.push_back(*buf.position()); + + if (*buf.position() == '\'') + { + readQuotedStringInto(s, buf); + s.push_back('\''); + } + else if (*buf.position() == opening_bracket) + { + ++balance; + ++buf.position(); + } + else if (*buf.position() == closing_bracket) + { + --balance; + ++buf.position(); + } + } +} + +void readQuotedFieldIntoString(String & s, ReadBuffer & buf) +{ + s.clear(); + + if (buf.eof()) + return; + + /// Possible values in 'Quoted' field: + /// - Strings: '...' + /// - Arrays: [...] + /// - Tuples: (...) + /// - Maps: {...} + /// - NULL + /// - Number: integer, float, decimal. + + if (*buf.position() == '\'') + readQuotedString(s, buf); + else if (*buf.position() == '[') + readQuotedFieldInBrackets<'[', ']'>(s, buf); + else if (*buf.position() == '(') + readQuotedFieldInBrackets<'(', ')'>(s, buf); + else if (*buf.position() == '{') + readQuotedFieldInBrackets<'{', '}'>(s, buf); + else if (checkCharCaseInsensitive('n', buf)) + { + /// NULL or NaN + if (checkCharCaseInsensitive('u', buf)) + { + assertStringCaseInsensitive("ll", buf); + s.append("NULL"); + } + else + { + assertStringCaseInsensitive("an", buf); + s.append("NaN"); + } + } + else + { + /// It's an integer, float or decimal. They all can be parsed as float. + /// Use PeekableReadBuffer to copy field to string after parsing. + PeekableReadBuffer peekable_buf(buf); + peekable_buf.setCheckpoint(); + Float64 tmp; + readFloatText(tmp, peekable_buf); + peekable_buf.makeContinuousMemoryFromCheckpointToPos(); + auto * end = peekable_buf.position(); + peekable_buf.rollbackToCheckpoint(); + s.append(peekable_buf.position(), end); + peekable_buf.position() = end; + } +} + + } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index da59fc7973c..c48306cf6d3 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -184,6 +184,15 @@ inline void assertChar(char symbol, ReadBuffer & buf) } } +inline bool checkCharCaseInsensitive(char c, ReadBuffer & buf) +{ + char a; + if (!buf.peek(a) || !equalsCaseInsensitive(a, c)) + return false; + buf.ignore(); + return true; +} + inline void assertString(const String & s, ReadBuffer & buf) { assertString(s.c_str(), buf); @@ -1375,4 +1384,6 @@ struct PcgDeserializer } }; +void readQuotedFieldIntoString(String & s, ReadBuffer & buf); + } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 6ff9a8cca2c..8cd9d154ae4 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -205,7 +205,7 @@ void CustomSeparatedRowInputFormat::syncAfterError() bool CustomSeparatedRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out) { - return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_before_delimiter, "delimiter before first firld", ignore_spaces); + return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_before_delimiter, "delimiter before first field", ignore_spaces); } bool CustomSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) diff --git a/tests/queries/0_stateless/02129_skip_quoted_fields.reference b/tests/queries/0_stateless/02129_skip_quoted_fields.reference new file mode 100644 index 00000000000..312f526ca28 --- /dev/null +++ b/tests/queries/0_stateless/02129_skip_quoted_fields.reference @@ -0,0 +1,26 @@ +1 42 +2 42 +3 42 +4 42 +5 42 +6 42 +7 42 +8 42 +9 42 +10 42 +11 42 +12 42 +13 42 +14 42 +15 42 +16 42 +17 42 +18 42 +19 42 +20 42 +21 42 +22 42 +23 42 +24 42 +25 42 +26 42 diff --git a/tests/queries/0_stateless/02129_skip_quoted_fields.sh b/tests/queries/0_stateless/02129_skip_quoted_fields.sh new file mode 100755 index 00000000000..c1baeb5b8f2 --- /dev/null +++ b/tests/queries/0_stateless/02129_skip_quoted_fields.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "drop table if exists test_02129" +$CLICKHOUSE_CLIENT -q "create table test_02129 (x UInt64, y UInt64) engine=Memory()" + +QUERY="insert into test_02129 format CustomSeparatedWithNames settings input_format_skip_unknown_fields=1, format_custom_escaping_rule='Quoted'" + +# Skip string +echo -e "'x'\t'trash'\t'y'\n1\t'Some string'\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" + +# Skip number +echo -e "'x'\t'trash'\t'y'\n2\t42\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n3\t4242.4242\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n4\t-42\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n5\t+42\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n6\t-4242.424242\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n7\t+4242.424242\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n8\tnan\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n9\tinf\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n10\t+nan\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n11\t+inf\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n12\t-nan\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n13\t-inf\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n14\t44444444444444444444444444.444444444444444444444444\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n15\t30e30\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n16\t-30e-30\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" + +# Skip NULL +echo -e "'x'\t'trash'\t'y'\n17\tNULL\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" + +# Skip an array +echo -e "'x'\t'trash'\t'y'\n18\t[1,2,3,4]\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n19\t['some string ]][[][][]', 'one more string (){}][[{[[[[[[']\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n20\t[[(1,2), (3,4)], [(5,6), (7,8)]]\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" + +# Skip a tuple +echo -e "'x'\t'trash'\t'y'\n21\t(1,2,3,4)\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n22\t('some string ()))))(()(())', 'one more string (){}][[{[)))))')\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n23\t(([1,2], (3,4)), ([5,6], (7,8)))\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" + +# Skip a map +echo -e "'x'\t'trash'\t'y'\n24\t{1:2,2:3,3:4,4:5}\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n25\t{'some string }}}}}}{{{{':123, 'one more string (){}][[{[{{{{{':123}\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" +echo -e "'x'\t'trash'\t'y'\n26\t{'key':{1:(1,2), 2:(3,4)}, 'foo':{1:(5,6), 2:(7,8)}}\t42" | $CLICKHOUSE_CLIENT -q "$QUERY" + +$CLICKHOUSE_CLIENT -q "select * from test_02129 order by x" +$CLICKHOUSE_CLIENT -q "drop table test_02129" + From d1f0b249d328d5d8aabc1eb5a7a67bbcc31bf270 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 3 Dec 2021 14:50:33 +0100 Subject: [PATCH 489/609] Fix style --- tests/queries/0_stateless/02128_wait_end_of_query_fix.reference | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 tests/queries/0_stateless/02128_wait_end_of_query_fix.reference diff --git a/tests/queries/0_stateless/02128_wait_end_of_query_fix.reference b/tests/queries/0_stateless/02128_wait_end_of_query_fix.reference old mode 100755 new mode 100644 From c319b6fa32352e07a411bc56c9096d37726de805 Mon Sep 17 00:00:00 2001 From: frank chen Date: Fri, 3 Dec 2021 22:09:04 +0800 Subject: [PATCH 490/609] Fix style Signed-off-by: frank chen --- src/Formats/FormatFactory.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 898012eeaf9..4539a0d6e6a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -313,10 +313,10 @@ String FormatFactory::getContentType( auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - Block emptyBlock; - RowOutputFormatParams emptyParams; - WriteBufferFromOwnString emptyBuffer; - auto format = output_getter(emptyBuffer, emptyBlock, emptyParams, format_settings); + Block empty_block; + RowOutputFormatParams empty_params; + WriteBufferFromOwnString empty_buffer; + auto format = output_getter(empty_buffer, empty_block, empty_params, format_settings); return format->getContentType(); } From 90eba0c0f6f396066ac0eb98b9da2cd5423f99f5 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 3 Dec 2021 17:45:53 +0300 Subject: [PATCH 491/609] fix uncaught exception in DatabaseLazy --- src/Databases/DatabaseLazy.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index 4b3e06e318e..1ff84b53eee 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -269,6 +269,7 @@ StoragePtr DatabaseLazy::loadTable(const String & table_name) const } void DatabaseLazy::clearExpiredTables() const +try { std::lock_guard lock(mutex); auto time_now = std::chrono::system_clock::to_time_t(std::chrono::system_clock::now()); @@ -303,6 +304,10 @@ void DatabaseLazy::clearExpiredTables() const cache_expiration_queue.splice(cache_expiration_queue.begin(), busy_tables, busy_tables.begin(), busy_tables.end()); } +catch (...) +{ + tryLogCurrentException(log, __PRETTY_FUNCTION__); +} DatabaseLazyIterator::DatabaseLazyIterator(const DatabaseLazy & database_, Strings && table_names_) From 738265eed23c6b947565bd560836c7a89c107fe7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 Dec 2021 18:17:36 +0300 Subject: [PATCH 492/609] Update ASTCreateQuery.cpp --- src/Parsers/ASTCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Parsers/ASTCreateQuery.cpp b/src/Parsers/ASTCreateQuery.cpp index 952e5c5a738..facc7e728c9 100644 --- a/src/Parsers/ASTCreateQuery.cpp +++ b/src/Parsers/ASTCreateQuery.cpp @@ -268,7 +268,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat what = "MATERIALIZED VIEW"; else if (is_live_view) what = "LIVE VIEW"; - if (is_window_view) + else if (is_window_view) what = "WINDOW VIEW"; settings.ostr From 7b1d4631bfe26a6561d29423e589baa5807d42f0 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 3 Dec 2021 18:56:29 +0300 Subject: [PATCH 493/609] Update FileLogSource.cpp --- src/Storages/FileLog/FileLogSource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/FileLog/FileLogSource.cpp b/src/Storages/FileLog/FileLogSource.cpp index a8da34f3204..7d4b5ac6fec 100644 --- a/src/Storages/FileLog/FileLogSource.cpp +++ b/src/Storages/FileLog/FileLogSource.cpp @@ -54,12 +54,12 @@ FileLogSource::~FileLogSource() { tryLogCurrentException(__PRETTY_FUNCTION__); } - storage.reduceStreams(); } void FileLogSource::onFinish() { storage.closeFilesAndStoreMeta(start, end); + storage.reduceStreams(); finished = true; } From 1cc5dd797901dca7350222bbdd332c36facda4e9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 3 Dec 2021 23:36:35 +0300 Subject: [PATCH 494/609] Fix --- tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql | 2 +- tests/queries/0_stateless/01055_window_view_proc_hop_to.sql | 2 +- tests/queries/0_stateless/01056_window_view_proc_hop_watch.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql index 8ecd93fbf87..f229969603b 100644 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql @@ -9,7 +9,7 @@ CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid; -INSERT INTO mt VALUES (1, now() + 1); +INSERT INTO mt VALUES (1, now('US/Samoa') + 1); SELECT sleep(3); SELECT count from dst; diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql index 8e28577f645..b75cc33e741 100644 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql @@ -9,7 +9,7 @@ CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; -INSERT INTO mt VALUES (1, now() + 1); +INSERT INTO mt VALUES (1, now('US/Samoa') + 1); SELECT sleep(3); SELECT count from dst; diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index 353bd38bc54..df83615d507 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -39,7 +39,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('WATCH 01056_window_view_proc_hop_watch.wv') client1.expect('Query id' + end_of_block) - client2.send("INSERT INTO 01056_window_view_proc_hop_watch.mt VALUES (1, now() + 1)") + client2.send("INSERT INTO 01056_window_view_proc_hop_watch.mt VALUES (1, now('US/Samoa') + 1)") client1.expect('1' + end_of_block) client1.expect('Progress: 1.00 rows.*\)') From 8e202d980a040d21d293acd6fb925476ccc8af64 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 3 Dec 2021 18:03:07 -0400 Subject: [PATCH 495/609] fix typo --- docs/en/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index b1c53b61b12..eb4673be18a 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -105,7 +105,7 @@ ClickHouse Keeper is bundled into the ClickHouse server package, just add config clickhouse-keeper --config /etc/your_path_to_config/config.xml --daemon ``` -## Four Latter Word Commands +## Four Letter Word Commands ClickHouse Keeper also provides 4lw commands which are almost the same with Zookeeper. Each command is composed of four letters such as `mntr`, `stat` etc. There are some more interesting commands: `stat` gives some general information about the server and connected clients, while `srvr` and `cons` give extended details on server and connections respectively. From 4709ff934031c99d32194a49a30c9a907068f3d8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 1 Dec 2021 22:01:05 +0300 Subject: [PATCH 496/609] More generic check for CMAKE_BUILD_TYPE in jemalloc --- contrib/jemalloc-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index fd52ce4a4f3..fb11879fb21 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ if (SANITIZE OR NOT ( ((OS_LINUX OR OS_FREEBSD) AND (ARCH_AMD64 OR ARCH_ARM OR ARCH_PPC64LE OR ARCH_RISCV64)) OR - (OS_DARWIN AND (CMAKE_BUILD_TYPE STREQUAL "RelWithDebInfo" OR CMAKE_BUILD_TYPE STREQUAL "Debug")) + (OS_DARWIN AND (CMAKE_BUILD_TYPE_UC STREQUAL "RELWITHDEBINFO" OR CMAKE_BUILD_TYPE_UC STREQUAL "DEBUG")) )) if (ENABLE_JEMALLOC) message (${RECONFIGURE_MESSAGE_LEVEL} From 130be9a4f92c7d15af7ca67f0fbdd6f20b3d6225 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 21:39:09 +0300 Subject: [PATCH 497/609] Bump libpqxx to fix assertion in jemalloc Refs: https://github.com/ClickHouse-Extras/libpqxx/pull/5 --- contrib/libpqxx | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/libpqxx b/contrib/libpqxx index 357608d11b7..63e20f9485b 160000 --- a/contrib/libpqxx +++ b/contrib/libpqxx @@ -1 +1 @@ -Subproject commit 357608d11b7a1961c3fb7db2ef9a5dbb2e87da77 +Subproject commit 63e20f9485b8cbeabf99008123248fc9f033e766 From 22a74dc68df9891756375a290b390208fcd5e9c5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 21:46:56 +0300 Subject: [PATCH 498/609] Make jemalloc under osx even more reliable - explicitly call zone_register() again - explicitly call malloc(free()) to initialize jemalloc() --- src/Common/new_delete.cpp | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/src/Common/new_delete.cpp b/src/Common/new_delete.cpp index fa32d56b350..27db87809d3 100644 --- a/src/Common/new_delete.cpp +++ b/src/Common/new_delete.cpp @@ -1,6 +1,44 @@ #include #include +#if defined(OS_DARWIN) && (USE_JEMALLOC) +/// In case of OSX jemalloc register itself as a default zone allocator. +/// +/// Sure jemalloc will register itself, since zone_register() declared with +/// constructor attribute (since zone_register is also forbidden from +/// optimizing out), however those constructors will be called before +/// constructors for global variable initializers (__cxx_global_var_init()). +/// +/// So to make jemalloc under OSX more stable, we will call it explicitly from +/// global variable initializers so that each allocation will use it. +/// (NOTE: It is ok to call it twice, since zone_register() is a no-op if the +/// default zone is already replaced with something.) +/// +/// Refs: https://github.com/jemalloc/jemalloc/issues/708 + +extern "C" +{ + extern void zone_register(); +} + +static struct InitializeJemallocZoneAllocatorForOSX +{ + InitializeJemallocZoneAllocatorForOSX() + { + zone_register(); + /// jemalloc() initializes itself only on malloc() + /// and so if some global initializer will have free(nullptr) + /// jemalloc may trigger some internal assertion. + /// + /// To prevent this, we explicitly call malloc(free()) here. + if (void * ptr = malloc(0)) + { + free(ptr); + } + } +} initializeJemallocZoneAllocatorForOSX; +#endif + /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new From abab7108e46f81192e326bd3927abf1b23b65bb0 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 4 Dec 2021 00:15:52 +0300 Subject: [PATCH 499/609] Fix QueryProfiler building under osx Fixes: #32165 --- src/Common/QueryProfiler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/QueryProfiler.cpp b/src/Common/QueryProfiler.cpp index 12410de6bf0..0b2cd602b38 100644 --- a/src/Common/QueryProfiler.cpp +++ b/src/Common/QueryProfiler.cpp @@ -25,13 +25,13 @@ namespace { #if defined(OS_LINUX) thread_local size_t write_trace_iteration = 0; +#endif /// Even after timer_delete() the signal can be delivered, /// since it does not do anything with pending signals. /// /// And so to overcome this flag is exists, /// to ignore delivered signals after timer_delete(). thread_local bool signal_handler_disarmed = true; -#endif void writeTraceInfo(TraceType trace_type, int /* sig */, siginfo_t * info, void * context) { From c522c06755bfa280b14bab4ab9c3f0a0f3d227f5 Mon Sep 17 00:00:00 2001 From: vxider Date: Sat, 4 Dec 2021 12:30:04 +0000 Subject: [PATCH 500/609] fix windowview parser --- src/Functions/FunctionsWindow.cpp | 28 +++++++++++++------ src/Storages/WindowView/StorageWindowView.cpp | 19 +++++++------ src/Storages/WindowView/StorageWindowView.h | 1 + .../01050_window_view_parser_tumble.reference | 1 + .../01050_window_view_parser_tumble.sql | 6 +++- .../01051_window_view_parser_hop.reference | 1 + .../01051_window_view_parser_hop.sql | 4 +++ 7 files changed, 43 insertions(+), 17 deletions(-) diff --git a/src/Functions/FunctionsWindow.cpp b/src/Functions/FunctionsWindow.cpp index 2ed5e9863d6..a26faac304d 100644 --- a/src/Functions/FunctionsWindow.cpp +++ b/src/Functions/FunctionsWindow.cpp @@ -238,12 +238,18 @@ struct WindowImpl [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String & function_name) { - const auto which_type = WhichDataType(arguments[0].type); + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column; - if (which_type.isDateTime()) - result_column= WindowImpl::dispatchForColumns(arguments, function_name); + if (arguments.size() == 1) + { + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column = time_column.column; + } else - result_column = arguments[0].column; + result_column = WindowImpl::dispatchForColumns(arguments, function_name); return executeWindowBound(result_column, 0, function_name); } }; @@ -260,12 +266,18 @@ struct WindowImpl [[maybe_unused]] static ColumnPtr dispatchForColumns(const ColumnsWithTypeAndName & arguments, const String& function_name) { - const auto which_type = WhichDataType(arguments[0].type); + const auto & time_column = arguments[0]; + const auto which_type = WhichDataType(time_column.type); ColumnPtr result_column; - if (which_type.isDateTime()) - result_column = WindowImpl::dispatchForColumns(arguments, function_name); + if (arguments.size() == 1) + { + if (which_type.isUInt32()) + return time_column.column; + else //isTuple + result_column = time_column.column; + } else - result_column = arguments[0].column; + result_column = WindowImpl::dispatchForColumns(arguments, function_name); return executeWindowBound(result_column, 1, function_name); } }; diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 79ba2568de4..f51f5ddab08 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -89,22 +89,23 @@ namespace { data.is_tumble = t->name == "TUMBLE"; data.is_hop = t->name == "HOP"; + auto temp_node = t->clone(); + temp_node->setAlias(""); if (!data.window_function) { + data.serialized_window_function = serializeAST(*temp_node); t->name = "WINDOW_ID"; data.window_id_name = t->getColumnName(); data.window_id_alias = t->alias; data.window_function = t->clone(); data.window_function->setAlias(""); - data.serialized_window_function = serializeAST(*data.window_function); data.timestamp_column_name = t->arguments->children[0]->getColumnName(); } else { - auto temp_node = t->clone(); - temp_node->setAlias(""); if (serializeAST(*temp_node) != data.serialized_window_function) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); + t->name = "WINDOW_ID"; } } } @@ -146,7 +147,7 @@ namespace void visit(ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "WINDOW_ID") + if (node.name == "WINDOW_ID" || node.name == "TUMBLE" || node.name == "HOP") { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") @@ -938,10 +939,12 @@ StorageWindowView::StorageWindowView( ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW, "UNION is not supported for {}", getName()); - ASTSelectQuery & select_query = typeid_cast(*query.select->list_of_selects->children.at(0)); + ASTSelectQuery & select_query_ = typeid_cast(*query.select->list_of_selects->children.at(0)); String select_database_name = getContext()->getCurrentDatabase(); String select_table_name; - extractDependentTable(getContext(), select_query, select_database_name, select_table_name); + extractDependentTable(getContext(), select_query_, select_database_name, select_table_name); + + select_query = select_query_.clone(); /// If the table is not specified - use the table `system.one` if (select_table_name.empty()) @@ -953,7 +956,7 @@ StorageWindowView::StorageWindowView( DatabaseCatalog::instance().addDependency(select_table_id, table_id_); /// Extract all info from query; substitute Function_TUMPLE and Function_HOP with Function_WINDOW_ID. - auto inner_query = innerQueryParser(select_query); + auto inner_query = innerQueryParser(select_query_); // Parse mergeable query mergeable_query = inner_query->clone(); @@ -1344,7 +1347,7 @@ Block & StorageWindowView::getHeader() const if (!sample_block) { sample_block = InterpreterSelectQuery( - getFinalQuery(), window_view_context, getParentStorage(), nullptr, + select_query->clone(), window_view_context, getParentStorage(), nullptr, SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); for (size_t i = 0; i < sample_block.columns(); ++i) diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 893647add79..e989663c7e5 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -150,6 +150,7 @@ public: private: Poco::Logger * log; + ASTPtr select_query; ASTPtr mergeable_query; ASTPtr final_query; diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.reference b/tests/queries/0_stateless/01050_window_view_parser_tumble.reference index 75cd8e28af5..6375c151906 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.reference +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.reference @@ -4,3 +4,4 @@ ---WITH--- ---WHERE--- ---ORDER_BY--- +---With now--- diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql index 12f67a68237..6837036263c 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql @@ -10,7 +10,7 @@ CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), TUMBLE_S SELECT '---With w_end---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(TUMBLE(timestamp, INTERVAL '3' SECOND)) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WithOut w_end---'; DROP TABLE IF EXISTS wv NO DELAY; @@ -27,3 +27,7 @@ CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt W SELECT '---ORDER_BY---'; DROP TABLE IF EXISTS wv NO DELAY; CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; + +SELECT '---With now---'; +DROP TABLE IF EXISTS wv NO DELAY; +CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(TUMBLE(now(), INTERVAL '3' SECOND)) AS w_end FROM mt GROUP BY TUMBLE(now(), INTERVAL '3' SECOND) AS wid; diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.reference b/tests/queries/0_stateless/01051_window_view_parser_hop.reference index 75cd8e28af5..6375c151906 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.reference +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.reference @@ -4,3 +4,4 @@ ---WITH--- ---WHERE--- ---ORDER_BY--- +---With now--- diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql index 3c1e3d16320..df0729108d0 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.sql @@ -27,3 +27,7 @@ CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHER SELECT '---ORDER_BY---'; DROP TABLE IF EXISTS wv NO DELAY; CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; + +SELECT '---With now---'; +DROP TABLE IF EXISTS wv NO DELAY; +CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND)) as w_end FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; From 55d50c0b4d939fb4007b4006a0faba312bc40d45 Mon Sep 17 00:00:00 2001 From: frank chen Date: Sat, 4 Dec 2021 21:56:52 +0800 Subject: [PATCH 501/609] Improve span operation name Signed-off-by: frank chen --- src/Interpreters/executeQuery.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2ab4167176f..37a0d87a120 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -626,7 +626,13 @@ static std::tuple executeQueryImpl( } { - OpenTelemetrySpanHolder span("IInterpreter::execute()"); + std::unique_ptr span; + if (context->query_trace_context.trace_id != UUID()) + { + auto raw_interpreter_ptr = interpreter.get(); + std::string class_name(abi::__cxa_demangle(typeid(*raw_interpreter_ptr).name(), nullptr, nullptr, nullptr)); + span = std::make_unique(class_name + "::execute()"); + } res = interpreter->execute(); } From 78ceb5c8839de811c57be383a8a6612692bf95d9 Mon Sep 17 00:00:00 2001 From: vxider Date: Sat, 4 Dec 2021 14:18:10 +0000 Subject: [PATCH 502/609] add illegal arg check --- src/Storages/WindowView/StorageWindowView.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index f51f5ddab08..94f88842cbb 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -91,6 +91,10 @@ namespace data.is_hop = t->name == "HOP"; auto temp_node = t->clone(); temp_node->setAlias(""); + if (startsWith(t->arguments->children[0]->getColumnName(), "toDateTime")) + throw Exception( + "The first argument of window function should not be a constant value.", + ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); if (!data.window_function) { data.serialized_window_function = serializeAST(*temp_node); From 15e3dbe3f210e5478825de8c997d3513c3ad890f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 30 Nov 2021 14:43:30 +0300 Subject: [PATCH 503/609] Fix skipping columns in Nested while writing protobuf. --- src/Formats/ProtobufSerializer.cpp | 132 +++++++++++------- ..._format_skipped_column_in_nested.reference | 27 ++++ ...rotobuf_format_skipped_column_in_nested.sh | 55 ++++++++ ...obuf_format_skipped_column_in_nested.proto | 29 ++++ 4 files changed, 194 insertions(+), 49 deletions(-) create mode 100644 tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.reference create mode 100755 tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh create mode 100644 tests/queries/0_stateless/format_schemas/00825_protobuf_format_skipped_column_in_nested.proto diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index ac89203c6e0..94a385aa067 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -2062,7 +2062,7 @@ namespace }; ProtobufSerializerMessage( - std::vector field_descs_, + std::vector && field_descs_, const FieldDescriptor * parent_field_descriptor_, bool with_length_delimiter_, const ProtobufReaderOrWriter & reader_or_writer_) @@ -2091,8 +2091,10 @@ namespace for (const FieldInfo & info : field_infos) { field_columns.clear(); + field_columns.reserve(info.column_indices.size()); for (size_t column_index : info.column_indices) { + assert(column_index < num_columns_); field_columns.emplace_back(columns_[column_index]); } info.field_serializer->setColumns(field_columns.data(), field_columns.size()); @@ -2103,11 +2105,9 @@ namespace missing_column_indices.resize(num_columns_); for (size_t column_index : collections::range(num_columns_)) missing_column_indices[column_index] = column_index; - for (const FieldInfo & info : field_infos) - { - for (size_t column_index : info.column_indices) + for (const auto & field_info : field_infos) + for (size_t column_index : field_info.column_indices) missing_column_indices[column_index] = static_cast(-1); - } boost::range::remove_erase(missing_column_indices, static_cast(-1)); } } @@ -2195,6 +2195,7 @@ namespace reader->endNestedMessage(); else reader->endMessage(false); + addDefaultsToMissingColumns(row_num); } @@ -2229,9 +2230,9 @@ namespace void addDefaultsToMissingColumns(size_t row_num) { - for (size_t column_idx : missing_column_indices) + for (size_t column_index : missing_column_indices) { - auto & column = columns[column_idx]; + auto & column = columns[column_index]; size_t old_size = column->size(); if (row_num >= old_size) column->assumeMutableRef().insertDefault(); @@ -2241,7 +2242,7 @@ namespace struct FieldInfo { FieldInfo( - std::vector column_indices_, + std::vector && column_indices_, const FieldDescriptor & field_descriptor_, std::unique_ptr field_serializer_) : column_indices(std::move(column_indices_)) @@ -2277,8 +2278,8 @@ namespace class ProtobufSerializerTupleAsNestedMessage : public ProtobufSerializer { public: - explicit ProtobufSerializerTupleAsNestedMessage(std::unique_ptr nested_message_serializer_) - : nested_message_serializer(std::move(nested_message_serializer_)) + explicit ProtobufSerializerTupleAsNestedMessage(std::unique_ptr message_serializer_) + : message_serializer(std::move(message_serializer_)) { } @@ -2292,7 +2293,7 @@ namespace element_columns.reserve(tuple_size); for (size_t i : collections::range(tuple_size)) element_columns.emplace_back(column_tuple.getColumnPtr(i)); - nested_message_serializer->setColumns(element_columns.data(), element_columns.size()); + message_serializer->setColumns(element_columns.data(), element_columns.size()); } void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override @@ -2302,12 +2303,12 @@ namespace setColumns(&column0, 1); } - void writeRow(size_t row_num) override { nested_message_serializer->writeRow(row_num); } - void readRow(size_t row_num) override { nested_message_serializer->readRow(row_num); } - void insertDefaults(size_t row_num) override { nested_message_serializer->insertDefaults(row_num); } + void writeRow(size_t row_num) override { message_serializer->writeRow(row_num); } + void readRow(size_t row_num) override { message_serializer->readRow(row_num); } + void insertDefaults(size_t row_num) override { message_serializer->insertDefaults(row_num); } private: - const std::unique_ptr nested_message_serializer; + const std::unique_ptr message_serializer; }; @@ -2317,8 +2318,8 @@ namespace { public: explicit ProtobufSerializerFlattenedNestedAsArrayOfNestedMessages( - std::unique_ptr nested_message_serializer_) - : nested_message_serializer(std::move(nested_message_serializer_)) + std::unique_ptr message_serializer_) + : message_serializer(std::move(message_serializer_)) { } @@ -2340,7 +2341,7 @@ namespace std::sort(offset_columns.begin(), offset_columns.end()); offset_columns.erase(std::unique(offset_columns.begin(), offset_columns.end()), offset_columns.end()); - nested_message_serializer->setColumns(data_columns.data(), data_columns.size()); + message_serializer->setColumns(data_columns.data(), data_columns.size()); } void setColumns(const MutableColumnPtr * columns, size_t num_columns) override @@ -2364,7 +2365,7 @@ namespace throw Exception("Components of FlattenedNested have different sizes", ErrorCodes::PROTOBUF_BAD_CAST); } for (size_t i : collections::range(start_offset, end_offset)) - nested_message_serializer->writeRow(i); + message_serializer->writeRow(i); } void readRow(size_t row_num) override @@ -2377,7 +2378,7 @@ namespace try { - nested_message_serializer->readRow(old_data_size); + message_serializer->readRow(old_data_size); size_t data_size = data_columns[0]->size(); if (data_size != old_data_size + 1) throw Exception("Unexpected number of elements of ColumnArray has been read", ErrorCodes::LOGICAL_ERROR); @@ -2433,7 +2434,7 @@ namespace } private: - const std::unique_ptr nested_message_serializer; + const std::unique_ptr message_serializer; Columns data_columns; Columns offset_columns; }; @@ -2445,7 +2446,7 @@ namespace public: explicit ProtobufSerializerBuilder(const ProtobufReaderOrWriter & reader_or_writer_) : reader_or_writer(reader_or_writer_) {} - std::unique_ptr buildMessageSerializer( + std::unique_ptr buildMessageSerializer( const Strings & column_names, const DataTypes & data_types, std::vector & missing_column_indices, @@ -2453,16 +2454,17 @@ namespace bool with_length_delimiter) { std::vector used_column_indices; - auto serializer = buildMessageSerializerImpl( + auto message_serializer = buildMessageSerializerImpl( /* num_columns = */ column_names.size(), column_names.data(), data_types.data(), - used_column_indices, message_descriptor, with_length_delimiter, - /* parent_field_descriptor = */ nullptr); + /* parent_field_descriptor = */ nullptr, + used_column_indices, + /* columns_are_reordered_outside = */ false); - if (!serializer) + if (!message_serializer) { throw Exception( "Not found matches between the names of the columns {" + boost::algorithm::join(column_names, ", ") @@ -2473,10 +2475,12 @@ namespace missing_column_indices.clear(); missing_column_indices.reserve(column_names.size() - used_column_indices.size()); - boost::range::set_difference(collections::range(column_names.size()), used_column_indices, + auto used_column_indices_sorted = std::move(used_column_indices); + std::sort(used_column_indices_sorted.begin(), used_column_indices_sorted.end()); + boost::range::set_difference(collections::range(column_names.size()), used_column_indices_sorted, std::back_inserter(missing_column_indices)); - return serializer; + return message_serializer; } private: @@ -2621,24 +2625,38 @@ namespace } /// Builds a serializer for a protobuf message (root or nested). + /// + /// Some of the passed columns might be skipped, the function sets `used_column_indices` to + /// the list of those columns which match any fields in the protobuf message. + /// + /// Normally `columns_are_reordered_outside` should be false - if it's false it means that + /// the used column indices will be passed to ProtobufSerializerMessage, which will write/read + /// only those columns and set the rest of columns by default. + /// Set `columns_are_reordered_outside` to true if you're going to reorder columns + /// according to `used_column_indices` returned and pass to + /// ProtobufSerializerMessage::setColumns() only the columns which are actually used. template std::unique_ptr buildMessageSerializerImpl( size_t num_columns, const StringOrStringViewT * column_names, const DataTypePtr * data_types, - std::vector & used_column_indices, const MessageDescriptor & message_descriptor, bool with_length_delimiter, - const FieldDescriptor * parent_field_descriptor) + const FieldDescriptor * parent_field_descriptor, + std::vector & used_column_indices, + bool columns_are_reordered_outside) { std::vector field_descs; boost::container::flat_map field_descriptors_in_use; used_column_indices.clear(); used_column_indices.reserve(num_columns); + boost::container::flat_set used_column_indices_sorted; + used_column_indices_sorted.reserve(num_columns); + size_t sequential_column_index = 0; auto add_field_serializer = [&](const std::string_view & column_name_, - std::vector column_indices_, + std::vector && column_indices_, const FieldDescriptor & field_descriptor_, std::unique_ptr field_serializer_) { @@ -2652,12 +2670,17 @@ namespace ErrorCodes::MULTIPLE_COLUMNS_SERIALIZED_TO_SAME_PROTOBUF_FIELD); } - for (size_t column_index : column_indices_) + used_column_indices.insert(used_column_indices.end(), column_indices_.begin(), column_indices_.end()); + used_column_indices_sorted.insert(column_indices_.begin(), column_indices_.end()); + + auto column_indices_to_pass_to_message_serializer = std::move(column_indices_); + if (columns_are_reordered_outside) { - /// Keep `used_column_indices` sorted. - used_column_indices.insert(boost::range::upper_bound(used_column_indices, column_index), column_index); + for (auto & index : column_indices_to_pass_to_message_serializer) + index = sequential_column_index++; } - field_descs.push_back({std::move(column_indices_), &field_descriptor_, std::move(field_serializer_)}); + + field_descs.push_back({std::move(column_indices_to_pass_to_message_serializer), &field_descriptor_, std::move(field_serializer_)}); field_descriptors_in_use.emplace(&field_descriptor_, column_name_); }; @@ -2666,7 +2689,7 @@ namespace /// We're going through all the passed columns. for (size_t column_idx : collections::range(num_columns)) { - if (boost::range::binary_search(used_column_indices, column_idx)) + if (used_column_indices_sorted.count(column_idx)) continue; const auto & column_name = column_names[column_idx]; @@ -2702,7 +2725,7 @@ namespace for (size_t j : collections::range(column_idx + 1, num_columns)) { - if (boost::range::binary_search(used_column_indices, j)) + if (used_column_indices_sorted.count(j)) continue; std::string_view other_suffix; if (!columnNameStartsWithFieldName(column_names[j], *field_descriptor, other_suffix)) @@ -2740,10 +2763,15 @@ namespace nested_column_names.size(), nested_column_names.data(), nested_data_types.data(), - used_column_indices_in_nested, *field_descriptor->message_type(), - false, - field_descriptor); + /* with_length_delimiter = */ false, + field_descriptor, + used_column_indices_in_nested, + /* columns_are_reordered_outside = */ true); + + /// `columns_are_reordered_outside` is true because column indices are + /// going to be transformed and then written to the outer message, + /// see add_field_serializer() below. if (nested_message_serializer) { @@ -2774,10 +2802,15 @@ namespace nested_column_names.size(), nested_column_names.data(), nested_data_types.data(), - used_column_indices_in_nested, *field_descriptor->message_type(), - false, - field_descriptor); + /* with_length_delimiter = */ false, + field_descriptor, + used_column_indices_in_nested, + /* columns_are_reordered_outside = */ true); + + /// `columns_are_reordered_outside` is true because column indices are + /// going to be transformed and then written to the outer message, + /// see add_field_serializer() below. if (nested_message_serializer) { @@ -2907,16 +2940,17 @@ namespace { /// Try to serialize as a nested message. std::vector used_column_indices; - auto nested_message_serializer = buildMessageSerializerImpl( + auto message_serializer = buildMessageSerializerImpl( size_of_tuple, tuple_data_type.getElementNames().data(), tuple_data_type.getElements().data(), - used_column_indices, *field_descriptor.message_type(), - false, - &field_descriptor); + /* with_length_delimiter = */ false, + &field_descriptor, + used_column_indices, + /* columns_are_reordered_outside = */ false); - if (!nested_message_serializer) + if (!message_serializer) { throw Exception( "Not found matches between the names of the tuple's elements {" @@ -2926,7 +2960,7 @@ namespace ErrorCodes::NO_COLUMNS_SERIALIZED_TO_PROTOBUF_FIELDS); } - return std::make_unique(std::move(nested_message_serializer)); + return std::make_unique(std::move(message_serializer)); } /// Serialize as a repeated field. diff --git a/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.reference b/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.reference new file mode 100644 index 00000000000..1a80e6401db --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.reference @@ -0,0 +1,27 @@ +e4048ead-30a2-45e5-90be-2af1c7137523 dummy [1] [50639] [58114] [[5393]] [[1]] [[3411]] [[17811]] [[(10,20)]] + +Binary representation: +00000000 44 0a 24 65 34 30 34 38 65 61 64 2d 33 30 61 32 |D.$e4048ead-30a2| +00000010 2d 34 35 65 35 2d 39 30 62 65 2d 32 61 66 31 63 |-45e5-90be-2af1c| +00000020 37 31 33 37 35 32 33 62 1c 10 01 18 cf 8b 03 20 |7137523b....... | +00000030 82 c6 03 5a 10 28 01 30 91 2a 40 93 8b 01 52 05 |...Z.(.0.*@...R.| +00000040 4d 00 00 a0 41 |M...A| +00000045 + +MESSAGE #1 AT 0x00000001 +identifier: "e4048ead-30a2-45e5-90be-2af1c7137523" +modules { + module_id: 1 + supply: 50639 + temp: 58114 + nodes { + node_id: 1 + opening_time: 5393 + current: 17811 + coords { + y: 20 + } + } +} + +Binary representation is as expected diff --git a/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh b/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh new file mode 100755 index 00000000000..b413385fb77 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_skipped_column_in_nested.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +# https://github.com/ClickHouse/ClickHouse/issues/31160 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -eo pipefail + +# Run the client. +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" + +# Check the output in the protobuf format +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage" --input "$BINARY_FILE_PATH" + +# Check the input in the protobuf format (now the table contains the same data twice). +#echo +#$CLICKHOUSE_CLIENT --query "INSERT INTO table_skipped_column_in_nested_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_skipped_column_in_nested:UpdateMessage'" < "$BINARY_FILE_PATH" +#$CLICKHOUSE_CLIENT --query "SELECT * FROM table_skipped_column_in_nested_00825" + +rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE table_skipped_column_in_nested_00825" diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_skipped_column_in_nested.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_skipped_column_in_nested.proto new file mode 100644 index 00000000000..054de349e24 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_skipped_column_in_nested.proto @@ -0,0 +1,29 @@ +syntax = "proto3"; + +message UpdateMessage { + string identifier = 1; + //string unused1 = 100; + + message Module { + uint32 module_id = 2; + uint32 supply = 3; + uint32 temp = 4; + + message ModuleNode { + uint32 node_id = 5; + uint32 opening_time = 6; + uint32 closing_time = 7; // The column in the table is named `closing_time_time` + uint32 current = 8; + + message Coords { + //float x = 8; + float y = 9; + } + Coords coords = 10; + } + + repeated ModuleNode nodes = 11; + } + + repeated Module modules = 12; +} From 2e0b4800440e76a9877ce6f8871b904d01c421aa Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 1 Dec 2021 21:19:47 +0300 Subject: [PATCH 504/609] Improve error handling while serializing protobufs. --- src/Formats/ProtobufSerializer.cpp | 494 ++++++++++++++++++++--------- src/Formats/ProtobufSerializer.h | 4 +- 2 files changed, 351 insertions(+), 147 deletions(-) diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 94a385aa067..efe01740cf6 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -28,6 +28,7 @@ # include # include # include +# include # include # include # include @@ -139,6 +140,15 @@ namespace } + WriteBuffer & writeIndent(WriteBuffer & out, size_t size) { return out << String(size * 4, ' '); } + + + [[noreturn]] void wrongNumberOfColumns(size_t number_of_columns, const String & expected) + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong number of columns: expected {}, specified {}", expected, number_of_columns); + } + + struct ProtobufReaderOrWriter { ProtobufReaderOrWriter(ProtobufReader & reader_) : reader(&reader_) {} // NOLINT(google-explicit-constructor) @@ -152,8 +162,12 @@ namespace class ProtobufSerializerSingleValue : public ProtobufSerializer { protected: - ProtobufSerializerSingleValue(const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : field_descriptor(field_descriptor_) + ProtobufSerializerSingleValue( + const std::string_view & column_name_, + const FieldDescriptor & field_descriptor_, + const ProtobufReaderOrWriter & reader_or_writer_) + : column_name(column_name_) + , field_descriptor(field_descriptor_) , field_typeid(field_descriptor_.type()) , field_tag(field_descriptor.number()) , reader(reader_or_writer_.reader) @@ -164,13 +178,15 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); column = columns[0]; } void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); column = columns[0]->getPtr(); } @@ -259,14 +275,28 @@ namespace return result; } + [[noreturn]] void incompatibleColumnType(const std::string_view & column_type) const + { + throw Exception( + ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD, + "The column {} ({}) cannot be serialized to the field {} ({}) due to their types are not compatible", + quoteString(column_name), + column_type, + quoteString(field_descriptor.full_name()), + field_descriptor.type_name()); + } + [[noreturn]] void cannotConvertValue(const std::string_view & src_value, const std::string_view & src_type_name, const std::string_view & dest_type_name) const { throw Exception( - "Could not convert value '" + String{src_value} + "' from type " + String{src_type_name} + " to type " + String{dest_type_name} + - " while " + (reader ? "reading" : "writing") + " field " + field_descriptor.name(), + "Could not convert value '" + String{src_value} + "' from type " + String{src_type_name} + " to type " + + String{dest_type_name} + " while " + (reader ? "reading" : "writing") + " field " + + quoteString(field_descriptor.name()) + " " + (reader ? "for inserting into" : "extracted from") + " column " + + quoteString(column_name), ErrorCodes::PROTOBUF_BAD_CAST); } + const String column_name; const FieldDescriptor & field_descriptor; const FieldTypeId field_typeid; const int field_tag; @@ -289,8 +319,8 @@ namespace public: using ColumnType = ColumnVector; - ProtobufSerializerNumber(const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + ProtobufSerializerNumber(const std::string_view & column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) + : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) { setFunctions(); } @@ -319,6 +349,13 @@ namespace column_vector.insertValue(getDefaultNumber()); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerNumber<" << TypeName << ">: column " << quoteString(column_name) + << " -> field " << quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() + << ")\n"; + } + private: void setFunctions() { @@ -469,7 +506,7 @@ namespace case FieldTypeId::TYPE_ENUM: { if (std::is_floating_point_v) - failedToSetFunctions(); + incompatibleColumnType(TypeName); write_function = [this](NumberType value) { @@ -484,18 +521,10 @@ namespace } default: - failedToSetFunctions(); + incompatibleColumnType(TypeName); } } - [[noreturn]] void failedToSetFunctions() const - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type " + quoteString(TypeName), - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } - NumberType getDefaultNumber() { if (!default_number) @@ -529,10 +558,11 @@ namespace using ColumnType = std::conditional_t; ProtobufSerializerString( + const std::string_view & column_name_, const std::shared_ptr & fixed_string_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) , fixed_string_data_type(fixed_string_data_type_) , n(fixed_string_data_type->getN()) { @@ -542,8 +572,10 @@ namespace } ProtobufSerializerString( - const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + const std::string_view & column_name_, + const google::protobuf::FieldDescriptor & field_descriptor_, + const ProtobufReaderOrWriter & reader_or_writer_) + : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) { static_assert(!is_fixed_string, "This constructor for String only"); setFunctions(); @@ -649,6 +681,13 @@ namespace } } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerString<" << (is_fixed_string ? "fixed" : "") << ">: column " + << quoteString(column_name) << " -> field " << quoteString(field_descriptor.full_name()) << " (" + << field_descriptor.type_name() << ")\n"; + } + private: void setFunctions() { @@ -799,18 +838,10 @@ namespace } default: - failedToSetFunctions(); + this->incompatibleColumnType(is_fixed_string ? "FixedString" : "String"); } } - [[noreturn]] void failedToSetFunctions() - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type " + quoteString(is_fixed_string ? "FixedString" : "String"), - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } - const PaddedPODArray & getDefaultString() { if (!default_string) @@ -890,16 +921,24 @@ namespace using BaseClass = ProtobufSerializerNumber; ProtobufSerializerEnum( + const std::string_view & column_name_, const std::shared_ptr & enum_data_type_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : BaseClass(field_descriptor_, reader_or_writer_), enum_data_type(enum_data_type_) + : BaseClass(column_name_, field_descriptor_, reader_or_writer_), enum_data_type(enum_data_type_) { assert(enum_data_type); setFunctions(); prepareEnumMapping(); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerEnum<" << TypeName << ">: column " << quoteString(this->column_name) + << " -> field " << quoteString(this->field_descriptor.full_name()) << " (" + << this->field_descriptor.type_name() << ")\n"; + } + private: void setFunctions() { @@ -964,18 +1003,10 @@ namespace } default: - failedToSetFunctions(); + this->incompatibleColumnType(enum_data_type->getName()); } } - [[noreturn]] void failedToSetFunctions() - { - throw Exception( - "The field " + quoteString(this->field_descriptor.full_name()) + " has an incompatible type " + this->field_descriptor.type_name() - + " for serialization of the data type " + quoteString(enum_data_type->getName()), - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } - void checkEnumDataTypeValue(NumberType value) { enum_data_type->findByValue(value); /// Throws an exception if the value isn't defined in the DataTypeEnum. @@ -1089,10 +1120,11 @@ namespace using ColumnType = ColumnDecimal; ProtobufSerializerDecimal( + const std::string_view & column_name_, const DataTypeDecimalBase & decimal_data_type_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) , precision(decimal_data_type_.getPrecision()) , scale(decimal_data_type_.getScale()) { @@ -1123,6 +1155,13 @@ namespace column_decimal.insertValue(getDefaultDecimal()); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerDecimal<" << TypeName << ">: column " << quoteString(column_name) + << " -> field " << quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() + << ")\n"; + } + private: void setFunctions() { @@ -1227,7 +1266,7 @@ namespace case FieldTypeId::TYPE_BOOL: { if (std::is_same_v) - failedToSetFunctions(); + incompatibleColumnType(TypeName); else { write_function = [this](const DecimalType & decimal) @@ -1281,18 +1320,10 @@ namespace } default: - failedToSetFunctions(); + incompatibleColumnType(TypeName); } } - [[noreturn]] void failedToSetFunctions() - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type " + quoteString(TypeName), - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } - DecimalType getDefaultDecimal() { if (!default_decimal) @@ -1349,13 +1380,20 @@ namespace { public: ProtobufSerializerDate( + const std::string_view & column_name_, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerNumber(field_descriptor_, reader_or_writer_) + : ProtobufSerializerNumber(column_name_, field_descriptor_, reader_or_writer_) { setFunctions(); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerDate: column " << quoteString(column_name) << " -> field " + << quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() << ")\n"; + } + private: void setFunctions() { @@ -1395,7 +1433,7 @@ namespace } default: - failedToSetFunctions(); + incompatibleColumnType("Date"); } } @@ -1412,14 +1450,6 @@ namespace readDateText(date, buf); return date; } - - [[noreturn]] void failedToSetFunctions() - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type 'Date'", - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } }; @@ -1428,15 +1458,22 @@ namespace { public: ProtobufSerializerDateTime( + const std::string_view & column_name_, const DataTypeDateTime & type, const FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerNumber(field_descriptor_, reader_or_writer_), + : ProtobufSerializerNumber(column_name_, field_descriptor_, reader_or_writer_), date_lut(type.getTimeZone()) { setFunctions(); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerDateTime: column " << quoteString(column_name) << " -> field " + << quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() << ")\n"; + } + protected: const DateLUTImpl & date_lut; @@ -1478,7 +1515,7 @@ namespace } default: - failedToSetFunctions(); + incompatibleColumnType("DateTime"); } } @@ -1497,14 +1534,6 @@ namespace tm = 0; return tm; } - - [[noreturn]] void failedToSetFunctions() - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type 'DateTime'", - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } }; @@ -1513,9 +1542,10 @@ namespace { public: ProtobufSerializerUUID( + const std::string_view & column_name_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) { setFunctions(); } @@ -1544,16 +1574,17 @@ namespace column_vector.insertDefault(); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerUUID: column " << quoteString(column_name) << " -> field " + << quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() << ")\n"; + } + private: void setFunctions() { if ((field_typeid != FieldTypeId::TYPE_STRING) && (field_typeid != FieldTypeId::TYPE_BYTES)) - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type UUID", - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } + incompatibleColumnType("UUID"); write_function = [this](UUID value) { @@ -1591,20 +1622,16 @@ namespace { public: ProtobufSerializerAggregateFunction( + const std::string_view & column_name_, const std::shared_ptr & aggregate_function_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) - : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + : ProtobufSerializerSingleValue(column_name_, field_descriptor_, reader_or_writer_) , aggregate_function_data_type(aggregate_function_data_type_) , aggregate_function(aggregate_function_data_type->getFunction()) { if ((field_typeid != FieldTypeId::TYPE_STRING) && (field_typeid != FieldTypeId::TYPE_BYTES)) - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) + " has an incompatible type " + field_descriptor.type_name() - + " for serialization of the data type " + quoteString(aggregate_function_data_type->getName()), - ErrorCodes::DATA_TYPE_INCOMPATIBLE_WITH_PROTOBUF_FIELD); - } + incompatibleColumnType(aggregate_function_data_type->getName()); } void writeRow(size_t row_num) override @@ -1642,6 +1669,12 @@ namespace column_af.getData().push_back(data); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerAggregateFunction: column " << quoteString(column_name) << " -> field " + << quoteString(field_descriptor.full_name()) << " (" << field_descriptor.type_name() << ")\n"; + } + private: void dataToString(ConstAggregateDataPtr data, String & str) const { @@ -1684,7 +1717,8 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); column = columns[0]; const auto & column_nullable = assert_cast(*column); ColumnPtr nested_column = column_nullable.getNestedColumnPtr(); @@ -1693,7 +1727,8 @@ namespace void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); ColumnPtr column0 = columns[0]->getPtr(); setColumns(&column0, 1); } @@ -1744,6 +1779,12 @@ namespace column_nullable.insertDefault(); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerNullable ->\n"; + nested_serializer->describeTree(out, indent + 1); + } + private: const std::unique_ptr nested_serializer; ColumnPtr column; @@ -1761,7 +1802,8 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); const auto & column_map = assert_cast(*columns[0]); ColumnPtr nested_column = column_map.getNestedColumnPtr(); nested_serializer->setColumns(&nested_column, 1); @@ -1769,7 +1811,8 @@ namespace void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); ColumnPtr column0 = columns[0]->getPtr(); setColumns(&column0, 1); } @@ -1778,6 +1821,12 @@ namespace void readRow(size_t row_num) override { nested_serializer->readRow(row_num); } void insertDefaults(size_t row_num) override { nested_serializer->insertDefaults(row_num); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerMap ->\n"; + nested_serializer->describeTree(out, indent + 1); + } + private: const std::unique_ptr nested_serializer; }; @@ -1794,7 +1843,8 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); column = columns[0]; const auto & column_lc = assert_cast(*column); ColumnPtr nested_column = column_lc.getDictionary().getNestedColumn(); @@ -1804,7 +1854,8 @@ namespace void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); ColumnPtr column0 = columns[0]->getPtr(); setColumns(&column0, 1); } @@ -1862,6 +1913,12 @@ namespace column_lc.insertFromFullColumn(*default_value_column, 0); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerLowCardinality ->\n"; + nested_serializer->describeTree(out, indent + 1); + } + private: const std::unique_ptr nested_serializer; ColumnPtr column; @@ -1882,7 +1939,8 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); column = columns[0]; const auto & column_array = assert_cast(*column); ColumnPtr data_column = column_array.getDataPtr(); @@ -1891,7 +1949,8 @@ namespace void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); ColumnPtr column0 = columns[0]->getPtr(); setColumns(&column0, 1); } @@ -1944,6 +2003,12 @@ namespace column_array.insertDefault(); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerArray ->\n"; + element_serializer->describeTree(out, indent + 1); + } + private: const std::unique_ptr element_serializer; ColumnPtr column; @@ -1955,10 +2020,12 @@ namespace { public: ProtobufSerializerTupleAsArray( + const std::string_view & column_name_, const std::shared_ptr & tuple_data_type_, const FieldDescriptor & field_descriptor_, std::vector> element_serializers_) - : tuple_data_type(tuple_data_type_) + : column_name(column_name_) + , tuple_data_type(tuple_data_type_) , tuple_size(tuple_data_type->getElements().size()) , field_descriptor(field_descriptor_) , element_serializers(std::move(element_serializers_)) @@ -1969,7 +2036,8 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); column = columns[0]; const auto & column_tuple = assert_cast(*column); for (size_t i : collections::range(tuple_size)) @@ -1982,7 +2050,8 @@ namespace void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); ColumnPtr column0 = columns[0]->getPtr(); setColumns(&column0, 1); } @@ -2006,9 +2075,12 @@ namespace if (current_element_index >= tuple_size) { throw Exception( - "Too many (" + std::to_string(current_element_index) + ") elements was read from the field " - + field_descriptor.full_name() + " to fit in the data type " + tuple_data_type->getName(), - ErrorCodes::PROTOBUF_BAD_CAST); + ErrorCodes::PROTOBUF_BAD_CAST, + "Column {}: More than {} elements was read from the field {} to fit in the data type {}", + quoteString(column_name), + tuple_size, + quoteString(field_descriptor.full_name()), + tuple_data_type->getName()); } element_serializers[current_element_index]->readRow(row_num); @@ -2040,7 +2112,17 @@ namespace } } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerTupleAsArray: column " << quoteString(column_name) << " (" + << tuple_data_type->getName() << ") -> field " << quoteString(field_descriptor.full_name()) << " (" + << field_descriptor.type_name() << ") ->\n"; + for (const auto & element_serializer : element_serializers) + element_serializer->describeTree(out, indent + 1); + } + private: + const String column_name; const std::shared_ptr tuple_data_type; const size_t tuple_size; const FieldDescriptor & field_descriptor; @@ -2085,6 +2167,9 @@ namespace void setColumns(const ColumnPtr * columns_, size_t num_columns_) override { + if (!num_columns_) + wrongNumberOfColumns(num_columns_, ">0"); + columns.assign(columns_, columns_ + num_columns_); std::vector field_columns; @@ -2094,7 +2179,8 @@ namespace field_columns.reserve(info.column_indices.size()); for (size_t column_index : info.column_indices) { - assert(column_index < num_columns_); + if (column_index >= num_columns_) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong column index {}, expected column indices <{}", column_index, num_columns_); field_columns.emplace_back(columns_[column_index]); } info.field_serializer->setColumns(field_columns.data(), field_columns.size()); @@ -2206,6 +2292,32 @@ namespace addDefaultsToMissingColumns(row_num); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + size_t num_columns = 0; + for (const auto & field_info : field_infos) + num_columns += field_info.column_indices.size(); + + writeIndent(out, indent) << "ProtobufSerializerMessage: " << num_columns << " columns ->"; + if (parent_field_descriptor) + out << " field " << quoteString(parent_field_descriptor->full_name()) << " (" << parent_field_descriptor->type_name() << ")"; + + for (size_t i = 0; i != field_infos.size(); ++i) + { + out << "\n"; + const auto & field_info = field_infos[i]; + writeIndent(out, indent + 1) << "Columns #"; + for (size_t j = 0; j != field_info.column_indices.size(); ++j) + { + if (j) + out << ", "; + out << field_info.column_indices[j]; + } + out << " ->\n"; + field_info.field_serializer->describeTree(out, indent + 2); + } + } + private: size_t findFieldIndexByFieldTag(int field_tag) { @@ -2285,7 +2397,8 @@ namespace void setColumns(const ColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); const auto & column_tuple = assert_cast(*columns[0]); size_t tuple_size = column_tuple.tupleSize(); assert(tuple_size); @@ -2298,7 +2411,8 @@ namespace void setColumns(const MutableColumnPtr * columns, [[maybe_unused]] size_t num_columns) override { - assert(num_columns == 1); + if (num_columns != 1) + wrongNumberOfColumns(num_columns, "1"); ColumnPtr column0 = columns[0]->getPtr(); setColumns(&column0, 1); } @@ -2307,6 +2421,12 @@ namespace void readRow(size_t row_num) override { message_serializer->readRow(row_num); } void insertDefaults(size_t row_num) override { message_serializer->insertDefaults(row_num); } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerTupleAsNestedMessage ->\n"; + message_serializer->describeTree(out, indent + 1); + } + private: const std::unique_ptr message_serializer; }; @@ -2318,14 +2438,23 @@ namespace { public: explicit ProtobufSerializerFlattenedNestedAsArrayOfNestedMessages( - std::unique_ptr message_serializer_) - : message_serializer(std::move(message_serializer_)) + const std::vector & column_names_, + const FieldDescriptor * parent_field_descriptor_, + std::unique_ptr message_serializer_, + const std::function & get_root_desc_function_) + : parent_field_descriptor(parent_field_descriptor_) + , message_serializer(std::move(message_serializer_)) + , get_root_desc_function(get_root_desc_function_) { + column_names.reserve(column_names_.size()); + for (const auto & column_name : column_names_) + column_names.emplace_back(column_name); } void setColumns(const ColumnPtr * columns, size_t num_columns) override { - assert(num_columns); + if (!num_columns) + wrongNumberOfColumns(num_columns, ">0"); data_columns.clear(); data_columns.reserve(num_columns); offset_columns.clear(); @@ -2335,11 +2464,26 @@ namespace { const auto & column_array = assert_cast(*columns[i]); data_columns.emplace_back(column_array.getDataPtr()); - offset_columns.emplace_back(column_array.getOffsetsPtr()); - } - std::sort(offset_columns.begin(), offset_columns.end()); - offset_columns.erase(std::unique(offset_columns.begin(), offset_columns.end()), offset_columns.end()); + auto offset_column = column_array.getOffsetsPtr(); + if (std::binary_search(offset_columns.begin(), offset_columns.end(), offset_column)) + continue; + + /// Keep `offset_columns` sorted. + offset_columns.insert(std::upper_bound(offset_columns.begin(), offset_columns.end(), offset_column), offset_column); + + /// All the columns listed in `offset_columns` should have equal offsets. + if (i >= 1) + { + const auto & column_array0 = assert_cast(*columns[0]); + if (!column_array0.hasEqualOffsets(column_array)) + { + throw Exception(ErrorCodes::PROTOBUF_BAD_CAST, + "Column #{} {} and column #{} {} are supposed to have equal offsets according to the following serialization tree:\n{}", + 0, quoteString(column_names[0]), i, quoteString(column_names[i]), get_root_desc_function(0)); + } + } + } message_serializer->setColumns(data_columns.data(), data_columns.size()); } @@ -2358,12 +2502,6 @@ namespace const auto & offset_column0 = assert_cast(*offset_columns[0]); size_t start_offset = offset_column0.getElement(row_num - 1); size_t end_offset = offset_column0.getElement(row_num); - for (size_t i : collections::range(1, offset_columns.size())) - { - const auto & offset_column = assert_cast(*offset_columns[i]); - if (offset_column.getElement(row_num) != end_offset) - throw Exception("Components of FlattenedNested have different sizes", ErrorCodes::PROTOBUF_BAD_CAST); - } for (size_t i : collections::range(start_offset, end_offset)) message_serializer->writeRow(i); } @@ -2433,8 +2571,26 @@ namespace } } + void describeTree(WriteBuffer & out, size_t indent) const override + { + writeIndent(out, indent) << "ProtobufSerializerFlattenedNestedAsArrayOfNestedMessages: columns "; + for (size_t i = 0; i != column_names.size(); ++i) + { + if (i) + out << ", "; + out << "#" << i << " " << quoteString(column_names[i]); + } + out << " ->"; + if (parent_field_descriptor) + out << " field " << quoteString(parent_field_descriptor->full_name()) << " (" << parent_field_descriptor->type_name() << ") ->\n"; + message_serializer->describeTree(out, indent + 1); + } + private: + Strings column_names; + const FieldDescriptor * parent_field_descriptor; const std::unique_ptr message_serializer; + const std::function get_root_desc_function; Columns data_columns; Columns offset_columns; }; @@ -2453,6 +2609,14 @@ namespace const MessageDescriptor & message_descriptor, bool with_length_delimiter) { + root_serializer_ptr = std::make_shared(); + get_root_desc_function = [root_serializer_ptr = root_serializer_ptr](size_t indent) -> String + { + WriteBufferFromOwnString buf; + (*root_serializer_ptr)->describeTree(buf, indent); + return buf.str(); + }; + std::vector used_column_indices; auto message_serializer = buildMessageSerializerImpl( /* num_columns = */ column_names.size(), @@ -2480,6 +2644,12 @@ namespace boost::range::set_difference(collections::range(column_names.size()), used_column_indices_sorted, std::back_inserter(missing_column_indices)); + *root_serializer_ptr = message_serializer.get(); + +#if 0 + LOG_INFO(&Poco::Logger::get("ProtobufSerializer"), "Serialization tree:\n{}", get_root_desc_function(0)); +#endif + return message_serializer; } @@ -2635,10 +2805,35 @@ namespace /// Set `columns_are_reordered_outside` to true if you're going to reorder columns /// according to `used_column_indices` returned and pass to /// ProtobufSerializerMessage::setColumns() only the columns which are actually used. - template std::unique_ptr buildMessageSerializerImpl( size_t num_columns, - const StringOrStringViewT * column_names, + const String * column_names, + const DataTypePtr * data_types, + const MessageDescriptor & message_descriptor, + bool with_length_delimiter, + const FieldDescriptor * parent_field_descriptor, + std::vector & used_column_indices, + bool columns_are_reordered_outside) + { + std::vector column_names_sv; + column_names_sv.reserve(num_columns); + for (size_t i = 0; i != num_columns; ++i) + column_names_sv.emplace_back(column_names[i]); + + return buildMessageSerializerImpl( + num_columns, + column_names_sv.data(), + data_types, + message_descriptor, + with_length_delimiter, + parent_field_descriptor, + used_column_indices, + columns_are_reordered_outside); + } + + std::unique_ptr buildMessageSerializerImpl( + size_t num_columns, + const std::string_view * column_names, const DataTypePtr * data_types, const MessageDescriptor & message_descriptor, bool with_length_delimiter, @@ -2814,7 +3009,11 @@ namespace if (nested_message_serializer) { - auto field_serializer = std::make_unique(std::move(nested_message_serializer)); + std::vector column_names_used; + for (size_t i : used_column_indices_in_nested) + column_names_used.emplace_back(nested_column_names[i]); + auto field_serializer = std::make_unique( + std::move(column_names_used), field_descriptor, std::move(nested_message_serializer), get_root_desc_function); transformColumnIndices(used_column_indices_in_nested, nested_column_indices); add_field_serializer(column_name, std::move(used_column_indices_in_nested), *field_descriptor, std::move(field_serializer)); break; @@ -2856,34 +3055,34 @@ namespace auto data_type_id = data_type->getTypeId(); switch (data_type_id) { - case TypeIndex::UInt8: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::UInt16: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::UInt32: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::UInt64: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::UInt128: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::UInt256: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Int8: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Int16: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Int32: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Int64: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Int128: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Int256: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Float32: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Float64: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::Date: return std::make_unique(field_descriptor, reader_or_writer); - case TypeIndex::DateTime: return std::make_unique(assert_cast(*data_type), field_descriptor, reader_or_writer); - case TypeIndex::DateTime64: return std::make_unique(assert_cast(*data_type), field_descriptor, reader_or_writer); - case TypeIndex::String: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::FixedString: return std::make_unique>(typeid_cast>(data_type), field_descriptor, reader_or_writer); - case TypeIndex::Enum8: return std::make_unique>(typeid_cast>(data_type), field_descriptor, reader_or_writer); - case TypeIndex::Enum16: return std::make_unique>(typeid_cast>(data_type), field_descriptor, reader_or_writer); - case TypeIndex::Decimal32: return std::make_unique>(assert_cast &>(*data_type), field_descriptor, reader_or_writer); - case TypeIndex::Decimal64: return std::make_unique>(assert_cast &>(*data_type), field_descriptor, reader_or_writer); - case TypeIndex::Decimal128: return std::make_unique>(assert_cast &>(*data_type), field_descriptor, reader_or_writer); - case TypeIndex::Decimal256: return std::make_unique>(assert_cast &>(*data_type), field_descriptor, reader_or_writer); - case TypeIndex::UUID: return std::make_unique(field_descriptor, reader_or_writer); - case TypeIndex::Interval: return std::make_unique(field_descriptor, reader_or_writer); - case TypeIndex::AggregateFunction: return std::make_unique(typeid_cast>(data_type), field_descriptor, reader_or_writer); + case TypeIndex::UInt8: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::UInt16: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::UInt32: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::UInt64: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::UInt128: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::UInt256: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Int8: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Int16: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Int32: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Int64: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Int128: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Int256: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Float32: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Float64: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Date: return std::make_unique(column_name, field_descriptor, reader_or_writer); + case TypeIndex::DateTime: return std::make_unique(column_name, assert_cast(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::DateTime64: return std::make_unique(column_name, assert_cast(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::String: return std::make_unique>(column_name, field_descriptor, reader_or_writer); + case TypeIndex::FixedString: return std::make_unique>(column_name, typeid_cast>(data_type), field_descriptor, reader_or_writer); + case TypeIndex::Enum8: return std::make_unique>(column_name, typeid_cast>(data_type), field_descriptor, reader_or_writer); + case TypeIndex::Enum16: return std::make_unique>(column_name, typeid_cast>(data_type), field_descriptor, reader_or_writer); + case TypeIndex::Decimal32: return std::make_unique>(column_name, assert_cast &>(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::Decimal64: return std::make_unique>(column_name, assert_cast &>(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::Decimal128: return std::make_unique>(column_name, assert_cast &>(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::Decimal256: return std::make_unique>(column_name, assert_cast &>(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::UUID: return std::make_unique(column_name, field_descriptor, reader_or_writer); + case TypeIndex::Interval: return std::make_unique(column_name, field_descriptor, reader_or_writer); + case TypeIndex::AggregateFunction: return std::make_unique(column_name, typeid_cast>(data_type), field_descriptor, reader_or_writer); case TypeIndex::Nullable: { @@ -2981,6 +3180,7 @@ namespace return nullptr; return std::make_unique( + column_name, typeid_cast>(data_type), field_descriptor, std::move(nested_serializers)); @@ -3007,6 +3207,8 @@ namespace } const ProtobufReaderOrWriter reader_or_writer; + std::function get_root_desc_function; + std::shared_ptr root_serializer_ptr; }; } diff --git a/src/Formats/ProtobufSerializer.h b/src/Formats/ProtobufSerializer.h index 315a138f9cf..3eaca6a18d6 100644 --- a/src/Formats/ProtobufSerializer.h +++ b/src/Formats/ProtobufSerializer.h @@ -15,7 +15,7 @@ class ProtobufWriter; class IDataType; using DataTypePtr = std::shared_ptr; using DataTypes = std::vector; - +class WriteBuffer; /// Utility class, does all the work for serialization in the Protobuf format. class ProtobufSerializer @@ -30,6 +30,8 @@ public: virtual void readRow(size_t row_num) = 0; virtual void insertDefaults(size_t row_num) = 0; + virtual void describeTree(WriteBuffer & out, size_t indent) const = 0; + static std::unique_ptr create( const Strings & column_names, const DataTypes & data_types, From 030ea3dc3f058a391d6cd49d10a842900e500147 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 5 Dec 2021 02:08:42 +0300 Subject: [PATCH 505/609] .. --- docs/en/sql-reference/functions/string-search-functions.md | 4 +++- docs/ru/sql-reference/functions/string-search-functions.md | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index c62603a50b9..7d46f676a42 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -373,7 +373,9 @@ The same as `multiMatchAny`, but returns the array of all indicies that match th ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} -The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). This function relies on the experimental feature of [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) library, and can be slow for some corner cases. The performance depends on the edit distance value and patterns used, but it's always more expensive compared to a non-fuzzy variants. +The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). + This function relies on the experimental feature of [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) library, and can be slow for some corner cases. + The performance depends on the edit distance value and patterns used, but it's always more expensive compared to a non-fuzzy variants. ## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index 5bbd760bfb6..6967b0e82d6 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -357,7 +357,9 @@ Result: ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} -То же, что и `multiMatchAny`, но возвращает 1 если любой pattern соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция также находится в экспериментальном режиме и может быть очень медленной. За подробностями обращайтесь к [документации hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching). +То же, что и `multiMatchAny`, но возвращает 1 если любой pattern соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). + Эта функция также находится в экспериментальном режиме и может быть очень медленной. + За подробностями обращайтесь к [документации hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching). ## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} From f4055f1957655111780073cccb40e08c0940f1d9 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 5 Dec 2021 19:36:12 +0300 Subject: [PATCH 506/609] Update multiFuzzyMatchAny --- docs/en/sql-reference/functions/string-search-functions.md | 4 +--- docs/ru/sql-reference/functions/string-search-functions.md | 5 ++--- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/string-search-functions.md b/docs/en/sql-reference/functions/string-search-functions.md index 7d46f676a42..c62603a50b9 100644 --- a/docs/en/sql-reference/functions/string-search-functions.md +++ b/docs/en/sql-reference/functions/string-search-functions.md @@ -373,9 +373,7 @@ The same as `multiMatchAny`, but returns the array of all indicies that match th ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} -The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). - This function relies on the experimental feature of [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) library, and can be slow for some corner cases. - The performance depends on the edit distance value and patterns used, but it's always more expensive compared to a non-fuzzy variants. +The same as `multiMatchAny`, but returns 1 if any pattern matches the haystack within a constant [edit distance](https://en.wikipedia.org/wiki/Edit_distance). This function relies on the experimental feature of [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) library, and can be slow for some corner cases. The performance depends on the edit distance value and patterns used, but it's always more expensive compared to a non-fuzzy variants. ## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index 6967b0e82d6..5b4a02b9e5b 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -357,9 +357,8 @@ Result: ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} -То же, что и `multiMatchAny`, но возвращает 1 если любой pattern соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). - Эта функция также находится в экспериментальном режиме и может быть очень медленной. - За подробностями обращайтесь к [документации hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching). +То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). +Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с нечеткими вариантами. ## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} From 3efb4038ca8abd24324732160754810b3b969a65 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 5 Dec 2021 19:46:24 +0300 Subject: [PATCH 507/609] sparkbar description --- .../aggregate-functions/reference/sparkbar.md | 56 +++++++++++++++++++ 1 file changed, 56 insertions(+) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/sparkbar.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md new file mode 100644 index 00000000000..9fcf752d36e --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md @@ -0,0 +1,56 @@ +--- +toc_priority: 311 +toc_title: sparkbar +--- + +# sparkbar {#sparkbar} + +The function plots a frequency histogram for values `x` and the repetition rate of these `y` values over the interval `[min_x, max_x]`. + +If no interval is specified, then the minimum `x` will be used as the interval start, and the maximum `x` will use as the interval end. + +**Syntax** + +``` sql +sparkbar(width, min_x, max_x)(x, y) +``` + +**Arguments** + +- `width` — The number of segments. Must be [Integer](../../../sql-reference/data-types/int-uint.md). +- `min_x` — The interval start. Optional value. +- `max_x` — The interval end. Optional value. +- `x` — The range of values. +- `y` — The frequency of values. + +**Returned value** + +- The frequency histogram. + +**Example** + +Query: + +``` sql +CREATE TABLE spark_bar_data (`cnt` UInt64,`event_date` Date) ENGINE = MergeTree ORDER BY event_date SETTINGS index_granularity = 8192; + +INSERT INTO spark_bar_data VALUES(1,'2020-01-01'),(4,'2020-01-02'),(5,'2020-01-03'),(2,'2020-01-04'),(3,'2020-01-05'),(7,'2020-01-06'),(6,'2020-01-07'),(8,'2020-01-08'),(2,'2020-01-11'); + +SELECT sparkbar(9)(event_date,cnt) FROM spark_bar_data; + +SELECT sparkbar(9,toDate('2020-01-01'),toDate('2020-01-10'))(event_date,cnt) FROM spark_bar_data; +``` + +Result: + +``` text + +┌─sparkbar(9)(event_date, cnt)─┐ +│ ▁▅▄▃██▅ ▁ │ +└──────────────────────────────┘ + +┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐ +│ ▁▄▄▂▅▇█▁ │ +└──────────────────────────────────────────────────────────────────────────┘ +``` + From f386ce3495807734efc0c958bd58cdf3f744f941 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 5 Dec 2021 20:29:10 +0300 Subject: [PATCH 508/609] fix example --- .../aggregate-functions/reference/sparkbar.md | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md index 9fcf752d36e..34052aac86f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md @@ -46,11 +46,15 @@ Result: ``` text ┌─sparkbar(9)(event_date, cnt)─┐ -│ ▁▅▄▃██▅ ▁ │ +│ │ +│ ▁▅▄▃██▅ ▁ │ +│ │ └──────────────────────────────┘ ┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐ -│ ▁▄▄▂▅▇█▁ │ +│ │ +│▁▄▄▂▅▇█▁ │ +│ │ └──────────────────────────────────────────────────────────────────────────┘ ``` From 715a25d4f484dbf178a181a29e1f61e5604b8a46 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 5 Dec 2021 20:12:42 +0300 Subject: [PATCH 509/609] Update query_log settings Update query_views_log.md Update query_views_log Minor fixes --- docs/en/operations/settings/settings.md | 11 ++- .../system-tables/query_views_log.md | 4 +- .../settings.md | 8 +- docs/ru/operations/settings/settings.md | 13 ++- docs/ru/operations/system-tables/query_log.md | 1 + .../system-tables/query_thread_log.md | 4 +- .../system-tables/query_views_log.md | 87 ++++++++++++++++++- .../functions/string-search-functions.md | 3 +- 8 files changed, 115 insertions(+), 16 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index bb1a807c766..c4c13a48560 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -992,9 +992,16 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. +Query threads log into `system.query_thread_log` table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. -Example: +Possible values: + +- 0 — Disabled. +- 1 — Enabled. + +Default value: `1`. + +**Example** ``` text log_query_threads=1 diff --git a/docs/en/operations/system-tables/query_views_log.md b/docs/en/operations/system-tables/query_views_log.md index 25d7f6522f5..6a6bbef45e2 100644 --- a/docs/en/operations/system-tables/query_views_log.md +++ b/docs/en/operations/system-tables/query_views_log.md @@ -4,8 +4,8 @@ Contains information about the dependent views executed when running a query, fo To start logging: -1. Configure parameters in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) section. -2. Set [log_query_views](../../operations/settings/settings.md#settings-log-query-views) to 1. +1. Configure parameters in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) section. +2. Set [log_query_views](../../operations/settings/settings.md#settings-log-query-views) to 1. The flushing period of data is set in `flush_interval_milliseconds` parameter of the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) server settings section. To force flushing, use the [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs) query. diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index c194c70ebbc..a19380e36f4 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -999,14 +999,14 @@ ClickHouse проверяет условия для `min_part_size` и `min_part Настройки логирования информации о зависимых представлениях (materialized, live и т.п.) в запросах принятых с настройкой [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views). -Запросы сохраняются в таблицу system.query_views_log. Вы можете изменить название этой таблицы в параметре `table` (см. ниже). +Запросы логируются в таблице [system.query_views_log](../../operations/system-tables/query_views_log.md#system_tables-query_views_log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). При настройке логирования используются следующие параметры: - `database` – имя базы данных. -- `table` – имя таблицы куда будут записываться использованные представления. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если используется `engine` -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если используется `partition_by`. +- `table` – имя системной таблицы, где будут логироваться запросы. +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если указывается параметр `engine`. +- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если задан параметр `partition_by`. - `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1cbdfc913b5..d6830a30321 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -912,11 +912,18 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' ## log_query_threads {#settings-log-query-threads} -Установка логирования информации о потоках выполнения запроса. +Управляет логированием информации о потоках выполнения запросов. -Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). +Информация о потоках выполнения запросов сохраняется в системной таблице `system.query_thread_log`. Работает только в том случае, если включена настройка [log_queries](#settings-log-queries). Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). -Пример: +Возможные значения: + +- 0 — отключено. +- 1 — включено. + +Значение по умолчанию: `1`. + +**Пример** ``` text log_query_threads=1 diff --git a/docs/ru/operations/system-tables/query_log.md b/docs/ru/operations/system-tables/query_log.md index 644cee853cc..aa4d01a4d47 100644 --- a/docs/ru/operations/system-tables/query_log.md +++ b/docs/ru/operations/system-tables/query_log.md @@ -55,6 +55,7 @@ ClickHouse не удаляет данные из таблица автомати - `query_kind` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) — тип запроса. - `databases` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена баз данных, присутствующих в запросе. - `tables` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена таблиц, присутствующих в запросе. +- `views` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена представлений (материализованные или live), которые представленны в запросе. - `columns` ([Array](../../sql-reference/data-types/array.md)([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md))) — имена столбцов, присутствующих в запросе. - `projections` ([String](../../sql-reference/data-types/string.md)) — имена проекций, использованных при выполнении запроса. - `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — код исключения. diff --git a/docs/ru/operations/system-tables/query_thread_log.md b/docs/ru/operations/system-tables/query_thread_log.md index 00538c9c9ae..c23d2828520 100644 --- a/docs/ru/operations/system-tables/query_thread_log.md +++ b/docs/ru/operations/system-tables/query_thread_log.md @@ -112,5 +112,5 @@ ProfileEvents: {'Query':1,'SelectQuery':1,'ReadCompressedBytes':36,'Compr **Смотрите также** -- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах. - +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах. +- [system.query_views_log](../../operations/system-tables/query_views_log.md#system_tables-query_views_log) — описание системной таблицы `query_views_log`, которая содержит информацию о всех представлениях, участвующих в выполненных запросах. diff --git a/docs/ru/operations/system-tables/query_views_log.md b/docs/ru/operations/system-tables/query_views_log.md index f606e4108ca..7b67d7b0b82 120000 --- a/docs/ru/operations/system-tables/query_views_log.md +++ b/docs/ru/operations/system-tables/query_views_log.md @@ -1 +1,86 @@ -../../../en/operations/system-tables/query_views_log.md \ No newline at end of file +# system.query_views_log {#system_tables-query_views_log} + +Содержит информацию о зависимых представлениях, выполняемых при выполнении запроса, например, тип представления или время выполнения. + +Чтобы начать ведение журнала: + +1. Настройте параметры в разделе [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log). +2. Включите настройку [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views). + + + +Период сброса данных из буфера в памяти задается в параметре `flush_interval_milliseconds` в разделе настроек сервера [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log ). Для принудительного сброса используйте запрос [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs). + +ClickHouse не удаляет данные из таблицы автоматически. Подробнее смотрите раздел [Системные таблицы](../../operations/system-tables/index.md#system-tables-introduction). + +Вы можете включить настройку [log_queries_probability](../../operations/settings/settings.md#log-queries-probability), чтобы уменьшить количество запросов, регистрируемых в таблице `query_views_log`. + +Столбцы: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата, когда произошло последнее событие с представлением. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления с точностью до микросекунд. +- `view_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — продолжительность выполнения представления (сумма его этапов) в миллисекундах. +- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — идентификатор начального запроса (при распределённом выполнении запроса). +- `view_name` ([String](../../sql-reference/data-types/string.md)) — имя представления. +- `view_uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — UUID представления. +- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — тип представления. Возможные значения: + - `'Default' = 1` — [обычные представления](../../sql-reference/statements/create/view.md#normal). Не должно появляться в этом журнале. + - `'Materialized' = 2` — [материализованные представления](../../sql-reference/statements/create/view.md#materialized). + - `'Live' = 3` — [live представления](../../sql-reference/statements/create/view.md#live-view). +- `view_query` ([String](../../sql-reference/data-types/string.md)) — запрос, выполняемый представлением. +- `view_target` ([String](../../sql-reference/data-types/string.md)) — имя целевой таблицы представления. +- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных строк. +- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных байт. +- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных строк. +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных байт. +- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — максимальная разница между объемом выделенной и освобожденной памяти в контексте этого представления. +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — События профиля, которые измеряют различные показатели. Их описание можно найти в таблице [system.events](../../operations/system-tables/events.md#system_tables-events). +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — статус представления. Возможные значения: + - `'QueryStart' = 1` — успешное начало выполнения представления. Не должно отображаться. + - `'QueryFinish' = 2` — успешное завершение выполнения представления. + - `'ExceptionBeforeStart' = 3` — исключение до начала выполнения представления. + - `'ExceptionWhileProcessing' = 4` — исключение во время выполнения представления. +- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — код исключения. +- `exception` ([String](../../sql-reference/data-types/string.md)) — сообщение исключения. +- `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [трассировка стека](https://ru.wikipedia.org/wiki/Трассировка_стека). Пустая строка, если запрос был успешно выполнен. + +**Пример** + +Запрос: + +``` sql +SELECT * FROM system.query_views_log LIMIT 1 \G; +``` + +Результат: + +``` text +Row 1: +────── +event_date: 2021-06-22 +event_time: 2021-06-22 13:23:07 +event_time_microseconds: 2021-06-22 13:23:07.738221 +view_duration_ms: 0 +initial_query_id: c3a1ac02-9cad-479b-af54-9e9c0a7afd70 +view_name: default.matview_inner +view_uuid: 00000000-0000-0000-0000-000000000000 +view_type: Materialized +view_query: SELECT * FROM default.table_b +view_target: default.`.inner.matview_inner` +read_rows: 4 +read_bytes: 64 +written_rows: 2 +written_bytes: 32 +peak_memory_usage: 4196188 +ProfileEvents: {'FileOpen':2,'WriteBufferFromFileDescriptorWrite':2,'WriteBufferFromFileDescriptorWriteBytes':187,'IOBufferAllocs':3,'IOBufferAllocBytes':3145773,'FunctionExecute':3,'DiskWriteElapsedMicroseconds':13,'InsertedRows':2,'InsertedBytes':16,'SelectedRows':4,'SelectedBytes':48,'ContextLock':16,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':698,'SoftPageFaults':4,'OSReadChars':463} +status: QueryFinish +exception_code: 0 +exception: +stack_trace: +``` + +**См. также** + +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах. +- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — описание системной таблицы `query_thread_log`, которая содержит информацию о каждом потоке выполнения запроса. diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index 5b4a02b9e5b..f0c8f51225e 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -357,8 +357,7 @@ Result: ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} -То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). -Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с нечеткими вариантами. +То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с нечеткими вариантами. ## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} From 428e34d81131ad5eb123a156b7fe33f4478d3224 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 5 Dec 2021 11:34:10 +0300 Subject: [PATCH 510/609] Fix versioning of aggregate functions (fixes performance tests) --- src/Columns/ColumnAggregateFunction.cpp | 17 ++++++++++++----- src/DataTypes/DataTypeAggregateFunction.cpp | 2 +- .../01881_aggregate_functions_versioning.sql | 12 ++++++++++++ 3 files changed, 25 insertions(+), 6 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index aa19aa2d8b0..0bd6b54a25a 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -28,13 +28,20 @@ namespace ErrorCodes } -static String getTypeString(const AggregateFunctionPtr & func) +static String getTypeString(const AggregateFunctionPtr & func, std::optional version = std::nullopt) { WriteBufferFromOwnString stream; - stream << "AggregateFunction(" << func->getName(); + + stream << "AggregateFunction("; + + /// If aggregate function does not support versioning its version is 0 and is not printed. + if (version && *version) + stream << *version << ", "; + + stream << func->getName(); + const auto & parameters = func->getParameters(); const auto & argument_types = func->getArgumentTypes(); - if (!parameters.empty()) { stream << '('; @@ -56,7 +63,7 @@ static String getTypeString(const AggregateFunctionPtr & func) ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & func_, std::optional version_) - : func(func_), type_string(getTypeString(func)), version(version_) + : func(func_), type_string(getTypeString(func, version_)), version(version_) { } @@ -403,7 +410,7 @@ void ColumnAggregateFunction::protect() MutableColumnPtr ColumnAggregateFunction::cloneEmpty() const { - return create(func); + return create(func, version); } Field ColumnAggregateFunction::operator[](size_t n) const diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index a5bf047f092..d572da1ecd0 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -85,7 +85,7 @@ String DataTypeAggregateFunction::getNameImpl(bool with_version) const MutableColumnPtr DataTypeAggregateFunction::createColumn() const { - return ColumnAggregateFunction::create(function, version); + return ColumnAggregateFunction::create(function, getVersion()); } diff --git a/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql b/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql index 2f6a60409d5..f5007f8ef9b 100644 --- a/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql +++ b/tests/queries/0_stateless/01881_aggregate_functions_versioning.sql @@ -8,3 +8,15 @@ CREATE TABLE test_table ENGINE = AggregatingMergeTree() ORDER BY (col1, col2); SHOW CREATE TABLE test_table; + +-- regression from performance tests comparison script +DROP TABLE IF EXISTS test; +CREATE TABLE test +ENGINE = Null AS +WITH ( + SELECT arrayReduce('sumMapState', [(['foo'], arrayMap(x -> -0., ['foo']))]) + ) AS all_metrics +SELECT + (finalizeAggregation(arrayReduce('sumMapMergeState', [all_metrics])) AS metrics_tuple).1 AS metric_names, + metrics_tuple.2 AS metric_values +FROM system.one; From 2b1789adc5dc240ce4f85eb5af4cd05c3548af01 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 5 Dec 2021 20:25:14 +0300 Subject: [PATCH 511/609] Pass through version for aggregate function from SerializationAggregateFunction too Suggested-by: @kssenii --- src/Columns/ColumnAggregateFunction.cpp | 5 +++-- src/Columns/ColumnAggregateFunction.h | 2 +- .../Serializations/SerializationAggregateFunction.cpp | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index 0bd6b54a25a..d675d166f5f 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -73,10 +73,11 @@ ColumnAggregateFunction::ColumnAggregateFunction(const AggregateFunctionPtr & fu } -void ColumnAggregateFunction::set(const AggregateFunctionPtr & func_) +void ColumnAggregateFunction::set(const AggregateFunctionPtr & func_, size_t version_) { func = func_; - type_string = getTypeString(func); + version = version_; + type_string = getTypeString(func, version); } diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index 548ad238f0d..b5efff928bb 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -103,7 +103,7 @@ private: public: ~ColumnAggregateFunction() override; - void set(const AggregateFunctionPtr & func_); + void set(const AggregateFunctionPtr & func_, size_t version_); AggregateFunctionPtr getAggregateFunction() { return func; } AggregateFunctionPtr getAggregateFunction() const { return func; } diff --git a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp index 47b4bed9b31..442df47a773 100644 --- a/src/DataTypes/Serializations/SerializationAggregateFunction.cpp +++ b/src/DataTypes/Serializations/SerializationAggregateFunction.cpp @@ -79,7 +79,7 @@ void SerializationAggregateFunction::deserializeBinaryBulk(IColumn & column, Rea ColumnAggregateFunction::Container & vec = real_column.getData(); Arena & arena = real_column.createOrGetArena(); - real_column.set(function); + real_column.set(function, version); vec.reserve(vec.size() + limit); size_t size_of_state = function->sizeOfData(); From 9222e6ed9a2438d6f962d69dbfa6fcab370b447a Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Sun, 5 Dec 2021 23:32:27 +0300 Subject: [PATCH 512/609] Update query_views_log.md Delete query_views_log.md Create query_views_log.md --- docs/ru/operations/system-tables/query_views_log.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) mode change 120000 => 100644 docs/ru/operations/system-tables/query_views_log.md diff --git a/docs/ru/operations/system-tables/query_views_log.md b/docs/ru/operations/system-tables/query_views_log.md deleted file mode 120000 index 7b67d7b0b82..00000000000 --- a/docs/ru/operations/system-tables/query_views_log.md +++ /dev/null @@ -1,86 +0,0 @@ -# system.query_views_log {#system_tables-query_views_log} - -Содержит информацию о зависимых представлениях, выполняемых при выполнении запроса, например, тип представления или время выполнения. - -Чтобы начать ведение журнала: - -1. Настройте параметры в разделе [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log). -2. Включите настройку [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views). - - - -Период сброса данных из буфера в памяти задается в параметре `flush_interval_milliseconds` в разделе настроек сервера [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log ). Для принудительного сброса используйте запрос [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs). - -ClickHouse не удаляет данные из таблицы автоматически. Подробнее смотрите раздел [Системные таблицы](../../operations/system-tables/index.md#system-tables-introduction). - -Вы можете включить настройку [log_queries_probability](../../operations/settings/settings.md#log-queries-probability), чтобы уменьшить количество запросов, регистрируемых в таблице `query_views_log`. - -Столбцы: - -- `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата, когда произошло последнее событие с представлением. -- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления. -- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления с точностью до микросекунд. -- `view_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — продолжительность выполнения представления (сумма его этапов) в миллисекундах. -- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — идентификатор начального запроса (при распределённом выполнении запроса). -- `view_name` ([String](../../sql-reference/data-types/string.md)) — имя представления. -- `view_uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — UUID представления. -- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — тип представления. Возможные значения: - - `'Default' = 1` — [обычные представления](../../sql-reference/statements/create/view.md#normal). Не должно появляться в этом журнале. - - `'Materialized' = 2` — [материализованные представления](../../sql-reference/statements/create/view.md#materialized). - - `'Live' = 3` — [live представления](../../sql-reference/statements/create/view.md#live-view). -- `view_query` ([String](../../sql-reference/data-types/string.md)) — запрос, выполняемый представлением. -- `view_target` ([String](../../sql-reference/data-types/string.md)) — имя целевой таблицы представления. -- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных строк. -- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных байт. -- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных строк. -- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных байт. -- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — максимальная разница между объемом выделенной и освобожденной памяти в контексте этого представления. -- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — События профиля, которые измеряют различные показатели. Их описание можно найти в таблице [system.events](../../operations/system-tables/events.md#system_tables-events). -- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — статус представления. Возможные значения: - - `'QueryStart' = 1` — успешное начало выполнения представления. Не должно отображаться. - - `'QueryFinish' = 2` — успешное завершение выполнения представления. - - `'ExceptionBeforeStart' = 3` — исключение до начала выполнения представления. - - `'ExceptionWhileProcessing' = 4` — исключение во время выполнения представления. -- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — код исключения. -- `exception` ([String](../../sql-reference/data-types/string.md)) — сообщение исключения. -- `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [трассировка стека](https://ru.wikipedia.org/wiki/Трассировка_стека). Пустая строка, если запрос был успешно выполнен. - -**Пример** - -Запрос: - -``` sql -SELECT * FROM system.query_views_log LIMIT 1 \G; -``` - -Результат: - -``` text -Row 1: -────── -event_date: 2021-06-22 -event_time: 2021-06-22 13:23:07 -event_time_microseconds: 2021-06-22 13:23:07.738221 -view_duration_ms: 0 -initial_query_id: c3a1ac02-9cad-479b-af54-9e9c0a7afd70 -view_name: default.matview_inner -view_uuid: 00000000-0000-0000-0000-000000000000 -view_type: Materialized -view_query: SELECT * FROM default.table_b -view_target: default.`.inner.matview_inner` -read_rows: 4 -read_bytes: 64 -written_rows: 2 -written_bytes: 32 -peak_memory_usage: 4196188 -ProfileEvents: {'FileOpen':2,'WriteBufferFromFileDescriptorWrite':2,'WriteBufferFromFileDescriptorWriteBytes':187,'IOBufferAllocs':3,'IOBufferAllocBytes':3145773,'FunctionExecute':3,'DiskWriteElapsedMicroseconds':13,'InsertedRows':2,'InsertedBytes':16,'SelectedRows':4,'SelectedBytes':48,'ContextLock':16,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':698,'SoftPageFaults':4,'OSReadChars':463} -status: QueryFinish -exception_code: 0 -exception: -stack_trace: -``` - -**См. также** - -- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах. -- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — описание системной таблицы `query_thread_log`, которая содержит информацию о каждом потоке выполнения запроса. diff --git a/docs/ru/operations/system-tables/query_views_log.md b/docs/ru/operations/system-tables/query_views_log.md new file mode 100644 index 00000000000..9715d739af1 --- /dev/null +++ b/docs/ru/operations/system-tables/query_views_log.md @@ -0,0 +1,84 @@ +# system.query_views_log {#system_tables-query_views_log} + +Содержит информацию о зависимых представлениях, выполняемых при выполнении запроса, например, тип представления или время выполнения. + +Чтобы начать ведение журнала: + +1. Настройте параметры в разделе [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log). +2. Включите настройку [log_query_views=1](../../operations/settings/settings.md#settings-log-query-views). + +Период сброса данных из буфера в памяти задается в параметре `flush_interval_milliseconds` в разделе настроек сервера [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log ). Для принудительного сброса используйте запрос [SYSTEM FLUSH LOGS](../../sql-reference/statements/system.md#query_language-system-flush_logs). + +ClickHouse не удаляет данные из таблицы автоматически. Подробнее смотрите раздел [Системные таблицы](../../operations/system-tables/index.md#system-tables-introduction). + +Вы можете включить настройку [log_queries_probability](../../operations/settings/settings.md#log-queries-probability), чтобы уменьшить количество запросов, регистрируемых в таблице `query_views_log`. + +Столбцы: + +- `event_date` ([Date](../../sql-reference/data-types/date.md)) — дата, когда произошло последнее событие с представлением. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления. +- `event_time_microseconds` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время завершения выполнения представления с точностью до микросекунд. +- `view_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — продолжительность выполнения представления (сумма его этапов) в миллисекундах. +- `initial_query_id` ([String](../../sql-reference/data-types/string.md)) — идентификатор начального запроса (при распределённом выполнении запроса). +- `view_name` ([String](../../sql-reference/data-types/string.md)) — имя представления. +- `view_uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — UUID представления. +- `view_type` ([Enum8](../../sql-reference/data-types/enum.md)) — тип представления. Возможные значения: + - `'Default' = 1` — [обычные представления](../../sql-reference/statements/create/view.md#normal). Не должно появляться в этом журнале. + - `'Materialized' = 2` — [материализованные представления](../../sql-reference/statements/create/view.md#materialized). + - `'Live' = 3` — [live представления](../../sql-reference/statements/create/view.md#live-view). +- `view_query` ([String](../../sql-reference/data-types/string.md)) — запрос, выполняемый представлением. +- `view_target` ([String](../../sql-reference/data-types/string.md)) — имя целевой таблицы представления. +- `read_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных строк. +- `read_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество прочитанных байт. +- `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных строк. +- `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных байт. +- `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — максимальная разница между объемом выделенной и освобожденной памяти в контексте этого представления. +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — События профиля, которые измеряют различные показатели. Их описание можно найти в таблице [system.events](../../operations/system-tables/events.md#system_tables-events). +- `status` ([Enum8](../../sql-reference/data-types/enum.md)) — статус представления. Возможные значения: + - `'QueryStart' = 1` — успешное начало выполнения представления. Не должно отображаться. + - `'QueryFinish' = 2` — успешное завершение выполнения представления. + - `'ExceptionBeforeStart' = 3` — исключение до начала выполнения представления. + - `'ExceptionWhileProcessing' = 4` — исключение во время выполнения представления. +- `exception_code` ([Int32](../../sql-reference/data-types/int-uint.md)) — код исключения. +- `exception` ([String](../../sql-reference/data-types/string.md)) — сообщение исключения. +- `stack_trace` ([String](../../sql-reference/data-types/string.md)) — [трассировка стека](https://ru.wikipedia.org/wiki/Трассировка_стека). Пустая строка, если запрос был успешно выполнен. + +**Пример** + +Запрос: + +``` sql +SELECT * FROM system.query_views_log LIMIT 1 \G; +``` + +Результат: + +``` text +Row 1: +────── +event_date: 2021-06-22 +event_time: 2021-06-22 13:23:07 +event_time_microseconds: 2021-06-22 13:23:07.738221 +view_duration_ms: 0 +initial_query_id: c3a1ac02-9cad-479b-af54-9e9c0a7afd70 +view_name: default.matview_inner +view_uuid: 00000000-0000-0000-0000-000000000000 +view_type: Materialized +view_query: SELECT * FROM default.table_b +view_target: default.`.inner.matview_inner` +read_rows: 4 +read_bytes: 64 +written_rows: 2 +written_bytes: 32 +peak_memory_usage: 4196188 +ProfileEvents: {'FileOpen':2,'WriteBufferFromFileDescriptorWrite':2,'WriteBufferFromFileDescriptorWriteBytes':187,'IOBufferAllocs':3,'IOBufferAllocBytes':3145773,'FunctionExecute':3,'DiskWriteElapsedMicroseconds':13,'InsertedRows':2,'InsertedBytes':16,'SelectedRows':4,'SelectedBytes':48,'ContextLock':16,'RWLockAcquiredReadLocks':1,'RealTimeMicroseconds':698,'SoftPageFaults':4,'OSReadChars':463} +status: QueryFinish +exception_code: 0 +exception: +stack_trace: +``` + +**См. также** + +- [system.query_log](../../operations/system-tables/query_log.md#system_tables-query_log) — описание системной таблицы `query_log`, которая содержит общую информацию о выполненных запросах. +- [system.query_thread_log](../../operations/system-tables/query_thread_log.md#system_tables-query_thread_log) — описание системной таблицы `query_thread_log`, которая содержит информацию о каждом потоке выполнения запроса. From 9d1e63be0fe0233e49982d1b3bfd59afa2a92f71 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 6 Dec 2021 11:54:45 +0800 Subject: [PATCH 513/609] 1. support default expression for storage hdfs; 2. optimize storage hdfs when format is column oriented --- src/Functions/FunctionMapMapped.h | 240 ++++++++++++++++++++ src/Storages/HDFS/StorageHDFS.cpp | 126 +++++++--- src/Storages/HDFS/StorageHDFS.h | 7 + tests/integration/test_storage_hdfs/test.py | 12 + 4 files changed, 349 insertions(+), 36 deletions(-) create mode 100644 src/Functions/FunctionMapMapped.h diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapMapped.h new file mode 100644 index 00000000000..ff746b58f71 --- /dev/null +++ b/src/Functions/FunctionMapMapped.h @@ -0,0 +1,240 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +// TODO 只要map没有实现去重,那么这个函数就不能用了。先等着吧 +/** Higher-order functions for maps. + * These functions optionally apply a map (transform) to map(or multiple map of identical size) by lambda function, + * and return some result based on that transformation. + * + * Examples: + * arrayMap(x1,...,xn -> expression, array1,...,arrayn) - apply the expression to each element of the array (or set of parallel arrays). + * arrayFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. + * + * For some functions arrayCount, arrayExists, arrayAll, an overload of the form f(array) is available, + * which works in the same way as f(x -> x, array). + * + * See the example of Impl template parameter in arrayMap.cpp + */ +template +class FunctionArrayMapped : public IFunction +{ +public: + static constexpr auto name = Name::name; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + + /// Called if at least one function argument is a lambda expression. + /// For argument-lambda expressions, it defines the types of arguments of these expressions. + void getLambdaArgumentTypes(DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception("Function " + getName() + " needs at least one argument; passed " + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments.size() == 1) + throw Exception("Function " + getName() + " needs at least one array argument.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + DataTypes nested_types(arguments.size() - 1); + for (size_t i = 0; i < nested_types.size(); ++i) + { + const DataTypeArray * array_type = checkAndGetDataType(&*arguments[i + 1]); + if (!array_type) + throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found " + + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); + } + + const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); + if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) + throw Exception("First argument for this overload of " + getName() + " must be a function with " + + toString(nested_types.size()) + " arguments. Found " + + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + arguments[0] = std::make_shared(nested_types); + } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + size_t min_args = Impl::needExpression() ? 2 : 1; + if (arguments.size() < min_args) + throw Exception("Function " + getName() + " needs at least " + + toString(min_args) + " argument; passed " + + toString(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (arguments.size() == 1) + { + const auto * array_type = checkAndGetDataType(arguments[0].type.get()); + + if (!array_type) + throw Exception("The only argument for function " + getName() + " must be array. Found " + + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + DataTypePtr nested_type = array_type->getNestedType(); + + if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8()) + throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " + + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return Impl::getReturnType(nested_type, nested_type); + } + else + { + if (arguments.size() > 2 && Impl::needOneArray()) + throw Exception("Function " + getName() + " needs one array argument.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); + + if (!data_type_function) + throw Exception("First argument for function " + getName() + " must be a function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. + + DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); + if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) + throw Exception("Expression for function " + getName() + " must return UInt8, found " + + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); + + return Impl::getReturnType(return_type, first_array_type->getNestedType()); + } + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override + { + if (arguments.size() == 1) + { + ColumnPtr column_array_ptr = arguments[0].column; + const auto * column_array = checkAndGetColumn(column_array_ptr.get()); + + if (!column_array) + { + const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); + if (!column_const_array) + throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + column_array_ptr = column_const_array->convertToFullColumn(); + column_array = assert_cast(column_array_ptr.get()); + } + + return Impl::execute(*column_array, column_array->getDataPtr()); + } + else + { + const auto & column_with_type_and_name = arguments[0]; + + if (!column_with_type_and_name.column) + throw Exception("First argument for function " + getName() + " must be a function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); + + if (!column_function) + throw Exception("First argument for function " + getName() + " must be a function.", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + ColumnPtr offsets_column; + + ColumnPtr column_first_array_ptr; + const ColumnArray * column_first_array = nullptr; + + ColumnsWithTypeAndName arrays; + arrays.reserve(arguments.size() - 1); + + for (size_t i = 1; i < arguments.size(); ++i) + { + const auto & array_with_type_and_name = arguments[i]; + + ColumnPtr column_array_ptr = array_with_type_and_name.column; + const auto * column_array = checkAndGetColumn(column_array_ptr.get()); + + const DataTypePtr & array_type_ptr = array_with_type_and_name.type; + const auto * array_type = checkAndGetDataType(array_type_ptr.get()); + + if (!column_array) + { + const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); + if (!column_const_array) + throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); + column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn()); + column_array = checkAndGetColumn(column_array_ptr.get()); + } + + if (!array_type) + throw Exception("Expected array type, found " + array_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!offsets_column) + { + offsets_column = column_array->getOffsetsPtr(); + } + else + { + /// The first condition is optimization: do not compare data if the pointers are equal. + if (column_array->getOffsetsPtr() != offsets_column + && column_array->getOffsets() != typeid_cast(*offsets_column).getData()) + throw Exception("Arrays passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + + if (i == 1) + { + column_first_array_ptr = column_array_ptr; + column_first_array = column_array; + } + + arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), + recursiveRemoveLowCardinality(array_type->getNestedType()), + array_with_type_and_name.name)); + } + + /// Put all the necessary columns multiplied by the sizes of arrays into the columns. + auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets())); + auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); + replicated_column_function->appendArguments(arrays); + + auto lambda_result = replicated_column_function->reduce().column; + if (lambda_result->lowCardinality()) + lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); + + return Impl::execute(*column_first_array, lambda_result); + } + } +}; + +} diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 90e63aef46d..cc974ccdce9 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -63,7 +64,11 @@ StorageHDFS::StorageHDFS( ContextPtr context_, const String & compression_method_ = "", ASTPtr partition_by_) - : IStorage(table_id_), WithContext(context_), uri(uri_), format_name(format_name_), compression_method(compression_method_) + : IStorage(table_id_) + , WithContext(context_) + , uri(uri_) + , format_name(format_name_) + , compression_method(compression_method_) , partition_by(partition_by_) { context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); @@ -76,8 +81,7 @@ StorageHDFS::StorageHDFS( setInMemoryMetadata(storage_metadata); } -namespace -{ +using StorageHDFSPtr = std::shared_ptr; class HDFSSource : public SourceWithProgress, WithContext { @@ -93,8 +97,12 @@ public: using SourcesInfoPtr = std::shared_ptr; - static Block getHeader(Block header, bool need_path_column, bool need_file_column) + static Block getHeader(const StorageMetadataPtr & metadata_snapshot, bool need_path_column, bool need_file_column) { + auto header = metadata_snapshot->getSampleBlock(); + + /// Note: AddingDefaultsBlockInputStream doesn't change header. + if (need_path_column) header.insert({DataTypeString().createColumn(), std::make_shared(), "_path"}); if (need_file_column) @@ -103,22 +111,35 @@ public: return header; } + static Block getBlockForSource( + const StorageHDFSPtr & storage, + const StorageMetadataPtr & metadata_snapshot, + const ColumnsDescription & columns_description, + const SourcesInfoPtr & files_info) + { + if (storage->isColumnOriented()) + return metadata_snapshot->getSampleBlockForColumns( + columns_description.getNamesOfPhysical(), storage->getVirtuals(), storage->getStorageID()); + else + return getHeader(metadata_snapshot, files_info->need_path_column, files_info->need_file_column); + } + HDFSSource( - SourcesInfoPtr source_info_, - String uri_, - String format_, - String compression_method_, - Block sample_block_, + StorageHDFSPtr storage_, + const StorageMetadataPtr & metadata_snapshot_, ContextPtr context_, - UInt64 max_block_size_) - : SourceWithProgress(getHeader(sample_block_, source_info_->need_path_column, source_info_->need_file_column)) + UInt64 max_block_size_, + SourcesInfoPtr source_info_, + String uri_without_path_, + ColumnsDescription columns_description_) + : SourceWithProgress(getBlockForSource(storage_, metadata_snapshot_, columns_description_, source_info_)) , WithContext(context_) + , storage(std::move(storage_)) + , metadata_snapshot(metadata_snapshot_) , source_info(std::move(source_info_)) - , uri(std::move(uri_)) - , format(std::move(format_)) - , compression_method(compression_method_) + , uri_without_path(std::move(uri_without_path_)) , max_block_size(max_block_size_) - , sample_block(std::move(sample_block_)) + , columns_description(std::move(columns_description_)) { } @@ -138,14 +159,30 @@ public: return {}; auto path = source_info->uris[pos]; - current_path = uri + path; + current_path = uri_without_path + path; - auto compression = chooseCompressionMethod(path, compression_method); - read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri, path, getContext()->getGlobalContext()->getConfigRef()), compression); - auto input_format = getContext()->getInputFormat(format, *read_buf, sample_block, max_block_size); - pipeline = QueryPipeline(std::move(input_format)); + auto compression = chooseCompressionMethod(path, storage->compression_method); + read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri_without_path, path, getContext()->getGlobalContext()->getConfigRef()), compression); - reader = std::make_unique(pipeline); + auto get_block_for_format = [&]() -> Block + { + if (storage->isColumnOriented()) + return metadata_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical()); + return metadata_snapshot->getSampleBlock(); + }; + auto input_format = getContext()->getInputFormat(storage->format_name, *read_buf, get_block_for_format(), max_block_size); + + QueryPipelineBuilder builder; + builder.init(Pipe(input_format)); + if (columns_description.hasDefaults()) + { + builder.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, columns_description, *input_format, getContext()); + }); + } + pipeline = std::make_unique(QueryPipelineBuilder::getPipeline(std::move(builder))); + reader = std::make_unique(*pipeline); } Block res; @@ -180,17 +217,17 @@ public: } private: - std::unique_ptr read_buf; - QueryPipeline pipeline; - std::unique_ptr reader; + StorageHDFSPtr storage; + StorageMetadataPtr metadata_snapshot; SourcesInfoPtr source_info; - String uri; - String format; - String compression_method; - String current_path; - + String uri_without_path; UInt64 max_block_size; - Block sample_block; + ColumnsDescription columns_description; + + std::unique_ptr read_buf; + std::unique_ptr pipeline; + std::unique_ptr reader; + String current_path; }; class HDFSSink : public SinkToStorage @@ -235,7 +272,6 @@ private: OutputFormatPtr writer; }; - class PartitionedHDFSSink : public PartitionedSink { public: @@ -314,13 +350,14 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c } } } - return result; } +bool StorageHDFS::isColumnOriented() const +{ + return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name); } - Pipe StorageHDFS::read( const Names & column_names, const StorageMetadataPtr & metadata_snapshot, @@ -355,11 +392,27 @@ Pipe StorageHDFS::read( num_streams = sources_info->uris.size(); Pipes pipes; - + auto this_ptr = std::static_pointer_cast(shared_from_this()); for (size_t i = 0; i < num_streams; ++i) - pipes.emplace_back(std::make_shared( - sources_info, uri_without_path, format_name, compression_method, metadata_snapshot->getSampleBlock(), context_, max_block_size)); + { + const auto get_columns_for_format = [&]() -> ColumnsDescription + { + if (isColumnOriented()) + return ColumnsDescription{ + metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()).getNamesAndTypesList()}; + else + return metadata_snapshot->getColumns(); + }; + pipes.emplace_back(std::make_shared( + this_ptr, + metadata_snapshot, + context_, + max_block_size, + sources_info, + uri_without_path, + get_columns_for_format())); + } return Pipe::unitePipes(std::move(pipes)); } @@ -450,6 +503,7 @@ NamesAndTypesList StorageHDFS::getVirtuals() const {"_file", std::make_shared()} }; } + } #endif diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 945f0b9f0f1..db6b078265d 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -38,7 +38,14 @@ public: bool supportsPartitionBy() const override { return true; } + /// Check if the format is column-oriented. + /// Is is useful because column oriented formats could effectively skip unknown columns + /// So we can create a header of only required columns in read method and ask + /// format to read only them. Note: this hack cannot be done with ordinary formats like TSV. + bool isColumnOriented() const; + protected: + friend class HDFSSource; StorageHDFS( const String & uri_, const StorageID & table_id_, diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index fda4d89805a..ede1dafefb1 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -310,6 +310,18 @@ def test_seekable_formats(started_cluster): result = node1.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) +def test_read_table_with_default(started_cluster): + hdfs_api = started_cluster.hdfs_api + + data = "n\n100\n" + hdfs_api.write_data("/simple_table_function", data) + assert hdfs_api.read_data("/simple_table_function") == data + + output = "n\tm\n100\t200\n" + assert node1.query( + "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSVWithNames', 'n UInt32, m UInt32 DEFAULT n * 2') FORMAT TSVWithNames") == output + + if __name__ == '__main__': cluster.start() From a98f740bf21440a5f521add95867902d11ccbc99 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Mon, 6 Dec 2021 12:01:39 +0800 Subject: [PATCH 514/609] remove unused file --- src/Functions/FunctionMapMapped.h | 240 ------------------------------ 1 file changed, 240 deletions(-) delete mode 100644 src/Functions/FunctionMapMapped.h diff --git a/src/Functions/FunctionMapMapped.h b/src/Functions/FunctionMapMapped.h deleted file mode 100644 index ff746b58f71..00000000000 --- a/src/Functions/FunctionMapMapped.h +++ /dev/null @@ -1,240 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -} - - -// TODO 只要map没有实现去重,那么这个函数就不能用了。先等着吧 -/** Higher-order functions for maps. - * These functions optionally apply a map (transform) to map(or multiple map of identical size) by lambda function, - * and return some result based on that transformation. - * - * Examples: - * arrayMap(x1,...,xn -> expression, array1,...,arrayn) - apply the expression to each element of the array (or set of parallel arrays). - * arrayFilter(x -> predicate, array) - leave in the array only the elements for which the expression is true. - * - * For some functions arrayCount, arrayExists, arrayAll, an overload of the form f(array) is available, - * which works in the same way as f(x -> x, array). - * - * See the example of Impl template parameter in arrayMap.cpp - */ -template -class FunctionArrayMapped : public IFunction -{ -public: - static constexpr auto name = Name::name; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return true; } - size_t getNumberOfArguments() const override { return 0; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - - /// Called if at least one function argument is a lambda expression. - /// For argument-lambda expressions, it defines the types of arguments of these expressions. - void getLambdaArgumentTypes(DataTypes & arguments) const override - { - if (arguments.empty()) - throw Exception("Function " + getName() + " needs at least one argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (arguments.size() == 1) - throw Exception("Function " + getName() + " needs at least one array argument.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - DataTypes nested_types(arguments.size() - 1); - for (size_t i = 0; i < nested_types.size(); ++i) - { - const DataTypeArray * array_type = checkAndGetDataType(&*arguments[i + 1]); - if (!array_type) - throw Exception("Argument " + toString(i + 2) + " of function " + getName() + " must be array. Found " - + arguments[i + 1]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - nested_types[i] = recursiveRemoveLowCardinality(array_type->getNestedType()); - } - - const DataTypeFunction * function_type = checkAndGetDataType(arguments[0].get()); - if (!function_type || function_type->getArgumentTypes().size() != nested_types.size()) - throw Exception("First argument for this overload of " + getName() + " must be a function with " - + toString(nested_types.size()) + " arguments. Found " - + arguments[0]->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - arguments[0] = std::make_shared(nested_types); - } - - DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override - { - size_t min_args = Impl::needExpression() ? 2 : 1; - if (arguments.size() < min_args) - throw Exception("Function " + getName() + " needs at least " - + toString(min_args) + " argument; passed " - + toString(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (arguments.size() == 1) - { - const auto * array_type = checkAndGetDataType(arguments[0].type.get()); - - if (!array_type) - throw Exception("The only argument for function " + getName() + " must be array. Found " - + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - DataTypePtr nested_type = array_type->getNestedType(); - - if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8()) - throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found " - + arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return Impl::getReturnType(nested_type, nested_type); - } - else - { - if (arguments.size() > 2 && Impl::needOneArray()) - throw Exception("Function " + getName() + " needs one array argument.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const auto * data_type_function = checkAndGetDataType(arguments[0].type.get()); - - if (!data_type_function) - throw Exception("First argument for function " + getName() + " must be a function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - /// The types of the remaining arguments are already checked in getLambdaArgumentTypes. - - DataTypePtr return_type = removeLowCardinality(data_type_function->getReturnType()); - if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8()) - throw Exception("Expression for function " + getName() + " must return UInt8, found " - + return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * first_array_type = checkAndGetDataType(arguments[1].type.get()); - - return Impl::getReturnType(return_type, first_array_type->getNestedType()); - } - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override - { - if (arguments.size() == 1) - { - ColumnPtr column_array_ptr = arguments[0].column; - const auto * column_array = checkAndGetColumn(column_array_ptr.get()); - - if (!column_array) - { - const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); - if (!column_const_array) - throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - column_array_ptr = column_const_array->convertToFullColumn(); - column_array = assert_cast(column_array_ptr.get()); - } - - return Impl::execute(*column_array, column_array->getDataPtr()); - } - else - { - const auto & column_with_type_and_name = arguments[0]; - - if (!column_with_type_and_name.column) - throw Exception("First argument for function " + getName() + " must be a function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); - - if (!column_function) - throw Exception("First argument for function " + getName() + " must be a function.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - ColumnPtr offsets_column; - - ColumnPtr column_first_array_ptr; - const ColumnArray * column_first_array = nullptr; - - ColumnsWithTypeAndName arrays; - arrays.reserve(arguments.size() - 1); - - for (size_t i = 1; i < arguments.size(); ++i) - { - const auto & array_with_type_and_name = arguments[i]; - - ColumnPtr column_array_ptr = array_with_type_and_name.column; - const auto * column_array = checkAndGetColumn(column_array_ptr.get()); - - const DataTypePtr & array_type_ptr = array_with_type_and_name.type; - const auto * array_type = checkAndGetDataType(array_type_ptr.get()); - - if (!column_array) - { - const ColumnConst * column_const_array = checkAndGetColumnConst(column_array_ptr.get()); - if (!column_const_array) - throw Exception("Expected array column, found " + column_array_ptr->getName(), ErrorCodes::ILLEGAL_COLUMN); - column_array_ptr = recursiveRemoveLowCardinality(column_const_array->convertToFullColumn()); - column_array = checkAndGetColumn(column_array_ptr.get()); - } - - if (!array_type) - throw Exception("Expected array type, found " + array_type_ptr->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!offsets_column) - { - offsets_column = column_array->getOffsetsPtr(); - } - else - { - /// The first condition is optimization: do not compare data if the pointers are equal. - if (column_array->getOffsetsPtr() != offsets_column - && column_array->getOffsets() != typeid_cast(*offsets_column).getData()) - throw Exception("Arrays passed to " + getName() + " must have equal size", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); - } - - if (i == 1) - { - column_first_array_ptr = column_array_ptr; - column_first_array = column_array; - } - - arrays.emplace_back(ColumnWithTypeAndName(column_array->getDataPtr(), - recursiveRemoveLowCardinality(array_type->getNestedType()), - array_with_type_and_name.name)); - } - - /// Put all the necessary columns multiplied by the sizes of arrays into the columns. - auto replicated_column_function_ptr = IColumn::mutate(column_function->replicate(column_first_array->getOffsets())); - auto * replicated_column_function = typeid_cast(replicated_column_function_ptr.get()); - replicated_column_function->appendArguments(arrays); - - auto lambda_result = replicated_column_function->reduce().column; - if (lambda_result->lowCardinality()) - lambda_result = lambda_result->convertToFullColumnIfLowCardinality(); - - return Impl::execute(*column_first_array, lambda_result); - } - } -}; - -} From 445b0983eab355de77d376653618f18514f45e73 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 01:32:06 +0300 Subject: [PATCH 515/609] stress: ignore server memory limits for hung check In [1] hung check failed because the server was under memory pressure. [1]: https://s3.amazonaws.com/clickhouse-test-reports/32019/b6290ae00223fc91d514b82de25e195ab34f8bca/stress_test__undefined__actions_.html --- docker/test/stress/stress | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index acb45b05636..3bb518131c8 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -188,7 +188,13 @@ if __name__ == "__main__": if args.hung_check: have_long_running_queries = prepare_for_hung_check(args.drop_databases) logging.info("Checking if some queries hung") - cmd = "{} {} {}".format(args.test_cmd, "--hung-check", "00001_select_1") + cmd = ' '.join([args.test_cmd, + # Do not track memory allocations up to 100MiB, + # this will allow to ignore server memory limit (max_server_memory_usage) for this query. + "--client-option", "max_untracked_memory=100Mi", + "--hung-check", + "00001_select_1" + ]) res = call(cmd, shell=True, stderr=STDOUT) hung_check_status = "No queries hung\tOK\n" if res != 0 and have_long_running_queries: From 169941c5d07f643df00d23262ff1e79be0e57845 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 01:32:06 +0300 Subject: [PATCH 516/609] stress: allow 100MiB of memory to overcommit for SHOW/DROP DATABASE CI: https://s3.amazonaws.com/clickhouse-test-reports/32019/b6290ae00223fc91d514b82de25e195ab34f8bca/stress_test__thread__actions_.html --- docker/test/stress/stress | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 3bb518131c8..b5aba692a93 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -117,11 +117,14 @@ def prepare_for_hung_check(drop_databases): try: # Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too. # Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds). - databases = check_output('clickhouse client -q "SHOW DATABASES"', shell=True, timeout=30).decode('utf-8').strip().split() + # + # Also specify max_untracked_memory to allow 100MiB of memory to overcommit. + databases = check_output('clickhouse client -q "SHOW DATABASES" --max_untracked_memory=100Mi', + shell=True, timeout=30).decode('utf-8').strip().split() for db in databases: if db == "system": continue - command = f'clickhouse client -q "DROP DATABASE {db}"' + command = f'clickhouse client -q "DROP DATABASE {db}" --max_untracked_memory=100Mi' # we don't wait for drop Popen(command, shell=True) break From d3a7aed0054d1ef5d6615317302a6f1717ba7d19 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 21:27:40 +0300 Subject: [PATCH 517/609] stress: tune memory_profiler_step too --- docker/test/stress/stress | 20 +++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index b5aba692a93..f9259891b57 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -118,13 +118,13 @@ def prepare_for_hung_check(drop_databases): # Here we try to drop all databases in async mode. If some queries really hung, than drop will hung too. # Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds). # - # Also specify max_untracked_memory to allow 100MiB of memory to overcommit. - databases = check_output('clickhouse client -q "SHOW DATABASES" --max_untracked_memory=100Mi', + # Also specify max_untracked_memory to allow 1GiB of memory to overcommit. + databases = check_output('clickhouse client -q "SHOW DATABASES" --max_untracked_memory=1Gi --memory_profiler_step=1Gi', shell=True, timeout=30).decode('utf-8').strip().split() for db in databases: if db == "system": continue - command = f'clickhouse client -q "DROP DATABASE {db}" --max_untracked_memory=100Mi' + command = f'clickhouse client -q "DROP DATABASE {db}" --max_untracked_memory=1Gi --memory_profiler_step=1Gi' # we don't wait for drop Popen(command, shell=True) break @@ -192,9 +192,19 @@ if __name__ == "__main__": have_long_running_queries = prepare_for_hung_check(args.drop_databases) logging.info("Checking if some queries hung") cmd = ' '.join([args.test_cmd, - # Do not track memory allocations up to 100MiB, + # Do not track memory allocations up to 1Gi, # this will allow to ignore server memory limit (max_server_memory_usage) for this query. - "--client-option", "max_untracked_memory=100Mi", + # + # NOTE: memory_profiler_step should be also adjusted, because: + # + # untracked_memory_limit = min(settings.max_untracked_memory, settings.memory_profiler_step) + # + # NOTE: that if there will be queries with GROUP BY, this trick + # will not work due to CurrentMemoryTracker::check() from + # Aggregator code. + # But right now it should work, since neither hung check, nor 00001_select_1 has GROUP BY. + "--client-option", "max_untracked_memory=1Gi", + "--client-option", "memory_profiler_step=1Gi", "--hung-check", "00001_select_1" ]) From adec69016653b96029afabdc4085367540af19ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 21:31:58 +0300 Subject: [PATCH 518/609] stress: SYSTEM DROP MARK CACHE --- docker/test/stress/stress | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index f9259891b57..85374d5efed 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -102,6 +102,7 @@ def prepare_for_hung_check(drop_databases): call_with_retry("clickhouse client -q 'SYSTEM START FETCHES'") call_with_retry("clickhouse client -q 'SYSTEM START REPLICATED SENDS'") call_with_retry("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'") + call_with_retry("clickhouse client -q 'SYSTEM DROP MARK CACHE'") # Issue #21004, live views are experimental, so let's just suppress it call_with_retry("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """) From 7063ea3ee2308e51367033048ee5d0b3436fe1de Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 21:36:22 +0300 Subject: [PATCH 519/609] stress: add make_query_command() helper --- docker/test/stress/stress | 30 ++++++++++++++++-------------- 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 85374d5efed..ddb309c7aee 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -75,6 +75,9 @@ def call_with_retry(query, timeout=30, retry_count=5): else: break +def make_query_command(query): + return f"""clickhouse client -q "{query}" --max_untracked_memory=1Gi --memory_profiler_step=1Gi""" + def prepare_for_hung_check(drop_databases): # FIXME this function should not exist, but... @@ -95,23 +98,23 @@ def prepare_for_hung_check(drop_databases): # Some tests execute SYSTEM STOP MERGES or similar queries. # It may cause some ALTERs to hang. # Possibly we should fix tests and forbid to use such queries without specifying table. - call_with_retry("clickhouse client -q 'SYSTEM START MERGES'") - call_with_retry("clickhouse client -q 'SYSTEM START DISTRIBUTED SENDS'") - call_with_retry("clickhouse client -q 'SYSTEM START TTL MERGES'") - call_with_retry("clickhouse client -q 'SYSTEM START MOVES'") - call_with_retry("clickhouse client -q 'SYSTEM START FETCHES'") - call_with_retry("clickhouse client -q 'SYSTEM START REPLICATED SENDS'") - call_with_retry("clickhouse client -q 'SYSTEM START REPLICATION QUEUES'") - call_with_retry("clickhouse client -q 'SYSTEM DROP MARK CACHE'") + call_with_retry(make_query_command('SYSTEM START MERGES')) + call_with_retry(make_query_command('SYSTEM START DISTRIBUTED SENDS')) + call_with_retry(make_query_command('SYSTEM START TTL MERGES')) + call_with_retry(make_query_command('SYSTEM START MOVES')) + call_with_retry(make_query_command('SYSTEM START FETCHES')) + call_with_retry(make_query_command('SYSTEM START REPLICATED SENDS')) + call_with_retry(make_query_command('SYSTEM START REPLICATION QUEUES')) + call_with_retry(make_query_command('SYSTEM DROP MARK CACHE')) # Issue #21004, live views are experimental, so let's just suppress it - call_with_retry("""clickhouse client -q "KILL QUERY WHERE upper(query) LIKE 'WATCH %'" """) + call_with_retry(make_query_command("KILL QUERY WHERE upper(query) LIKE 'WATCH %'")) # Kill other queries which known to be slow # It's query from 01232_preparing_sets_race_condition_long, it may take up to 1000 seconds in slow builds - call_with_retry("""clickhouse client -q "KILL QUERY WHERE query LIKE 'insert into tableB select %'" """) + call_with_retry(make_query_command("KILL QUERY WHERE query LIKE 'insert into tableB select %'")) # Long query from 00084_external_agregation - call_with_retry("""clickhouse client -q "KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'" """) + call_with_retry(make_query_command("KILL QUERY WHERE query LIKE 'SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u %'")) if drop_databases: for i in range(5): @@ -120,12 +123,11 @@ def prepare_for_hung_check(drop_databases): # Otherwise we will get rid of queries which wait for background pool. It can take a long time on slow builds (more than 900 seconds). # # Also specify max_untracked_memory to allow 1GiB of memory to overcommit. - databases = check_output('clickhouse client -q "SHOW DATABASES" --max_untracked_memory=1Gi --memory_profiler_step=1Gi', - shell=True, timeout=30).decode('utf-8').strip().split() + databases = check_output(make_query_command('SHOW DATABASES'), shell=True, timeout=30).decode('utf-8').strip().split() for db in databases: if db == "system": continue - command = f'clickhouse client -q "DROP DATABASE {db}" --max_untracked_memory=1Gi --memory_profiler_step=1Gi' + command = make_query_command(f'DROP DATABASE {db}') # we don't wait for drop Popen(command, shell=True) break From 116731396137cad5988236ad8cc36faf05b74d51 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 3 Dec 2021 23:07:16 +0300 Subject: [PATCH 520/609] stress: allow memory overcommit for one more query --- docker/test/stress/stress | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index ddb309c7aee..d9b0d9a0dc7 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -139,9 +139,15 @@ def prepare_for_hung_check(drop_databases): # Wait for last queries to finish if any, not longer than 300 seconds - call("""clickhouse client -q "select sleepEachRow(( - select maxOrDefault(300 - elapsed) + 1 from system.processes where query not like '%from system.processes%' and elapsed < 300 - ) / 300) from numbers(300) format Null" """, shell=True, stderr=STDOUT, timeout=330) + call(make_query_command(""" + select sleepEachRow(( + select maxOrDefault(300 - elapsed) + 1 + from system.processes + where query not like '%from system.processes%' and elapsed < 300 + ) / 300) + from numbers(300) + format Null + """), shell=True, stderr=STDOUT, timeout=330) # Even if all clickhouse-test processes are finished, there are probably some sh scripts, # which still run some new queries. Let's ignore them. From f9a95fce131ba7a5330843163f33a00f8e86ffed Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 4 Dec 2021 00:14:00 +0300 Subject: [PATCH 521/609] stress: allow memory overcommit for SELECT 1 query --- docker/test/stress/stress | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docker/test/stress/stress b/docker/test/stress/stress index d9b0d9a0dc7..62a42cdf548 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -91,9 +91,7 @@ def prepare_for_hung_check(drop_databases): logging.info("Will terminate gdb (if any)") call_with_retry("kill -TERM $(pidof gdb)") - # Some tests set too low memory limit for default user and forget to reset in back. - # It may cause SYSTEM queries to fail, let's disable memory limit. - call_with_retry("clickhouse client --max_memory_usage_for_user=0 -q 'SELECT 1 FORMAT Null'") + call_with_retry(make_query_command('SELECT 1 FORMAT Null')) # Some tests execute SYSTEM STOP MERGES or similar queries. # It may cause some ALTERs to hang. From d68d01988ec3d156f77ccd67470c27a69d7fc215 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 4 Dec 2021 21:56:15 +0300 Subject: [PATCH 522/609] clickhouse-test: apply --client-option in get_stacktraces_from_clickhouse() --- tests/clickhouse-test | 20 ++++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 01d632a1f50..8a87227519f 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -205,26 +205,31 @@ def get_stacktraces_from_gdb(server_pid): # collect server stacktraces from system.stack_trace table # it does not work in Sandbox -def get_stacktraces_from_clickhouse(client, replicated_database=False): +def get_stacktraces_from_clickhouse(args): + settings_str = ' '.join([ + get_additional_client_options(args), + '--allow_introspection_functions=1', + '--skip_unavailable_shards=1', + ]) replicated_msg = \ - "{} --allow_introspection_functions=1 --skip_unavailable_shards=1 --query \ + "{} {} --query \ \"SELECT materialize((hostName(), tcpPort())) as host, thread_id, \ arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \ arrayMap(x -> addressToLine(x), trace), \ arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \ FROM clusterAllReplicas('test_cluster_database_replicated', 'system.stack_trace') \ - ORDER BY host, thread_id FORMAT Vertical\"".format(client) + ORDER BY host, thread_id FORMAT Vertical\"".format(args.client, settings_str) msg = \ - "{} --allow_introspection_functions=1 --query \ + "{} {} --query \ \"SELECT arrayStringConcat(arrayMap(x, y -> concat(x, ': ', y), \ arrayMap(x -> addressToLine(x), trace), \ arrayMap(x -> demangle(addressToSymbol(x)), trace)), '\n') as trace \ - FROM system.stack_trace FORMAT Vertical\"".format(client) + FROM system.stack_trace FORMAT Vertical\"".format(args.client, settings_str) try: return subprocess.check_output( - replicated_msg if replicated_database else msg, + replicated_msg if args.replicated_database else msg, shell=True, stderr=subprocess.STDOUT).decode('utf-8') except Exception as e: print(f"Error occurred while receiving stack traces from client: {e}") @@ -250,8 +255,7 @@ def print_stacktraces() -> None: if bt is None: print("\nCollecting stacktraces from system.stacktraces table:") - bt = get_stacktraces_from_clickhouse( - args.client, args.replicated_database) + bt = get_stacktraces_from_clickhouse(args) if bt is not None: print(bt) From f8bf3b19932752e985e59f03fe2bdf52b280e0ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 6 Dec 2021 09:05:34 +0300 Subject: [PATCH 523/609] stress: use max_memory_usage_for_user as a soft limit to avoid memory limit exceeded By using max_memory_usage_for_user as a soft limit, and after max_server_memory_usage as a hard, we can allow normal overcommit, using max_memory_usage_for_user=0 instead of relying on max_untracked_memory. --- docker/test/stress/run.sh | 36 ++++++++++++++++++++++++++++++++++-- docker/test/stress/stress | 3 ++- 2 files changed, 36 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 673e4c11570..6d720d02cdc 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -55,9 +55,41 @@ function configure() echo "1" \ > /etc/clickhouse-server/config.d/asynchronous_metrics_update_period_s.xml + local total_mem + total_mem=$(awk '/MemTotal/ { print $(NF-1) }' /proc/meminfo) # KiB + total_mem=$(( total_mem*1024 )) # bytes # Set maximum memory usage as half of total memory (less chance of OOM). - echo "0.5" \ - > /etc/clickhouse-server/config.d/max_server_memory_usage_to_ram_ratio.xml + # + # But not via max_server_memory_usage but via max_memory_usage_for_user, + # so that we can override this setting and execute service queries, like: + # - hung check + # - show/drop database + # - ... + # + # So max_memory_usage_for_user will be a soft limit, and + # max_server_memory_usage will be hard limit, and queries that should be + # executed regardless memory limits will use max_memory_usage_for_user=0, + # instead of relying on max_untracked_memory + local max_server_mem + max_server_mem=$((total_mem*75/100)) # 75% + echo "Setting max_server_memory_usage=$max_server_mem" + cat > /etc/clickhouse-server/config.d/max_server_memory_usage.xml < + ${max_server_mem} + +EOL + local max_users_mem + max_users_mem=$((total_mem*50/100)) # 50% + echo "Setting max_memory_usage_for_user=$max_users_mem" + cat > /etc/clickhouse-server/users.d/max_memory_usage_for_user.xml < + + + ${max_users_mem} + + + +EOL } function stop() diff --git a/docker/test/stress/stress b/docker/test/stress/stress index 62a42cdf548..c89c5ff5e27 100755 --- a/docker/test/stress/stress +++ b/docker/test/stress/stress @@ -76,7 +76,7 @@ def call_with_retry(query, timeout=30, retry_count=5): break def make_query_command(query): - return f"""clickhouse client -q "{query}" --max_untracked_memory=1Gi --memory_profiler_step=1Gi""" + return f"""clickhouse client -q "{query}" --max_untracked_memory=1Gi --memory_profiler_step=1Gi --max_memory_usage_for_user=0""" def prepare_for_hung_check(drop_databases): @@ -211,6 +211,7 @@ if __name__ == "__main__": # Aggregator code. # But right now it should work, since neither hung check, nor 00001_select_1 has GROUP BY. "--client-option", "max_untracked_memory=1Gi", + "--client-option", "max_memory_usage_for_user=0", "--client-option", "memory_profiler_step=1Gi", "--hung-check", "00001_select_1" From ef77f45f599713ca662ebae4279829522043c8a7 Mon Sep 17 00:00:00 2001 From: sunlisheng Date: Wed, 1 Dec 2021 20:11:07 +0800 Subject: [PATCH 524/609] Make HDFS replication configurable in WriteBufferFromHDFSImpl#WriteBufferFromHDFSImpl Signed-off-by: sunlisheng --- src/Disks/HDFS/DiskHDFS.cpp | 5 +++-- src/Disks/HDFS/DiskHDFS.h | 7 +++++-- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 6 ++++-- src/Storages/HDFS/WriteBufferFromHDFS.h | 1 + 5 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 5264e6413e7..bae2a57bc67 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -97,7 +97,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path /// Single O_WRONLY in libhdfs adds O_TRUNC auto hdfs_buffer = std::make_unique(hdfs_path, - config, buf_size, + config, settings->replication, buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); return std::make_unique>(std::move(hdfs_buffer), @@ -147,7 +147,8 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigur return std::make_unique( config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".thread_pool_size", 16), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); + config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), + config.getInt(config_prefix + ".dfs.replication", 3)); } } diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 881d6e2937c..47150f1cfd8 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -14,14 +14,17 @@ struct DiskHDFSSettings size_t min_bytes_for_seek; int thread_pool_size; int objects_chunk_size_to_delete; + int replication; DiskHDFSSettings( int min_bytes_for_seek_, int thread_pool_size_, - int objects_chunk_size_to_delete_) + int objects_chunk_size_to_delete_, + int replication_) : min_bytes_for_seek(min_bytes_for_seek_) , thread_pool_size(thread_pool_size_) - , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {} + , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) + , replication(replication_) {} }; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 90e63aef46d..54351deffd8 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -203,7 +203,7 @@ public: const CompressionMethod compression_method) : SinkToStorage(sample_block) { - write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef()), compression_method, 3); + write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), 0), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index d6a10452f97..9f5e3c1f7d2 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -29,6 +29,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl explicit WriteBufferFromHDFSImpl( const std::string & hdfs_uri_, const Poco::Util::AbstractConfiguration & config_, + int replication_, int flags) : hdfs_uri(hdfs_uri_) , builder(createHDFSBuilder(hdfs_uri, config_)) @@ -43,7 +44,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl if (!hdfsExists(fs.get(), path.c_str())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} already exists", path); - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here if (fout == nullptr) { @@ -82,10 +83,11 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl WriteBufferFromHDFS::WriteBufferFromHDFS( const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, + int replication_, size_t buf_size_, int flags_) : BufferWithOwnMemory(buf_size_) - , impl(std::make_unique(hdfs_name_, config_, flags_)) + , impl(std::make_unique(hdfs_name_, config_, replication_, flags_)) { } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index 503371f6118..fe9af7dfba4 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -23,6 +23,7 @@ public: WriteBufferFromHDFS( const String & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, + int replication_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags = O_WRONLY); From 0f045fa20ad60bd19942f2e9dd8ff6798847630d Mon Sep 17 00:00:00 2001 From: sunlisheng Date: Wed, 1 Dec 2021 20:30:59 +0800 Subject: [PATCH 525/609] rename replication Signed-off-by: sunlisheng --- src/Disks/HDFS/DiskHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index bae2a57bc67..5194983100b 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -148,7 +148,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigur config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".thread_pool_size", 16), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), - config.getInt(config_prefix + ".dfs.replication", 3)); + config.getInt(config_prefix + ".replication", 3)); } } From 5be58a88398612587882542864aafeec191acee1 Mon Sep 17 00:00:00 2001 From: leosunli Date: Thu, 2 Dec 2021 11:40:22 +0800 Subject: [PATCH 526/609] Make HDFS replication configurable in WriteBufferFromHDFSImpl#WriteBufferFromHDFSImpl Signed-off-by: leosunli --- src/Core/Settings.h | 1 + src/Disks/HDFS/DiskHDFS.cpp | 11 ++++++----- src/Disks/HDFS/DiskHDFS.h | 11 +++++------ src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 8 ++++---- src/Storages/HDFS/WriteBufferFromHDFS.h | 3 ++- 6 files changed, 19 insertions(+), 17 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bcbe45dd002..4d014930195 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -75,6 +75,7 @@ class IColumn; M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ + M(UInt64, hdfs_replication, 3, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 5194983100b..e92bcf2ec16 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -60,9 +60,11 @@ DiskHDFS::DiskHDFS( const String & hdfs_root_path_, SettingsPtr settings_, DiskPtr metadata_disk_, - const Poco::Util::AbstractConfiguration & config_) + const Poco::Util::AbstractConfiguration & config_, + const Settings & contextSettings_) : IDiskRemote(disk_name_, hdfs_root_path_, metadata_disk_, "DiskHDFS", settings_->thread_pool_size) , config(config_) + , contextSettings(contextSettings_) , hdfs_builder(createHDFSBuilder(hdfs_root_path_, config)) , hdfs_fs(createHDFSFS(hdfs_builder.get())) , settings(std::move(settings_)) @@ -97,7 +99,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path /// Single O_WRONLY in libhdfs adds O_TRUNC auto hdfs_buffer = std::make_unique(hdfs_path, - config, settings->replication, buf_size, + config, contextSettings, buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); return std::make_unique>(std::move(hdfs_buffer), @@ -147,8 +149,7 @@ std::unique_ptr getSettings(const Poco::Util::AbstractConfigur return std::make_unique( config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".thread_pool_size", 16), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), - config.getInt(config_prefix + ".replication", 3)); + config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); } } @@ -175,7 +176,7 @@ void registerDiskHDFS(DiskFactory & factory) return std::make_shared( name, uri, getSettings(config, config_prefix), - metadata_disk, config); + metadata_disk, config, context.getSettingsRef()); }; factory.registerDiskType("hdfs", creator); diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 47150f1cfd8..0c67983722a 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -14,17 +14,14 @@ struct DiskHDFSSettings size_t min_bytes_for_seek; int thread_pool_size; int objects_chunk_size_to_delete; - int replication; DiskHDFSSettings( int min_bytes_for_seek_, int thread_pool_size_, - int objects_chunk_size_to_delete_, - int replication_) + int objects_chunk_size_to_delete_) : min_bytes_for_seek(min_bytes_for_seek_) , thread_pool_size(thread_pool_size_) - , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) - , replication(replication_) {} + , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {} }; @@ -43,7 +40,8 @@ public: const String & hdfs_root_path_, SettingsPtr settings_, DiskPtr metadata_disk_, - const Poco::Util::AbstractConfiguration & config_); + const Poco::Util::AbstractConfiguration & config_, + const Settings & contextSettings_); DiskType getType() const override { return DiskType::HDFS; } bool isRemote() const override { return true; } @@ -70,6 +68,7 @@ private: String getRandomName() { return toString(UUIDHelpers::generateV4()); } const Poco::Util::AbstractConfiguration & config; + const Settings & contextSettings; HDFSBuilderWrapper hdfs_builder; HDFSFSPtr hdfs_fs; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 54351deffd8..5d1b98892e9 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -203,7 +203,7 @@ public: const CompressionMethod compression_method) : SinkToStorage(sample_block) { - write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), 0), compression_method, 3); + write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), context.getSettingsRef()), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 9f5e3c1f7d2..a84849359f8 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -29,7 +29,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl explicit WriteBufferFromHDFSImpl( const std::string & hdfs_uri_, const Poco::Util::AbstractConfiguration & config_, - int replication_, + const Settings & settings_, int flags) : hdfs_uri(hdfs_uri_) , builder(createHDFSBuilder(hdfs_uri, config_)) @@ -44,7 +44,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl if (!hdfsExists(fs.get(), path.c_str())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} already exists", path); - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, settings_.hdfs_replication, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here if (fout == nullptr) { @@ -83,11 +83,11 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl WriteBufferFromHDFS::WriteBufferFromHDFS( const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, - int replication_, + const Settings & settings_, size_t buf_size_, int flags_) : BufferWithOwnMemory(buf_size_) - , impl(std::make_unique(hdfs_name_, config_, replication_, flags_)) + , impl(std::make_unique(hdfs_name_, config_, settings_, flags_)) { } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index fe9af7dfba4..752ea6659ef 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -23,7 +24,7 @@ public: WriteBufferFromHDFS( const String & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, - int replication_, + const Settings & settings_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags = O_WRONLY); From e81a1dbe6a1fcb9684d3eed7cbd680467c4b91d8 Mon Sep 17 00:00:00 2001 From: leosunli Date: Thu, 2 Dec 2021 15:28:25 +0800 Subject: [PATCH 527/609] Make HDFS replication configurable in WriteBufferFromHDFSImpl#WriteBufferFromHDFSImpl Signed-off-by: leosunli --- src/Disks/HDFS/DiskHDFS.cpp | 11 +++++------ src/Disks/HDFS/DiskHDFS.h | 11 ++++++----- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 8 ++++---- src/Storages/HDFS/WriteBufferFromHDFS.h | 2 +- 5 files changed, 17 insertions(+), 17 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index e92bcf2ec16..befd4ed0f5f 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -60,11 +60,9 @@ DiskHDFS::DiskHDFS( const String & hdfs_root_path_, SettingsPtr settings_, DiskPtr metadata_disk_, - const Poco::Util::AbstractConfiguration & config_, - const Settings & contextSettings_) + const Poco::Util::AbstractConfiguration & config_) : IDiskRemote(disk_name_, hdfs_root_path_, metadata_disk_, "DiskHDFS", settings_->thread_pool_size) , config(config_) - , contextSettings(contextSettings_) , hdfs_builder(createHDFSBuilder(hdfs_root_path_, config)) , hdfs_fs(createHDFSFS(hdfs_builder.get())) , settings(std::move(settings_)) @@ -99,7 +97,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path /// Single O_WRONLY in libhdfs adds O_TRUNC auto hdfs_buffer = std::make_unique(hdfs_path, - config, contextSettings, buf_size, + config, settings->replication, buf_size, mode == WriteMode::Rewrite ? O_WRONLY : O_WRONLY | O_APPEND); return std::make_unique>(std::move(hdfs_buffer), @@ -144,12 +142,13 @@ bool DiskHDFS::checkUniqueId(const String & hdfs_uri) const namespace { -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix) +std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & setttings) { return std::make_unique( config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".thread_pool_size", 16), - config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); + config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), + setttings.hdfs_replication); } } diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 0c67983722a..47150f1cfd8 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -14,14 +14,17 @@ struct DiskHDFSSettings size_t min_bytes_for_seek; int thread_pool_size; int objects_chunk_size_to_delete; + int replication; DiskHDFSSettings( int min_bytes_for_seek_, int thread_pool_size_, - int objects_chunk_size_to_delete_) + int objects_chunk_size_to_delete_, + int replication_) : min_bytes_for_seek(min_bytes_for_seek_) , thread_pool_size(thread_pool_size_) - , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {} + , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) + , replication(replication_) {} }; @@ -40,8 +43,7 @@ public: const String & hdfs_root_path_, SettingsPtr settings_, DiskPtr metadata_disk_, - const Poco::Util::AbstractConfiguration & config_, - const Settings & contextSettings_); + const Poco::Util::AbstractConfiguration & config_); DiskType getType() const override { return DiskType::HDFS; } bool isRemote() const override { return true; } @@ -68,7 +70,6 @@ private: String getRandomName() { return toString(UUIDHelpers::generateV4()); } const Poco::Util::AbstractConfiguration & config; - const Settings & contextSettings; HDFSBuilderWrapper hdfs_builder; HDFSFSPtr hdfs_fs; diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 5d1b98892e9..6b03a2490ae 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -203,7 +203,7 @@ public: const CompressionMethod compression_method) : SinkToStorage(sample_block) { - write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), context.getSettingsRef()), compression_method, 3); + write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), context.getSettingsRef().hdfs_replication), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index a84849359f8..9f5e3c1f7d2 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -29,7 +29,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl explicit WriteBufferFromHDFSImpl( const std::string & hdfs_uri_, const Poco::Util::AbstractConfiguration & config_, - const Settings & settings_, + int replication_, int flags) : hdfs_uri(hdfs_uri_) , builder(createHDFSBuilder(hdfs_uri, config_)) @@ -44,7 +44,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl if (!hdfsExists(fs.get(), path.c_str())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} already exists", path); - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, settings_.hdfs_replication, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here + fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here if (fout == nullptr) { @@ -83,11 +83,11 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl WriteBufferFromHDFS::WriteBufferFromHDFS( const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, - const Settings & settings_, + int replication_, size_t buf_size_, int flags_) : BufferWithOwnMemory(buf_size_) - , impl(std::make_unique(hdfs_name_, config_, settings_, flags_)) + , impl(std::make_unique(hdfs_name_, config_, replication_, flags_)) { } diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index 752ea6659ef..32425ee3394 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -24,7 +24,7 @@ public: WriteBufferFromHDFS( const String & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, - const Settings & settings_, + int replication_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags = O_WRONLY); From 8d044958405ab1d9f6da011b9a233eb36df10263 Mon Sep 17 00:00:00 2001 From: leosunli Date: Thu, 2 Dec 2021 16:31:47 +0800 Subject: [PATCH 528/609] set hdfs_replication default value 0 Signed-off-by: leosunli --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4d014930195..4e0e50cc521 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -75,7 +75,7 @@ class IColumn; M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ - M(UInt64, hdfs_replication, 3, "The actual number of replications can be specified when the hdfs file is created.", 0) \ + M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ From 4017bb8f78b5e664ea1b33184e3181ac1cb0c5a7 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 2 Dec 2021 23:01:32 +0300 Subject: [PATCH 529/609] Update WriteBufferFromHDFS.h --- src/Storages/HDFS/WriteBufferFromHDFS.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index 32425ee3394..fe9af7dfba4 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include From 6f611eec91d38cbe1d45cdc52d5a3fb651419db2 Mon Sep 17 00:00:00 2001 From: leosunli Date: Fri, 3 Dec 2021 16:01:14 +0800 Subject: [PATCH 530/609] Fix bug Signed-off-by: leosunli --- src/Disks/HDFS/DiskHDFS.cpp | 8 ++++---- src/Storages/HDFS/StorageHDFS.cpp | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index befd4ed0f5f..21c8c965361 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -142,13 +142,13 @@ bool DiskHDFS::checkUniqueId(const String & hdfs_uri) const namespace { -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & setttings) +std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Settings & settings) { return std::make_unique( config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".thread_pool_size", 16), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000), - setttings.hdfs_replication); + settings.hdfs_replication); } } @@ -174,8 +174,8 @@ void registerDiskHDFS(DiskFactory & factory) return std::make_shared( name, uri, - getSettings(config, config_prefix), - metadata_disk, config, context.getSettingsRef()); + getSettings(config, config_prefix, context_.getSettingsRef()), + metadata_disk, config); }; factory.registerDiskType("hdfs", creator); diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 6b03a2490ae..f8b29f5499d 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -203,7 +203,7 @@ public: const CompressionMethod compression_method) : SinkToStorage(sample_block) { - write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), context.getSettingsRef().hdfs_replication), compression_method, 3); + write_buf = wrapWriteBufferWithCompressionMethod(std::make_unique(uri, context->getGlobalContext()->getConfigRef(), context->getSettingsRef().hdfs_replication), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context); } From 3946b156a1e758081bdb609691eb0c3d2764b3f9 Mon Sep 17 00:00:00 2001 From: leosunli Date: Mon, 6 Dec 2021 14:31:55 +0800 Subject: [PATCH 531/609] Fix bug Signed-off-by: leosunli --- src/Disks/HDFS/DiskHDFS.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 21c8c965361..c116a62a977 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -174,7 +174,7 @@ void registerDiskHDFS(DiskFactory & factory) return std::make_shared( name, uri, - getSettings(config, config_prefix, context_.getSettingsRef()), + getSettings(config, config_prefix, context_->getSettingsRef()), metadata_disk, config); }; From b68136d1e4e734fc9b449c17fb68811f7764e13a Mon Sep 17 00:00:00 2001 From: vxider Date: Mon, 6 Dec 2021 07:12:21 +0000 Subject: [PATCH 532/609] small code style update --- src/Storages/WindowView/StorageWindowView.cpp | 23 ++++++++++--------- src/Storages/WindowView/StorageWindowView.h | 2 +- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 94f88842cbb..915e775ff14 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -307,10 +307,12 @@ namespace } } -static void extractDependentTable(ContextPtr context, ASTSelectQuery & query, String & select_database_name, String & select_table_name) +static void extractDependentTable(ContextPtr context, ASTPtr & query, String & select_database_name, String & select_table_name) { - auto db_and_table = getDatabaseAndTable(query, 0); - ASTPtr subquery = extractTableExpression(query, 0); + ASTSelectQuery & select_query = typeid_cast(*query); + + auto db_and_table = getDatabaseAndTable(select_query, 0); + ASTPtr subquery = extractTableExpression(select_query, 0); if (!db_and_table && !subquery) return; @@ -323,7 +325,7 @@ static void extractDependentTable(ContextPtr context, ASTSelectQuery & query, St { db_and_table->database = select_database_name; AddDefaultDatabaseVisitor visitor(context, select_database_name); - visitor.visit(query); + visitor.visit(select_query); } else select_database_name = db_and_table->database; @@ -335,7 +337,7 @@ static void extractDependentTable(ContextPtr context, ASTSelectQuery & query, St auto & inner_select_query = ast_select->list_of_selects->children.at(0); - extractDependentTable(context, inner_select_query->as(), select_database_name, select_table_name); + extractDependentTable(context, inner_select_query, select_database_name, select_table_name); } else throw Exception( @@ -943,12 +945,11 @@ StorageWindowView::StorageWindowView( ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW, "UNION is not supported for {}", getName()); - ASTSelectQuery & select_query_ = typeid_cast(*query.select->list_of_selects->children.at(0)); + select_query = query.select->list_of_selects->children.at(0)->clone(); String select_database_name = getContext()->getCurrentDatabase(); String select_table_name; - extractDependentTable(getContext(), select_query_, select_database_name, select_table_name); - - select_query = select_query_.clone(); + auto select_query_tmp = select_query->clone(); + extractDependentTable(getContext(), select_query_tmp, select_database_name, select_table_name); /// If the table is not specified - use the table `system.one` if (select_table_name.empty()) @@ -960,7 +961,7 @@ StorageWindowView::StorageWindowView( DatabaseCatalog::instance().addDependency(select_table_id, table_id_); /// Extract all info from query; substitute Function_TUMPLE and Function_HOP with Function_WINDOW_ID. - auto inner_query = innerQueryParser(select_query_); + auto inner_query = innerQueryParser(select_query->as()); // Parse mergeable query mergeable_query = inner_query->clone(); @@ -1029,7 +1030,7 @@ StorageWindowView::StorageWindowView( } -ASTPtr StorageWindowView::innerQueryParser(ASTSelectQuery & query) +ASTPtr StorageWindowView::innerQueryParser(const ASTSelectQuery & query) { if (!query.groupBy()) throw Exception(ErrorCodes::INCORRECT_QUERY, "GROUP BY query is required for {}", getName()); diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index e989663c7e5..9ebca224a7c 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -209,7 +209,7 @@ private: String function_now_timezone; - ASTPtr innerQueryParser(ASTSelectQuery & inner_query); + ASTPtr innerQueryParser(const ASTSelectQuery & query); void eventTimeParser(const ASTCreateQuery & query); std::shared_ptr getInnerTableCreateQuery( From 31648d95e2f70e0af6cb1286a6674d83ee07b732 Mon Sep 17 00:00:00 2001 From: Dmitriy Dorofeev Date: Mon, 6 Dec 2021 10:49:14 +0300 Subject: [PATCH 533/609] use application/x-ndjson for streaming JSON (#32223) --- src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 147169b5e91..ac03c2991bf 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -27,7 +27,7 @@ public: /// Content-Type to set when sending HTTP response. String getContentType() const override { - return settings.json.array_of_rows ? "application/json; charset=UTF-8" : IRowOutputFormat::getContentType(); + return settings.json.array_of_rows ? "application/json; charset=UTF-8" : "application/x-ndjson; charset=UTF-8" ; } protected: From 2220d1784c1023f11bac5583ba02e21048b825a5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 6 Dec 2021 10:49:44 +0300 Subject: [PATCH 534/609] Disable --- tests/queries/0_stateless/01050_window_view_parser_tumble.sql | 2 ++ tests/queries/0_stateless/01051_window_view_parser_hop.sql | 2 ++ .../0_stateless/01052_window_view_proc_tumble_to_now.sql | 2 ++ tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql | 2 ++ tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql | 2 ++ tests/queries/0_stateless/01055_window_view_proc_hop_to.sql | 2 ++ .../01057_window_view_event_tumble_to_strict_asc.sql | 2 ++ .../0_stateless/01058_window_view_event_hop_to_strict_asc.sql | 2 ++ .../0_stateless/01060_window_view_event_tumble_to_asc.sql | 2 ++ .../queries/0_stateless/01061_window_view_event_hop_to_asc.sql | 2 ++ .../0_stateless/01063_window_view_event_tumble_to_bounded.sql | 2 ++ .../0_stateless/01064_window_view_event_hop_to_bounded.sql | 2 ++ .../01066_window_view_event_tumble_to_strict_asc_lateness.sql | 2 ++ .../01067_window_view_event_tumble_to_asc_lateness.sql | 2 ++ .../01068_window_view_event_tumble_to_bounded_lateness.sql | 2 ++ 15 files changed, 30 insertions(+) diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql index 12f67a68237..f11c4507a98 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql index 3c1e3d16320..6d79cd9a5f5 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql index 787811ad942..f7b4080f2d6 100644 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql index 4413e52025d..fb905041a1e 100644 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql index 8ecd93fbf87..6417c81fe40 100644 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql index 8e28577f645..589892b31e8 100644 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql index 3f1200667ee..47b641dbe97 100644 --- a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql +++ b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql index adf55eb6f2c..9056d91da1e 100644 --- a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql +++ b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql index 2576636f622..4dacf7b8554 100644 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql index 892168226c5..5b086e382be 100644 --- a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql +++ b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql index 290ae600845..9168d966c1c 100644 --- a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql +++ b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql index 25041cdf244..5735f14c569 100644 --- a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql +++ b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql index b8a72c1f0bb..c5859b7dd9f 100644 --- a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql +++ b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql index 92cb366d28d..afc1c54005d 100644 --- a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql +++ b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql index 81c3ef227d8..6ccc335d9a5 100644 --- a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql +++ b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql @@ -1,3 +1,5 @@ +-- Tags: disabled + SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; From bf3d3ea25c8a67fadb88f076103d9b75f9f47d64 Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Mon, 6 Dec 2021 16:20:10 +0800 Subject: [PATCH 535/609] Fix style --- src/IO/WriteBufferFromHTTP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 352e38f3529..102cfd69197 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -10,7 +10,7 @@ WriteBufferFromHTTP::WriteBufferFromHTTP( const Poco::URI & uri, const std::string & method, const std::string & content_type, - const ConnectionTimeouts & timeouts, + const ConnectionTimeouts & timeouts, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) , session{makeHTTPSession(uri, timeouts)} From 55c6e16f375c4f59fb3ca8f65ad73ca89bb9291e Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 11:25:18 +0300 Subject: [PATCH 536/609] Update docs/en/sql-reference/aggregate-functions/reference/sparkbar.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/aggregate-functions/reference/sparkbar.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md index 34052aac86f..63a53ce2f03 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md @@ -7,7 +7,7 @@ toc_title: sparkbar The function plots a frequency histogram for values `x` and the repetition rate of these `y` values over the interval `[min_x, max_x]`. -If no interval is specified, then the minimum `x` will be used as the interval start, and the maximum `x` will use as the interval end. +If no interval is specified, then the minimum `x` is used as the interval start, and the maximum `x` — as the interval end. **Syntax** From e6d00819640ca4125df7b11f1472942cc563b07b Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 11:25:32 +0300 Subject: [PATCH 537/609] Update docs/en/sql-reference/aggregate-functions/reference/sparkbar.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../aggregate-functions/reference/sparkbar.md | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md index 63a53ce2f03..adbe1d551ca 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md @@ -15,13 +15,16 @@ If no interval is specified, then the minimum `x` is used as the interval start, sparkbar(width, min_x, max_x)(x, y) ``` +**Parameters** + +- `width` — The number of segments. Type: [Integer](../../../sql-reference/data-types/int-uint.md). +- `min_x` — The interval start. Optional parameter. +- `max_x` — The interval end. Optional parameter. + **Arguments** -- `width` — The number of segments. Must be [Integer](../../../sql-reference/data-types/int-uint.md). -- `min_x` — The interval start. Optional value. -- `max_x` — The interval end. Optional value. -- `x` — The range of values. -- `y` — The frequency of values. +- `x` — The field with values. +- `y` — The field with the frequency of values. **Returned value** From 59a701ee6ec3a2d49e978fe658fc46ab55a9f0dc Mon Sep 17 00:00:00 2001 From: Frank Chen Date: Mon, 6 Dec 2021 16:31:47 +0800 Subject: [PATCH 538/609] Fix style --- src/IO/WriteBufferFromHTTP.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/WriteBufferFromHTTP.cpp b/src/IO/WriteBufferFromHTTP.cpp index 102cfd69197..5ddc28d2db1 100644 --- a/src/IO/WriteBufferFromHTTP.cpp +++ b/src/IO/WriteBufferFromHTTP.cpp @@ -7,8 +7,8 @@ namespace DB { WriteBufferFromHTTP::WriteBufferFromHTTP( - const Poco::URI & uri, - const std::string & method, + const Poco::URI & uri, + const std::string & method, const std::string & content_type, const ConnectionTimeouts & timeouts, size_t buffer_size_) From e7a431276852019a9491d99c9877646595c6cf3e Mon Sep 17 00:00:00 2001 From: vxider Date: Mon, 6 Dec 2021 08:41:56 +0000 Subject: [PATCH 539/609] add comments --- src/Storages/WindowView/StorageWindowView.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 9ebca224a7c..08f24816d72 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -150,8 +150,11 @@ public: private: Poco::Logger * log; + /// Stored query, e.g. SELECT * FROM * GROUP BY TUMBLE(now(), *) ASTPtr select_query; + /// Used to generate the mergeable state of select_query, e.g. SELECT * FROM * GROUP BY WINDOW_ID(____timestamp, *) ASTPtr mergeable_query; + /// Used to fetch the mergeable state and generate the final result. e.g. SELECT * FROM * GROUP BY TUMBLE(____timestamp, *) ASTPtr final_query; ContextMutablePtr window_view_context; From 5bab484422a09bcbcb537f99675bd87b44bae183 Mon Sep 17 00:00:00 2001 From: vxider Date: Mon, 6 Dec 2021 08:50:10 +0000 Subject: [PATCH 540/609] increase sleep time in tests --- .../0_stateless/01052_window_view_proc_tumble_to_now.sql | 2 +- tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql | 2 +- .../01057_window_view_event_tumble_to_strict_asc.sql | 2 +- .../0_stateless/01058_window_view_event_hop_to_strict_asc.sql | 2 +- .../0_stateless/01060_window_view_event_tumble_to_asc.sql | 2 +- .../queries/0_stateless/01061_window_view_event_hop_to_asc.sql | 2 +- .../0_stateless/01063_window_view_event_tumble_to_bounded.sql | 2 +- .../0_stateless/01064_window_view_event_hop_to_bounded.sql | 2 +- .../01066_window_view_event_tumble_to_strict_asc_lateness.sql | 2 +- .../01067_window_view_event_tumble_to_asc_lateness.sql | 2 +- .../01068_window_view_event_tumble_to_bounded_lateness.sql | 2 +- 11 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql index 787811ad942..169a66e7bc6 100644 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql @@ -9,7 +9,7 @@ CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now('US/Samoa'), INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); -SELECT sleep(2); +SELECT sleep(3); SELECT count from dst; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql index 4413e52025d..c39bab21cb1 100644 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql @@ -9,7 +9,7 @@ CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(now('US/Samoa'), INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); -SELECT sleep(2); +SELECT sleep(3); SELECT count from dst; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql index 3f1200667ee..4883e006e85 100644 --- a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql +++ b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql @@ -17,7 +17,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql index adf55eb6f2c..944fd9939b4 100644 --- a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql +++ b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql @@ -17,7 +17,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql index 2576636f622..18b17fd3d2b 100644 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql @@ -18,7 +18,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql index 892168226c5..2cf98d6b08f 100644 --- a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql +++ b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql @@ -18,7 +18,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql index 290ae600845..37757fd77b3 100644 --- a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql +++ b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql @@ -18,7 +18,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql index 25041cdf244..5f148900905 100644 --- a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql +++ b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql @@ -17,7 +17,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:30'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end; DROP TABLE wv NO DELAY; diff --git a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql index b8a72c1f0bb..1c55b70f3aa 100644 --- a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql +++ b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql @@ -21,7 +21,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:07'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end, count; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql index 92cb366d28d..11409203d4c 100644 --- a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql +++ b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql @@ -21,7 +21,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:07'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end, count; DROP TABLE wv; diff --git a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql index 81c3ef227d8..74a095c632f 100644 --- a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql +++ b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql @@ -22,7 +22,7 @@ INSERT INTO mt VALUES (1, '1990/01/01 12:00:10'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:11'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:12'); -SELECT sleep(1); +SELECT sleep(3); SELECT * from dst order by w_end, count; DROP TABLE wv; From e0fa4731fb7e0dd02d0b0bfee268d28f4591754d Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 6 Dec 2021 12:57:09 +0300 Subject: [PATCH 541/609] Fix typo in tupleToNameValuePairs doc --- docs/en/sql-reference/functions/tuple-functions.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-functions.md b/docs/en/sql-reference/functions/tuple-functions.md index b72b75d6de6..8d06e8ea1cc 100644 --- a/docs/en/sql-reference/functions/tuple-functions.md +++ b/docs/en/sql-reference/functions/tuple-functions.md @@ -175,6 +175,7 @@ in which the `Strings` represents the named fields of the tuple and `T` are the ``` sql tupleToNameValuePairs(tuple) +``` **Arguments** @@ -196,7 +197,7 @@ CREATE TABLE tupletest (`col` Tuple(user_ID UInt64, session_ID UInt64) ENGINE = INSERT INTO tupletest VALUES (tuple( 100, 2502)), (tuple(1,100)); SELECT tupleToNameValuePairs(col) FROM tupletest; -``` +``` Result: From 1ec9039b1de05b08d039d60f7b2f943fffb7abef Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 6 Dec 2021 14:01:56 +0300 Subject: [PATCH 542/609] ExternalDictionariesLoader fix getCurrentDatabase multiple times --- src/Interpreters/ExternalDictionariesLoader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ExternalDictionariesLoader.cpp b/src/Interpreters/ExternalDictionariesLoader.cpp index 74bff33c914..f615aa24a91 100644 --- a/src/Interpreters/ExternalDictionariesLoader.cpp +++ b/src/Interpreters/ExternalDictionariesLoader.cpp @@ -98,12 +98,12 @@ QualifiedTableName ExternalDictionariesLoader::qualifyDictionaryNameWithDatabase /// If dictionary was not qualified with database name, try to resolve dictionary as xml dictionary. if (qualified_name->database.empty() && !has(qualified_name->table)) { - auto current_database_name = query_context->getCurrentDatabase(); + std::string current_database_name = query_context->getCurrentDatabase(); std::string resolved_name = resolveDictionaryNameFromDatabaseCatalog(dictionary_name, current_database_name); /// If after qualify dictionary_name with default_database_name we find it, add default_database to qualified name. if (has(resolved_name)) - qualified_name->database = query_context->getCurrentDatabase(); + qualified_name->database = std::move(current_database_name); } return *qualified_name; From ee2655b704a628ee0bb1501ec07003a9f8fdc475 Mon Sep 17 00:00:00 2001 From: Peignon Melvyn Date: Mon, 6 Dec 2021 12:46:19 +0100 Subject: [PATCH 543/609] Update mergetree.md The link did not had the correct anchor --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index d08de080e6b..4b7473f76ad 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -262,7 +262,7 @@ In the example below, the index can’t be used. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) and [force_primary_key](../../../operations/settings/settings.md). +To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) and [force_primary_key](../../../operations/settings/settings.md#force-primary-key). The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. From 32b48752735094435a65722a59123352fa79a0aa Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Dec 2021 14:53:39 +0300 Subject: [PATCH 544/609] Fix possible Pipeline stuck in case of StrictResize processor. --- src/Processors/ResizeProcessor.cpp | 21 +++++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Processors/ResizeProcessor.cpp b/src/Processors/ResizeProcessor.cpp index d652a342150..f5ee1cb487c 100644 --- a/src/Processors/ResizeProcessor.cpp +++ b/src/Processors/ResizeProcessor.cpp @@ -403,12 +403,22 @@ IProcessor::Status StrictResizeProcessor::prepare(const PortNumbers & updated_in /// Close all other waiting for data outputs (there is no corresponding input for them). while (!waiting_outputs.empty()) { - auto & output = output_ports[waiting_outputs.front()]; - waiting_outputs.pop(); + auto & output = output_ports[waiting_outputs.front()]; + waiting_outputs.pop(); - output.status = OutputStatus::Finished; - output.port->finish(); - ++num_finished_outputs; + if (output.status != OutputStatus::Finished) + ++num_finished_outputs; + + output.status = OutputStatus::Finished; + output.port->finish(); + } + + if (num_finished_outputs == outputs.size()) + { + for (auto & input : inputs) + input.close(); + + return Status::Finished; } if (disabled_input_ports.empty()) @@ -418,4 +428,3 @@ IProcessor::Status StrictResizeProcessor::prepare(const PortNumbers & updated_in } } - From f064f2cdaaae09bfcad5d5f7b161dd36dfe040cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 6 Dec 2021 15:21:39 +0300 Subject: [PATCH 545/609] Use seq_cst semantic for MergeTreeBackgroundExecutor mertic. (#32125) --- .../MergeTree/MergeTreeBackgroundExecutor.h | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h index c59e53fb20e..f4635812e08 100644 --- a/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h +++ b/src/Storages/MergeTree/MergeTreeBackgroundExecutor.h @@ -34,11 +34,22 @@ struct TaskRuntimeData { TaskRuntimeData(ExecutableTaskPtr && task_, CurrentMetrics::Metric metric_) : task(std::move(task_)) - , increment(std::move(metric_)) - {} + , metric(metric_) + { + /// Increment and decrement a metric with sequentially consistent memory order + /// This is needed, because in unit test this metric is read from another thread + /// and some invariant is checked. With relaxed memory order we could read stale value + /// for this metric, that's why test can be failed. + CurrentMetrics::values[metric].fetch_add(1); + } + + ~TaskRuntimeData() + { + CurrentMetrics::values[metric].fetch_sub(1); + } ExecutableTaskPtr task; - CurrentMetrics::Increment increment; + CurrentMetrics::Metric metric; std::atomic_bool is_currently_deleting{false}; /// Actually autoreset=false is needed only for unit test /// where multiple threads could remove tasks corresponding to the same storage From 39554a6843749fa0e4a857af5064f7c862f7e120 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 6 Dec 2021 15:29:42 +0300 Subject: [PATCH 546/609] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index a19380e36f4..1750ffc340b 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1005,7 +1005,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - `database` – имя базы данных. - `table` – имя системной таблицы, где будут логироваться запросы. -- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать если указывается параметр `engine`. +- `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать, если задан параметр `engine`. - `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если задан параметр `partition_by`. - `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. From a45b204eb478be80c0fede42e6e33e025d690460 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 6 Dec 2021 15:29:47 +0300 Subject: [PATCH 547/609] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 1750ffc340b..1b0c7fc5897 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1006,7 +1006,7 @@ ClickHouse проверяет условия для `min_part_size` и `min_part - `database` – имя базы данных. - `table` – имя системной таблицы, где будут логироваться запросы. - `partition_by` — устанавливает [произвольный ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Нельзя использовать, если задан параметр `engine`. -- `engine` - устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать если задан параметр `partition_by`. +- `engine` — устанавливает [настройки MergeTree Engine](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-creating-a-table) для системной таблицы. Нельзя использовать, если задан параметр `partition_by`. - `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. From c3a7858a6378811cbf1ea2b427a033c0f35c12c8 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 6 Dec 2021 16:22:15 +0300 Subject: [PATCH 548/609] Fix comments --- docs/en/operations/settings/settings.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- docs/ru/operations/system-tables/query_views_log.md | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c4c13a48560..fa4cc41e8ff 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -992,7 +992,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Query threads log into `system.query_thread_log` table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. +Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. Possible values: diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index d6830a30321..f9717b0fb27 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -914,7 +914,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Управляет логированием информации о потоках выполнения запросов. -Информация о потоках выполнения запросов сохраняется в системной таблице `system.query_thread_log`. Работает только в том случае, если включена настройка [log_queries](#settings-log-queries). Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). +Информация о потоках выполнения запросов сохраняется в системной таблице [system.query_thread_log](../../operations/system-tables/query_thread_log.md). Работает только в том случае, если включена настройка [log_queries](#settings-log-queries). Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log). Возможные значения: diff --git a/docs/ru/operations/system-tables/query_views_log.md b/docs/ru/operations/system-tables/query_views_log.md index 9715d739af1..8b1a8d387a6 100644 --- a/docs/ru/operations/system-tables/query_views_log.md +++ b/docs/ru/operations/system-tables/query_views_log.md @@ -11,7 +11,7 @@ ClickHouse не удаляет данные из таблицы автоматически. Подробнее смотрите раздел [Системные таблицы](../../operations/system-tables/index.md#system-tables-introduction). -Вы можете включить настройку [log_queries_probability](../../operations/settings/settings.md#log-queries-probability), чтобы уменьшить количество запросов, регистрируемых в таблице `query_views_log`. +Чтобы уменьшить количество запросов, регистрируемых в таблице `query_views_log`, вы можете включить настройку [log_queries_probability](../../operations/settings/settings.md#log-queries-probability). Столбцы: @@ -33,7 +33,7 @@ ClickHouse не удаляет данные из таблицы автомати - `written_rows` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных строк. - `written_bytes` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — количество записанных байт. - `peak_memory_usage` ([Int64](../../sql-reference/data-types/int-uint.md)) — максимальная разница между объемом выделенной и освобожденной памяти в контексте этого представления. -- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — События профиля, которые измеряют различные показатели. Их описание можно найти в таблице [system.events](../../operations/system-tables/events.md#system_tables-events). +- `ProfileEvents` ([Map(String, UInt64)](../../sql-reference/data-types/array.md)) — события профиля, которые измеряют различные показатели. Их описание можно найти в таблице [system.events](../../operations/system-tables/events.md#system_tables-events). - `status` ([Enum8](../../sql-reference/data-types/enum.md)) — статус представления. Возможные значения: - `'QueryStart' = 1` — успешное начало выполнения представления. Не должно отображаться. - `'QueryFinish' = 2` — успешное завершение выполнения представления. From 7797a7290776deca2c20dc14b9730fb8cd7df25e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 6 Dec 2021 16:35:34 +0300 Subject: [PATCH 549/609] fix false-positive ciclyc dependency with XML dict --- src/Databases/DDLDependencyVisitor.cpp | 6 ++++- src/Databases/DDLDependencyVisitor.h | 2 +- src/Databases/DatabaseMemory.cpp | 2 +- src/Databases/DatabaseOrdinary.cpp | 4 +-- src/Databases/TablesLoader.cpp | 8 ++++-- src/Interpreters/InterpreterCreateQuery.cpp | 5 ++-- .../configs/dictionaries/node.xml | 25 +++++++++++++++++++ .../test_dictionaries_dependency_xml/test.py | 9 ++++++- 8 files changed, 51 insertions(+), 10 deletions(-) create mode 100644 tests/integration/test_dictionaries_dependency_xml/configs/dictionaries/node.xml diff --git a/src/Databases/DDLDependencyVisitor.cpp b/src/Databases/DDLDependencyVisitor.cpp index 09d3752b180..532691f7978 100644 --- a/src/Databases/DDLDependencyVisitor.cpp +++ b/src/Databases/DDLDependencyVisitor.cpp @@ -11,7 +11,7 @@ namespace DB { -TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const ASTPtr & ast) +TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast) { assert(global_context == global_context->getGlobalContext()); TableLoadingDependenciesVisitor::Data data; @@ -20,6 +20,7 @@ TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const data.global_context = global_context; TableLoadingDependenciesVisitor visitor{data}; visitor.visit(ast); + data.dependencies.erase(table); return data.dependencies; } @@ -132,7 +133,10 @@ void DDLDependencyVisitor::extractTableNameFromArgument(const ASTFunction & func } if (qualified_name.database.empty()) + { + /// It can be table/dictionary from default database or XML dictionary, but we cannot distinguish it here. qualified_name.database = data.default_database; + } data.dependencies.emplace(std::move(qualified_name)); } diff --git a/src/Databases/DDLDependencyVisitor.h b/src/Databases/DDLDependencyVisitor.h index b5ca976f665..ae7f7aa94d9 100644 --- a/src/Databases/DDLDependencyVisitor.h +++ b/src/Databases/DDLDependencyVisitor.h @@ -12,7 +12,7 @@ class ASTStorage; using TableNamesSet = std::unordered_set; -TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const ASTPtr & ast); +TableNamesSet getDependenciesSetFromCreateQuery(ContextPtr global_context, const QualifiedTableName & table, const ASTPtr & ast); /// Visits ASTCreateQuery and extracts names of table (or dictionary) dependencies /// from column default expressions (joinGet, dictGet, etc) diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 50e56885743..3309d25b1c2 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -121,7 +121,7 @@ void DatabaseMemory::alterTable(ContextPtr local_context, const StorageID & tabl throw Exception(ErrorCodes::UNKNOWN_TABLE, "Cannot alter: There is no metadata of table {}", table_id.getNameForLogs()); applyMetadataChangesToCreateQuery(it->second, metadata); - TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), it->second); + TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), it->second); DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index b7a0aff24d6..b5557d9a08d 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -181,8 +181,8 @@ void DatabaseOrdinary::loadTablesMetadata(ContextPtr local_context, ParsedTables return; } - TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext(), ast); QualifiedTableName qualified_name{database_name, create_query->getTable()}; + TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext(), qualified_name, ast); std::lock_guard lock{metadata.mutex}; metadata.parsed_tables[qualified_name] = ParsedTableMetadata{full_path.string(), ast}; @@ -297,7 +297,7 @@ void DatabaseOrdinary::alterTable(ContextPtr local_context, const StorageID & ta out.close(); } - TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), ast); + TableNamesSet new_dependencies = getDependenciesSetFromCreateQuery(local_context->getGlobalContext(), table_id.getQualifiedName(), ast); DatabaseCatalog::instance().updateLoadingDependencies(table_id, std::move(new_dependencies)); commitAlterTable(table_id, table_metadata_tmp_path, table_metadata_path, statement, local_context); diff --git a/src/Databases/TablesLoader.cpp b/src/Databases/TablesLoader.cpp index b8c380b7be1..4ce719279f3 100644 --- a/src/Databases/TablesLoader.cpp +++ b/src/Databases/TablesLoader.cpp @@ -133,10 +133,14 @@ void TablesLoader::removeUnresolvableDependencies(bool remove_loaded) /// Table exists and it's already loaded if (DatabaseCatalog::instance().isTableExist(StorageID(dependency_name.database, dependency_name.table), global_context)) return remove_loaded; - /// It's XML dictionary. It was loaded before tables and DDL dictionaries. + /// It's XML dictionary. if (dependency_name.database == metadata.default_database && global_context->getExternalDictionariesLoader().has(dependency_name.table)) - return remove_loaded; + { + LOG_WARNING(log, "Tables {} depend on XML dictionary {}, but XML dictionaries are loaded independently." + "Consider converting it to DDL dictionary.", fmt::join(info.dependent_database_objects, ", "), dependency_name); + return true; + } /// Some tables depends on table "dependency_name", but there is no such table in DatabaseCatalog and we don't have its metadata. /// We will ignore it and try to load dependent tables without "dependency_name" diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 0d9cad34545..8f003e75a07 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -981,9 +981,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) return {}; /// If table has dependencies - add them to the graph - TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext()->getGlobalContext(), query_ptr); + QualifiedTableName qualified_name{database_name, create.getTable()}; + TableNamesSet loading_dependencies = getDependenciesSetFromCreateQuery(getContext()->getGlobalContext(), qualified_name, query_ptr); if (!loading_dependencies.empty()) - DatabaseCatalog::instance().addLoadingDependencies(QualifiedTableName{database_name, create.getTable()}, std::move(loading_dependencies)); + DatabaseCatalog::instance().addLoadingDependencies(std::move(qualified_name), std::move(loading_dependencies)); return fillTableIfNeeded(create); } diff --git a/tests/integration/test_dictionaries_dependency_xml/configs/dictionaries/node.xml b/tests/integration/test_dictionaries_dependency_xml/configs/dictionaries/node.xml new file mode 100644 index 00000000000..2cd957a3720 --- /dev/null +++ b/tests/integration/test_dictionaries_dependency_xml/configs/dictionaries/node.xml @@ -0,0 +1,25 @@ + + + node + + + localhost + 9000 + default + + system + select dummy, toString(dummy) from system.one + + + 0 + + + key + + name + String + + + + + diff --git a/tests/integration/test_dictionaries_dependency_xml/test.py b/tests/integration/test_dictionaries_dependency_xml/test.py index 1b3ea32d09c..13635c7b969 100644 --- a/tests/integration/test_dictionaries_dependency_xml/test.py +++ b/tests/integration/test_dictionaries_dependency_xml/test.py @@ -3,7 +3,7 @@ from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', - 'configs/dictionaries/dep_z.xml'] + 'configs/dictionaries/dep_z.xml', 'configs/dictionaries/node.xml'] cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', dictionaries=DICTIONARY_FILES, stay_alive=True) @@ -117,3 +117,10 @@ def test_dependent_tables(started_cluster): query("drop table system.join") query("drop database a") query("drop database lazy") + + +def test_xml_dict_same_name(started_cluster): + instance.query("create table default.node ( key UInt64, name String ) Engine=Dictionary(node);") + instance.restart_clickhouse() + assert "node" in instance.query("show tables from default") + instance.query("drop table default.node") From 5b03dabb09362fc099467c6e4a960989aa17f8fc Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 6 Dec 2021 16:38:19 +0300 Subject: [PATCH 550/609] Update string-search-functions.md --- docs/ru/sql-reference/functions/string-search-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/string-search-functions.md b/docs/ru/sql-reference/functions/string-search-functions.md index f0c8f51225e..4969bf0f2eb 100644 --- a/docs/ru/sql-reference/functions/string-search-functions.md +++ b/docs/ru/sql-reference/functions/string-search-functions.md @@ -357,7 +357,7 @@ Result: ## multiFuzzyMatchAny(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyhaystack-distance-pattern1-pattern2-patternn} -То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с нечеткими вариантами. +То же, что и `multiMatchAny`, но возвращает 1 если любой шаблон соответствует haystack в пределах константного [редакционного расстояния](https://en.wikipedia.org/wiki/Edit_distance). Эта функция основана на экспериментальной библиотеке [hyperscan](https://intel.github.io/hyperscan/dev-reference/compilation.html#approximate-matching) и может быть медленной для некоторых частных случаев. Производительность зависит от значения редакционного расстояния и используемых шаблонов, но всегда медленнее по сравнению с non-fuzzy вариантами. ## multiFuzzyMatchAnyIndex(haystack, distance, \[pattern1, pattern2, …, patternn\]) {#multifuzzymatchanyindexhaystack-distance-pattern1-pattern2-patternn} From ee8a1607909d9867f7661d9bd6d0a605a5ab8820 Mon Sep 17 00:00:00 2001 From: vxider Date: Mon, 6 Dec 2021 14:26:39 +0000 Subject: [PATCH 551/609] enable window view tests --- tests/queries/0_stateless/01050_window_view_parser_tumble.sql | 2 -- tests/queries/0_stateless/01051_window_view_parser_hop.sql | 2 -- .../0_stateless/01052_window_view_proc_tumble_to_now.sql | 2 -- tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql | 2 -- tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql | 2 -- tests/queries/0_stateless/01055_window_view_proc_hop_to.sql | 2 -- .../01057_window_view_event_tumble_to_strict_asc.sql | 2 -- .../0_stateless/01058_window_view_event_hop_to_strict_asc.sql | 2 -- .../0_stateless/01060_window_view_event_tumble_to_asc.sql | 2 -- .../queries/0_stateless/01061_window_view_event_hop_to_asc.sql | 2 -- .../0_stateless/01063_window_view_event_tumble_to_bounded.sql | 2 -- .../0_stateless/01064_window_view_event_hop_to_bounded.sql | 2 -- .../01066_window_view_event_tumble_to_strict_asc_lateness.sql | 2 -- .../01067_window_view_event_tumble_to_asc_lateness.sql | 2 -- .../01068_window_view_event_tumble_to_bounded_lateness.sql | 2 -- 15 files changed, 30 deletions(-) diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql index 996ff4aa55a..6837036263c 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql index 48ffe4c7f21..df0729108d0 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql index d23461570b5..169a66e7bc6 100644 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql index 6ad9852c930..c39bab21cb1 100644 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql index 22435dc3309..f229969603b 100644 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql index fc2eccdf4db..b75cc33e741 100644 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql index 799fe9b71cd..4883e006e85 100644 --- a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql +++ b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql index 677f814efe4..944fd9939b4 100644 --- a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql +++ b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql index 2edec8b8f11..18b17fd3d2b 100644 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql index c10f09f2485..2cf98d6b08f 100644 --- a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql +++ b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql index 80e3ef02ea6..37757fd77b3 100644 --- a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql +++ b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql index ce9ae4c007f..5f148900905 100644 --- a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql +++ b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql index 4d1c9929727..1c55b70f3aa 100644 --- a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql +++ b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql index 34bccd86769..11409203d4c 100644 --- a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql +++ b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; diff --git a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql index f3ea2fc1f1e..74a095c632f 100644 --- a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql +++ b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql @@ -1,5 +1,3 @@ --- Tags: disabled - SET allow_experimental_window_view = 1; DROP TABLE IF EXISTS mt; From 303552f51507178fae443ddf822485031b6f7d9f Mon Sep 17 00:00:00 2001 From: msaf1980 Date: Thu, 25 Nov 2021 22:16:20 +0500 Subject: [PATCH 552/609] graphite: split tagged/plain rollup rules (for merges perfomance) --- base/base/StringRef.h | 5 +- src/CMakeLists.txt | 1 + src/Common/tests/gtest_global_context.cpp | 7 + src/Common/tests/gtest_global_context.h | 6 +- src/Processors/Merges/Algorithms/Graphite.cpp | 493 +++++++++++++++ src/Processors/Merges/Algorithms/Graphite.h | 37 +- .../GraphiteRollupSortedAlgorithm.cpp | 59 +- .../GraphiteRollupSortedAlgorithm.h | 10 - .../Algorithms/tests/gtest_graphite.cpp | 597 ++++++++++++++++++ .../MergeTree/registerStorageMergeTree.cpp | 175 ----- src/Storages/System/StorageSystemGraphite.cpp | 4 + tests/integration/helpers/test_tools.py | 16 + .../test_graphite_merge_tree/test.py | 20 +- .../__init__.py | 0 .../configs/graphite_rollup.xml | 120 ++++ .../configs/users.xml | 8 + .../test_graphite_merge_tree_typed/test.py | 580 +++++++++++++++++ ...ultiple_paths_and_versions.reference.plain | 84 +++ ...ltiple_paths_and_versions.reference.tagged | 84 +++ .../02117_show_create_table_system.reference | 2 +- utils/CMakeLists.txt | 1 + utils/graphite-rollup/CMakeLists.txt | 23 + .../graphite-rollup/graphite-rollup-bench.cpp | 147 +++++ utils/graphite-rollup/metrics.txt | 11 + utils/graphite-rollup/rollup-tag-list.xml | 167 +++++ utils/graphite-rollup/rollup-typed.xml | 167 +++++ utils/graphite-rollup/rollup.xml | 147 +++++ 27 files changed, 2705 insertions(+), 266 deletions(-) create mode 100644 src/Common/tests/gtest_global_context.cpp create mode 100644 src/Processors/Merges/Algorithms/Graphite.cpp create mode 100644 src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp create mode 100644 tests/integration/test_graphite_merge_tree_typed/__init__.py create mode 100644 tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml create mode 100644 tests/integration/test_graphite_merge_tree_typed/configs/users.xml create mode 100644 tests/integration/test_graphite_merge_tree_typed/test.py create mode 100644 tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain create mode 100644 tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged create mode 100644 utils/graphite-rollup/CMakeLists.txt create mode 100644 utils/graphite-rollup/graphite-rollup-bench.cpp create mode 100644 utils/graphite-rollup/metrics.txt create mode 100644 utils/graphite-rollup/rollup-tag-list.xml create mode 100644 utils/graphite-rollup/rollup-typed.xml create mode 100644 utils/graphite-rollup/rollup.xml diff --git a/base/base/StringRef.h b/base/base/StringRef.h index d0184dbc24c..98c322320a5 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -48,7 +48,10 @@ struct StringRef std::string toString() const { return std::string(data, size); } explicit operator std::string() const { return toString(); } - constexpr explicit operator std::string_view() const { return {data, size}; } + + std::string_view toView() const { return std::string_view(data, size); } + + constexpr explicit operator std::string_view() const { return std::string_view(data, size); } }; /// Here constexpr doesn't implicate inline, see https://www.viva64.com/en/w/v1043/ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5f4ebaaa895..1f7a2700e5a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -547,6 +547,7 @@ if (ENABLE_TESTS AND USE_GTEST) clickhouse_parsers clickhouse_storages_system dbms + clickhouse_common_config clickhouse_common_zookeeper string_utils) diff --git a/src/Common/tests/gtest_global_context.cpp b/src/Common/tests/gtest_global_context.cpp new file mode 100644 index 00000000000..19ba3cdc269 --- /dev/null +++ b/src/Common/tests/gtest_global_context.cpp @@ -0,0 +1,7 @@ +#include "gtest_global_context.h" + +const ContextHolder & getContext() +{ + static ContextHolder holder; + return holder; +} diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 9bd7c2490d6..7756be7ce9b 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -18,8 +18,4 @@ struct ContextHolder ContextHolder(ContextHolder &&) = default; }; -inline const ContextHolder & getContext() -{ - static ContextHolder holder; - return holder; -} +const ContextHolder & getContext(); diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp new file mode 100644 index 00000000000..38d3fa30b42 --- /dev/null +++ b/src/Processors/Merges/Algorithms/Graphite.cpp @@ -0,0 +1,493 @@ +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include + +#include + +using namespace std::literals; + +namespace DB::ErrorCodes +{ + extern const int NOT_IMPLEMENTED; + extern const int BAD_ARGUMENTS; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int NO_ELEMENTS_IN_CONFIG; + } + +namespace DB::Graphite +{ +static std::unordered_map ruleTypeMap = +{ + { RuleTypeAll, "all" }, + { RuleTypePlain, "plain" }, + { RuleTypeTagged, "tagged"}, + { RuleTypeTagList, "tag_list"} +}; + +const String & ruleTypeStr(RuleType rule_type) +{ + try + { + return ruleTypeMap.at(rule_type); + } + catch (...) + { + throw Exception("invalid rule type: " + std::to_string(rule_type), DB::ErrorCodes::BAD_ARGUMENTS); + } +} + +RuleType ruleType(const String & s) +{ + if (s == "all") + return RuleTypeAll; + else if (s == "plain") + return RuleTypePlain; + else if (s == "tagged") + return RuleTypeTagged; + else if (s == "tag_list") + return RuleTypeTagList; + else + throw Exception("invalid rule type: " + s, DB::ErrorCodes::BAD_ARGUMENTS); +} + +static const Graphite::Pattern undef_pattern = +{ /// empty pattern for selectPatternForPath + .rule_type = RuleTypeAll, + .regexp = nullptr, + .regexp_str = "", + .function = nullptr, + .retentions = Graphite::Retentions(), + .type = undef_pattern.TypeUndef, +}; + +inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, const StringRef path) +{ + if (params.patterns_typed) + { + std::string_view path_view = path.toView(); + if (path_view.find("?"sv) == path_view.npos) + return params.patterns_plain; + else + return params.patterns_tagged; + } + else + { + return params.patterns; + } +} + +Graphite::RollupRule selectPatternForPath( + const Graphite::Params & params, + const StringRef path) +{ + const Graphite::Pattern * first_match = &undef_pattern; + + const Patterns & patterns_check = selectPatternsForMetricType(params, path); + + for (const auto & pattern : patterns_check) + { + if (!pattern.regexp) + { + /// Default pattern + if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll) + { + /// There is only default pattern for both retention and aggregation + return std::pair(&pattern, &pattern); + } + if (pattern.type != first_match->type) + { + if (first_match->type == first_match->TypeRetention) + { + return std::pair(first_match, &pattern); + } + if (first_match->type == first_match->TypeAggregation) + { + return std::pair(&pattern, first_match); + } + } + } + else + { + if (pattern.regexp->match(path.data, path.size)) + { + /// General pattern with matched path + if (pattern.type == pattern.TypeAll) + { + /// Only for not default patterns with both function and retention parameters + return std::pair(&pattern, &pattern); + } + if (first_match->type == first_match->TypeUndef) + { + first_match = &pattern; + continue; + } + if (pattern.type != first_match->type) + { + if (first_match->type == first_match->TypeRetention) + { + return std::pair(first_match, &pattern); + } + if (first_match->type == first_match->TypeAggregation) + { + return std::pair(&pattern, first_match); + } + } + } + } + } + + return {nullptr, nullptr}; +} + +/** Is used to order Graphite::Retentions by age and precision descending. + * Throws exception if not both age and precision are less or greater then another. + */ +static bool compareRetentions(const Retention & a, const Retention & b) +{ + if (a.age > b.age && a.precision > b.precision) + { + return true; + } + else if (a.age < b.age && a.precision < b.precision) + { + return false; + } + String error_msg = "age and precision should only grow up: " + + std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs " + + std::to_string(b.age) + ":" + std::to_string(b.precision); + throw Exception( + error_msg, + DB::ErrorCodes::BAD_ARGUMENTS); +} + +bool operator==(const Retention & a, const Retention & b) +{ + return a.age == b.age && a.precision == b.precision; +} + +std::ostream & operator<<(std::ostream & stream, const Retentions & a) +{ + stream << "{ "; + for (size_t i = 0; i < a.size(); i++) + { + if (i > 0) + stream << ","; + stream << " { age = " << a[i].age << ", precision = " << a[i].precision << " }"; + } + stream << " }"; + + return stream; +} + +bool operator==(const Pattern & a, const Pattern & b) +{ + // equal + // Retentions retentions; /// Must be ordered by 'age' descending. + if (a.type != b.type || a.regexp_str != b.regexp_str || a.rule_type != b.rule_type) + return false; + + if (a.function == nullptr) + { + if (b.function != nullptr) + return false; + } + else if (b.function == nullptr) + { + return false; + } + else if (a.function->getName() != b.function->getName()) + { + return false; + } + + return a.retentions == b.retentions; +} + +std::ostream & operator<<(std::ostream & stream, const Pattern & a) +{ + stream << "{ rule_type = " << ruleTypeStr(a.rule_type); + if (!a.regexp_str.empty()) + stream << ", regexp = '" << a.regexp_str << "'"; + if (a.function != nullptr) + stream << ", function = " << a.function->getName(); + if (!a.retentions.empty()) + { + stream << ",\n retentions = {\n"; + for (size_t i = 0; i < a.retentions.size(); i++) + { + stream << " { " << a.retentions[i].age << ", " << a.retentions[i].precision << " }"; + if (i < a.retentions.size() - 1) + stream << ","; + stream << "\n"; + } + stream << " }\n"; + } + else + stream << " "; + + stream << "}"; + return stream; +} + +std::string buildTaggedRegex(std::string regexp_str) +{ + /* + * tags list in format (for name or any value can use regexp, alphabet sorting not needed) + * spaces are not stiped and used as tag and value part + * name must be first (if used) + * + * tag1=value1; tag2=VALUE2_REGEX;tag3=value3 + * or + * name;tag1=value1;tag2=VALUE2_REGEX;tag3=value3 + * or for one tag + * tag1=value1 + * + * Resulting regex against metric like + * name?tag1=value1&tag2=value2 + * + * So, + * + * name + * produce + * name\? + * + * tag2=val2 + * produce + * [\?&]tag2=val2(&.*)?$ + * + * nam.* ; tag1=val1 ; tag2=val2 + * produce + * nam.*\?(.*&)?tag1=val1&(.*&)?tag2=val2(&.*)?$ + */ + + std::vector tags; + + splitInto<';'>(tags, regexp_str); + /* remove empthy elements */ + using namespace std::string_literals; + tags.erase(std::remove(tags.begin(), tags.end(), ""s), tags.end()); + if (tags[0].find('=') == tags[0].npos) + { + if (tags.size() == 1) /* only name */ + return "^" + tags[0] + "\\?"; + /* start with name value */ + regexp_str = "^" + tags[0] + "\\?(.*&)?"; + tags.erase(std::begin(tags)); + } + else + regexp_str = "[\\?&]"; + + std::sort(std::begin(tags), std::end(tags)); /* sorted tag keys */ + regexp_str += fmt::format( + "{}{}", + fmt::join(tags, "&(.*&)?"), + "(&.*)?$" /* close regex */ + ); + + return regexp_str; +} + +/** Read the settings for Graphite rollup from config. + * Example + * + * + * Path + * + * click_cost + * any + * + * 0 + * 3600 + * + * + * 86400 + * 60 + * + * + * + * max + * + * 0 + * 60 + * + * + * 3600 + * 300 + * + * + * 86400 + * 3600 + * + * + * + */ +static const Pattern & +appendGraphitePattern( + const Poco::Util::AbstractConfiguration & config, + const String & config_element, Patterns & patterns, + bool default_rule, + ContextPtr context) +{ + Pattern pattern; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_element, keys); + + for (const auto & key : keys) + { + if (key == "regexp") + { + pattern.regexp_str = config.getString(config_element + ".regexp"); + } + else if (key == "function") + { + String aggregate_function_name_with_params = config.getString(config_element + ".function"); + String aggregate_function_name; + Array params_row; + getAggregateFunctionNameAndParametersArray( + aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context); + + /// TODO Not only Float64 + AggregateFunctionProperties properties; + pattern.function = AggregateFunctionFactory::instance().get( + aggregate_function_name, {std::make_shared()}, params_row, properties); + } + else if (key == "rule_type") + { + String rule_type = config.getString(config_element + ".rule_type"); + pattern.rule_type = ruleType(rule_type); + } + else if (startsWith(key, "retention")) + { + pattern.retentions.emplace_back(Graphite::Retention{ + .age = config.getUInt(config_element + "." + key + ".age"), + .precision = config.getUInt(config_element + "." + key + ".precision")}); + } + else + throw Exception("Unknown element in config: " + key, DB::ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + if (!pattern.regexp_str.empty()) + { + if (pattern.rule_type == RuleTypeTagList) + { + // construct tagged regexp + pattern.regexp_str = buildTaggedRegex(pattern.regexp_str); + pattern.rule_type = RuleTypeTagged; + } + pattern.regexp = std::make_shared(pattern.regexp_str); + } + + if (!pattern.function && pattern.retentions.empty()) + throw Exception( + "At least one of an aggregate function or retention rules is mandatory for rollup patterns in GraphiteMergeTree", + DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + if (default_rule && pattern.rule_type != RuleTypeAll) + { + throw Exception( + "Default must have rule_type all for rollup patterns in GraphiteMergeTree", + DB::ErrorCodes::BAD_ARGUMENTS); + } + + if (!pattern.function) + { + pattern.type = pattern.TypeRetention; + } + else if (pattern.retentions.empty()) + { + pattern.type = pattern.TypeAggregation; + } + else + { + pattern.type = pattern.TypeAll; + } + + if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll + if (pattern.function->allocatesMemoryInArena()) + throw Exception( + "Aggregate function " + pattern.function->getName() + " isn't supported in GraphiteMergeTree", DB::ErrorCodes::NOT_IMPLEMENTED); + + /// retention should be in descending order of age. + if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll + std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions); + + patterns.emplace_back(pattern); + return patterns.back(); +} + +void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params) +{ + const auto & config = context->getConfigRef(); + + if (!config.has(config_element)) + throw Exception("No '" + config_element + "' element in configuration file", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + params.config_name = config_element; + params.path_column_name = config.getString(config_element + ".path_column_name", "Path"); + params.time_column_name = config.getString(config_element + ".time_column_name", "Time"); + params.value_column_name = config.getString(config_element + ".value_column_name", "Value"); + params.version_column_name = config.getString(config_element + ".version_column_name", "Timestamp"); + + params.patterns_typed = false; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_element, keys); + + for (const auto & key : keys) + { + if (startsWith(key, "pattern")) + { + if (appendGraphitePattern(config, config_element + "." + key, params.patterns, false, context).rule_type != RuleTypeAll) + params.patterns_typed = true; + } + else if (key == "default") + { + /// See below. + } + else if (key == "path_column_name" || key == "time_column_name" || key == "value_column_name" || key == "version_column_name") + { + /// See above. + } + else + throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + if (config.has(config_element + ".default")) + appendGraphitePattern(config, config_element + "." + ".default", params.patterns, true, context); + + for (const auto & pattern : params.patterns) + { + if (pattern.rule_type == RuleTypeAll) + { + if (params.patterns_typed) + { + params.patterns_plain.push_back(pattern); + params.patterns_tagged.push_back(pattern); + } + } + else if (pattern.rule_type == RuleTypePlain) + { + params.patterns_plain.push_back(pattern); + } + else if (pattern.rule_type == RuleTypeTagged) + { + params.patterns_tagged.push_back(pattern); + } + else + { + throw Exception("Unhandled rule_type in config: " + ruleTypeStr(pattern.rule_type), ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + } +} + +} diff --git a/src/Processors/Merges/Algorithms/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h index ecb1aeb9804..dc39cb46386 100644 --- a/src/Processors/Merges/Algorithms/Graphite.h +++ b/src/Processors/Merges/Algorithms/Graphite.h @@ -1,13 +1,8 @@ #pragma once + +#include #include - -namespace DB -{ - -class IAggregateFunction; -using AggregateFunctionPtr = std::shared_ptr; - -} +#include /** Intended for implementation of "rollup" - aggregation (rounding) of older data * for a table with Graphite data (Graphite is the system for time series monitoring). @@ -97,16 +92,32 @@ using AggregateFunctionPtr = std::shared_ptr; namespace DB::Graphite { +// sync with rule_types_str +enum RuleType +{ + RuleTypeAll = 0, // default, with regex, compatible with old scheme + RuleTypePlain = 1, // plain metrics, with regex, compatible with old scheme + RuleTypeTagged = 2, // tagged metrics, with regex, compatible with old scheme + RuleTypeTagList = 3 // tagged metrics, with regex (converted to RuleTypeTagged from string like 'retention=10min ; env=(staging|prod)') +}; + +const String & ruleTypeStr(RuleType rule_type); + struct Retention { UInt32 age; UInt32 precision; }; +bool operator==(const Retention & a, const Retention & b); + using Retentions = std::vector; +std::ostream &operator<<(std::ostream & stream, const Retentions & a); + struct Pattern { + RuleType rule_type = RuleTypeAll; std::shared_ptr regexp; std::string regexp_str; AggregateFunctionPtr function; @@ -114,6 +125,9 @@ struct Pattern enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically }; +bool operator==(const Pattern & a, const Pattern & b); +std::ostream &operator<<(std::ostream & stream, const Pattern & a); + using Patterns = std::vector; using RetentionPattern = Pattern; using AggregationPattern = Pattern; @@ -125,9 +139,16 @@ struct Params String time_column_name; String value_column_name; String version_column_name; + bool patterns_typed; Graphite::Patterns patterns; + Graphite::Patterns patterns_plain; + Graphite::Patterns patterns_tagged; }; using RollupRule = std::pair; +Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, const StringRef path); + +void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params); + } diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index 328c34823a0..c4f60571dd9 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -52,62 +53,6 @@ GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm( columns_definition = defineColumns(header, params); } -Graphite::RollupRule GraphiteRollupSortedAlgorithm::selectPatternForPath(StringRef path) const -{ - const Graphite::Pattern * first_match = &undef_pattern; - - for (const auto & pattern : params.patterns) - { - if (!pattern.regexp) - { - /// Default pattern - if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll) - { - /// There is only default pattern for both retention and aggregation - return std::pair(&pattern, &pattern); - } - if (pattern.type != first_match->type) - { - if (first_match->type == first_match->TypeRetention) - { - return std::pair(first_match, &pattern); - } - if (first_match->type == first_match->TypeAggregation) - { - return std::pair(&pattern, first_match); - } - } - } - else if (pattern.regexp->match(path.data, path.size)) - { - /// General pattern with matched path - if (pattern.type == pattern.TypeAll) - { - /// Only for not default patterns with both function and retention parameters - return std::pair(&pattern, &pattern); - } - if (first_match->type == first_match->TypeUndef) - { - first_match = &pattern; - continue; - } - if (pattern.type != first_match->type) - { - if (first_match->type == first_match->TypeRetention) - { - return std::pair(first_match, &pattern); - } - if (first_match->type == first_match->TypeAggregation) - { - return std::pair(&pattern, first_match); - } - } - } - } - - return {nullptr, nullptr}; -} - UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions & retentions, time_t time) const { static_assert(is_signed_v, "time_t must be signed type"); @@ -188,7 +133,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() Graphite::RollupRule next_rule = merged_data.currentRule(); if (new_path) - next_rule = selectPatternForPath(next_path); + next_rule = selectPatternForPath(this->params, next_path); const Graphite::RetentionPattern * retention_pattern = std::get<0>(next_rule); time_t next_time_rounded; diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index 0155b73b238..4968cbfc470 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -102,16 +102,6 @@ private: time_t current_time = 0; time_t current_time_rounded = 0; - const Graphite::Pattern undef_pattern = - { /// temporary empty pattern for selectPatternForPath - .regexp = nullptr, - .regexp_str = "", - .function = nullptr, - .retentions = DB::Graphite::Retentions(), - .type = undef_pattern.TypeUndef, - }; - - Graphite::RollupRule selectPatternForPath(StringRef path) const; UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const; /// Insert the values into the resulting columns, which will not be changed in the future. diff --git a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp new file mode 100644 index 00000000000..1d739bf566a --- /dev/null +++ b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp @@ -0,0 +1,597 @@ +#include +#include +#include +#include + +#include + +#include +#include + +#include +#include +#include +#include + +using namespace DB; + +static int regAggregateFunctions = 0; + +void tryRegisterAggregateFunctions() +{ + if (!regAggregateFunctions) + { + registerAggregateFunctions(); + regAggregateFunctions = 1; + } +} + +static ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path) +{ + ConfigProcessor config_processor(config_path, true, true); + ConfigProcessor::LoadedConfig config = config_processor.loadConfig(false); + return config; +} + +static ConfigProcessor::LoadedConfig loadConfigurationFromString(std::string & s) +{ + char tmp_file[19]; + strcpy(tmp_file, "/tmp/rollup-XXXXXX"); + int fd = mkstemp(tmp_file); + if (fd == -1) + { + throw std::runtime_error(strerror(errno)); + } + try { + if (write(fd, s.c_str(), s.size()) < s.size()) + { + throw std::runtime_error("unable write to temp file"); + } + if (write(fd, "\n", 1) != 1) + { + throw std::runtime_error("unable write to temp file"); + } + close(fd); + auto config_path = std::string(tmp_file) + ".xml"; + if (std::rename(tmp_file, config_path.c_str())) + { + int err = errno; + remove(tmp_file); + throw std::runtime_error(strerror(err)); + } + ConfigProcessor::LoadedConfig config = loadConfiguration(config_path); + remove(tmp_file); + return config; + } + catch (...) + { + remove(tmp_file); + throw; + } +} + +static Graphite::Params setGraphitePatterns(ContextMutablePtr context, ConfigProcessor::LoadedConfig & config) +{ + context->setConfig(config.configuration); + + Graphite::Params params; + setGraphitePatternsFromConfig(context, "graphite_rollup", params); + + return params; +} + +struct PatternForCheck +{ + Graphite::RuleType rule_type; + std::string regexp_str; + String function; + Graphite::Retentions retentions; +}; + + +bool checkRule(const Graphite::Pattern & pattern, const struct PatternForCheck & pattern_check, + const std::string & typ, const std::string & path, std::string & message) +{ + bool rule_type_eq = (pattern.rule_type == pattern_check.rule_type); + bool regexp_eq = (pattern.regexp_str == pattern_check.regexp_str); + bool function_eq = (pattern.function == nullptr && pattern_check.function.empty()) + || (pattern.function != nullptr && pattern.function->getName() == pattern_check.function); + bool retentions_eq = (pattern.retentions == pattern_check.retentions); + + if (rule_type_eq && regexp_eq && function_eq && retentions_eq) + return true; + + message = typ + " rollup rule mismatch for '" + path + "'," + + (rule_type_eq ? "" : "rule_type ") + + (regexp_eq ? "" : "regexp ") + + (function_eq ? "" : "function ") + + (retentions_eq ? "" : "retentions "); + return false; +} + +std::ostream & operator<<(std::ostream & stream, const PatternForCheck & a) +{ + stream << "{ rule_type = " << ruleTypeStr(a.rule_type); + if (!a.regexp_str.empty()) + stream << ", regexp = '" << a.regexp_str << "'"; + if (!a.function.empty()) + stream << ", function = " << a.function; + if (!a.retentions.empty()) + { + stream << ",\n retentions = {\n"; + for (size_t i = 0; i < a.retentions.size(); i++) + { + stream << " { " << a.retentions[i].age << ", " << a.retentions[i].precision << " }"; + if (i < a.retentions.size() - 1) + stream << ","; + stream << "\n"; + } + stream << " }\n"; + } + else + stream << " "; + + stream << "}"; + return stream; +} + +struct PatternsForPath +{ + std::string path; + PatternForCheck retention_want; + PatternForCheck aggregation_want; +}; + +TEST(GraphiteTest, testSelectPattern) +{ + tryRegisterAggregateFunctions(); + + using namespace std::literals; + + std::string + xml(R"END( + + + \.sum$ + sum + + + ^((.*)|.)sum\? + sum + + + \.max$ + max + + + ^((.*)|.)max\? + max + + + \.min$ + min + + + ^((.*)|.)min\? + min + + + \.(count|sum|sum_sq)$ + sum + + + ^((.*)|.)(count|sum|sum_sq)\? + sum + + + ^retention\. + + 0 + 60 + + + 86400 + 3600 + + + + avg + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + +)END"); + + // Retentions must be ordered by 'age' descending. + std::vector tests + { + { + "test.sum", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(\.sum$)END", "sum", { } } + }, + { + "val.sum?env=test&tag=Fake3", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(^((.*)|.)sum\?)END", "sum", { } } + }, + { + "test.max", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(\.max$)END", "max", { } }, + }, + { + "val.max?env=test&tag=Fake4", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(^((.*)|.)max\?)END", "max", { } }, + }, + { + "test.min", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(\.min$)END", "min", { } }, + }, + { + "val.min?env=test&tag=Fake5", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(^((.*)|.)min\?)END", "min", { } }, + }, + { + "retention.count", + { Graphite::RuleTypeAll, R"END(^retention\.)END", "", { { 86400, 3600 }, { 0, 60 } } }, // ^retention + { Graphite::RuleTypeAll, R"END(\.(count|sum|sum_sq)$)END", "sum", { } }, + }, + { + "val.retention.count?env=test&tag=Fake5", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "val.count?env=test&tag=Fake5", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "test.p95", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + }, + { + "val.p95?env=test&tag=FakeNo", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + }, + { + "default", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + }, + { + "val.default?env=test&tag=FakeNo", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + } + }; + + auto config = loadConfigurationFromString(xml); + ContextMutablePtr context = getContext().context; + Graphite::Params params = setGraphitePatterns(context, config); + + for (const auto & t : tests) + { + auto rule = DB::Graphite::selectPatternForPath(params, t.path); + std:: string message; + if (!checkRule(*rule.first, t.retention_want, "retention", t.path, message)) + ADD_FAILURE() << message << ", got\n" << *rule.first << "\n, want\n" << t.retention_want << "\n"; + if (!checkRule(*rule.second, t.aggregation_want, "aggregation", t.path, message)) + ADD_FAILURE() << message << ", got\n" << *rule.second << "\n, want\n" << t.aggregation_want << "\n"; + } +} + + +namespace DB::Graphite +{ + std::string buildTaggedRegex(std::string regexp_str); +} + +struct RegexCheck +{ + std::string regex; + std::string regex_want; + std::string match; + std::string nomatch; +}; + +TEST(GraphiteTest, testBuildTaggedRegex) +{ + std::vector tests + { + { + "cpu\\.loadavg;project=DB.*;env=st.*", + R"END(^cpu\.loadavg\?(.*&)?env=st.*&(.*&)?project=DB.*(&.*)?$)END", + R"END(cpu.loadavg?env=staging&project=DBAAS)END", + R"END(cpu.loadavg?env=staging&project=D)END" + }, + { + R"END(project=DB.*;env=staging;)END", + R"END([\?&]env=staging&(.*&)?project=DB.*(&.*)?$)END", + R"END(cpu.loadavg?env=staging&project=DBPG)END", + R"END(cpu.loadavg?env=stagingN&project=DBAAS)END" + }, + { + "env=staging;", + R"END([\?&]env=staging(&.*)?$)END", + R"END(cpu.loadavg?env=staging&project=DPG)END", + R"END(cpu.loadavg?env=stagingN)END" + }, + { + " env = staging ;", // spaces are allowed, + R"END([\?&] env = staging (&.*)?$)END", + R"END(cpu.loadavg? env = staging &project=DPG)END", + R"END(cpu.loadavg?env=stagingN)END" + }, + { + "name;", + R"END(^name\?)END", + R"END(name?env=staging&project=DPG)END", + R"END(nameN?env=stagingN)END", + }, + { + "name", + R"END(^name\?)END", + R"END(name?env=staging&project=DPG)END", + R"END(nameN?env=stagingN)END", + } + }; + for (const auto & t : tests) + { + auto s = DB::Graphite::buildTaggedRegex(t.regex); + EXPECT_EQ(t.regex_want, s) << "result for '" << t.regex_want << "' mismatch"; + auto regexp = OptimizedRegularExpression(s); + EXPECT_TRUE(regexp.match(t.match.data(), t.match.size())) << t.match << " match for '" << s << "' failed"; + EXPECT_FALSE(regexp.match(t.nomatch.data(), t.nomatch.size())) << t.nomatch << " ! match for '" << s << "' failed"; + } +} + +TEST(GraphiteTest, testSelectPatternTyped) +{ + tryRegisterAggregateFunctions(); + + using namespace std::literals; + + std::string + xml(R"END( + + + plain + \.sum$ + sum + + + tagged + ^((.*)|.)sum\? + sum + + + plain + \.max$ + max + + + tagged + ^((.*)|.)max\? + max + + + plain + \.min$ + min + + + tagged + ^((.*)|.)min\? + min + + + plain + \.(count|sum|sum_sq)$ + sum + + + tagged + ^((.*)|.)(count|sum|sum_sq)\? + sum + + + plain + ^retention\. + + 0 + 60 + + + 86400 + 3600 + + + + tagged + + + 0 + 60 + + + 86400 + 3600 + + + + tag_list + retention=10min;env=staging + + 0 + 600 + + + 86400 + 3600 + + + + tag_list + retention=10min;env=[A-Za-z-]+rod[A-Za-z-]+ + + 0 + 600 + + + 86400 + 3600 + + + + tag_list + cpu\.loadavg + + 0 + 600 + + + 86400 + 3600 + + + + avg + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + +)END"); + + // Retentions must be ordered by 'age' descending. + std::vector tests + { + { + "test.sum", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypePlain, R"END(\.sum$)END", "sum", { } } + }, + { + "val.sum?env=test&tag=Fake3", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeTagged, R"END(^((.*)|.)sum\?)END", "sum", { } } + }, + { + "test.max", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypePlain, R"END(\.max$)END", "max", { } }, + }, + { + "val.max?env=test&tag=Fake4", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeTagged, R"END(^((.*)|.)max\?)END", "max", { } }, + }, + { + "test.min", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypePlain, R"END(\.min$)END", "min", { } }, + }, + { + "val.min?env=test&tag=Fake5", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeTagged, R"END(^((.*)|.)min\?)END", "min", { } }, + }, + { + "retention.count", + { Graphite::RuleTypePlain, R"END(^retention\.)END", "", { { 86400, 3600 }, { 0, 60 } } }, // ^retention + { Graphite::RuleTypePlain, R"END(\.(count|sum|sum_sq)$)END", "sum", { } }, + }, + { + "val.count?env=test&retention=hour&tag=Fake5", + { Graphite::RuleTypeTagged, R"END([\?&]retention=hour(&.*)?$)END", "", { { 86400, 3600 }, { 0, 60 } } }, // tagged retention=hour + { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "val.count?env=test&retention=hour", + { Graphite::RuleTypeTagged, R"END([\?&]retention=hour(&.*)?$)END", "", { { 86400, 3600 }, { 0, 60 } } }, // tagged retention=hour + { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "val.count?env=staging&retention=10min", + { Graphite::RuleTypeTagged, R"END([\?&]env=staging&(.*&)?retention=10min(&.*)?$)END", "", { { 86400, 3600 }, { 0, 600 } } }, // retention=10min ; env=staging + { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "val.count?env=production&retention=10min", + { Graphite::RuleTypeTagged, R"END([\?&]env=[A-Za-z-]+rod[A-Za-z-]+&(.*&)?retention=10min(&.*)?$)END", "", { { 86400, 3600 }, { 0, 600 } } }, // retention=10min ; env=[A-Za-z-]+rod[A-Za-z-]+ + { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "val.count?env=test&tag=Fake5", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, + }, + { + "cpu.loadavg?env=test&tag=FakeNo", + { Graphite::RuleTypeTagged, R"END(^cpu\.loadavg\?)END", "", { { 86400, 3600 }, { 0, 600 } } }, // name=cpu\.loadavg + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, + }, + { + "test.p95", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + }, + { + "val.p95?env=test&tag=FakeNo", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + }, + { + "default", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + }, + { + "val.default?env=test&tag=FakeNo", + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default + } + }; + + auto config = loadConfigurationFromString(xml); + ContextMutablePtr context = getContext().context; + Graphite::Params params = setGraphitePatterns(context, config); + + for (const auto & t : tests) + { + auto rule = DB::Graphite::selectPatternForPath(params, t.path); + std:: string message; + if (!checkRule(*rule.first, t.retention_want, "retention", t.path, message)) + ADD_FAILURE() << message << ", got\n" << *rule.first << "\n, want\n" << t.retention_want << "\n"; + if (!checkRule(*rule.second, t.aggregation_want, "aggregation", t.path, message)) + ADD_FAILURE() << message << ", got\n" << *rule.second << "\n, want\n" << t.aggregation_want << "\n"; + } +} diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index cb52c8b86c0..ac6f4d8b7a4 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -22,17 +22,13 @@ #include #include -#include namespace DB { namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int NO_ELEMENTS_IN_CONFIG; extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; } @@ -62,171 +58,6 @@ static Names extractColumnNames(const ASTPtr & node) } } -/** Is used to order Graphite::Retentions by age and precision descending. - * Throws exception if not both age and precision are less or greater then another. - */ -static bool compareRetentions(const Graphite::Retention & a, const Graphite::Retention & b) -{ - if (a.age > b.age && a.precision > b.precision) - { - return true; - } - else if (a.age < b.age && a.precision < b.precision) - { - return false; - } - String error_msg = "age and precision should only grow up: " - + std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs " - + std::to_string(b.age) + ":" + std::to_string(b.precision); - throw Exception( - error_msg, - ErrorCodes::BAD_ARGUMENTS); -} - -/** Read the settings for Graphite rollup from config. - * Example - * - * - * Path - * - * click_cost - * any - * - * 0 - * 3600 - * - * - * 86400 - * 60 - * - * - * - * max - * - * 0 - * 60 - * - * - * 3600 - * 300 - * - * - * 86400 - * 3600 - * - * - * - */ -static void appendGraphitePattern( - const Poco::Util::AbstractConfiguration & config, - const String & config_element, - Graphite::Patterns & out_patterns, - ContextPtr context) -{ - Graphite::Pattern pattern; - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_element, keys); - - for (const auto & key : keys) - { - if (key == "regexp") - { - pattern.regexp_str = config.getString(config_element + ".regexp"); - pattern.regexp = std::make_shared(pattern.regexp_str); - } - else if (key == "function") - { - String aggregate_function_name_with_params = config.getString(config_element + ".function"); - String aggregate_function_name; - Array params_row; - getAggregateFunctionNameAndParametersArray( - aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context); - - /// TODO Not only Float64 - AggregateFunctionProperties properties; - pattern.function = AggregateFunctionFactory::instance().get( - aggregate_function_name, {std::make_shared()}, params_row, properties); - } - else if (startsWith(key, "retention")) - { - pattern.retentions.emplace_back(Graphite::Retention{ - .age = config.getUInt(config_element + "." + key + ".age"), - .precision = config.getUInt(config_element + "." + key + ".precision")}); - } - else - throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - } - - if (!pattern.function && pattern.retentions.empty()) - throw Exception( - "At least one of an aggregate function or retention rules is mandatory for rollup patterns in GraphiteMergeTree", - ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - if (!pattern.function) - { - pattern.type = pattern.TypeRetention; - } - else if (pattern.retentions.empty()) - { - pattern.type = pattern.TypeAggregation; - } - else - { - pattern.type = pattern.TypeAll; - } - - if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll - if (pattern.function->allocatesMemoryInArena()) - throw Exception( - "Aggregate function " + pattern.function->getName() + " isn't supported in GraphiteMergeTree", ErrorCodes::NOT_IMPLEMENTED); - - /// retention should be in descending order of age. - if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll - std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions); - - out_patterns.emplace_back(pattern); -} - -static void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params) -{ - const auto & config = context->getConfigRef(); - - if (!config.has(config_element)) - throw Exception("No '" + config_element + "' element in configuration file", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - params.config_name = config_element; - params.path_column_name = config.getString(config_element + ".path_column_name", "Path"); - params.time_column_name = config.getString(config_element + ".time_column_name", "Time"); - params.value_column_name = config.getString(config_element + ".value_column_name", "Value"); - params.version_column_name = config.getString(config_element + ".version_column_name", "Timestamp"); - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_element, keys); - - for (const auto & key : keys) - { - if (startsWith(key, "pattern")) - { - appendGraphitePattern(config, config_element + "." + key, params.patterns, context); - } - else if (key == "default") - { - /// See below. - } - else if (key == "path_column_name" || key == "time_column_name" || key == "value_column_name" || key == "version_column_name") - { - /// See above. - } - else - throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - } - - if (config.has(config_element + ".default")) - appendGraphitePattern(config, config_element + "." + ".default", params.patterns, context); -} - - static String getMergeTreeVerboseHelp(bool) { using namespace std::string_literals; @@ -542,12 +373,6 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// to make possible copying metadata files between replicas. Macros::MacroExpansionInfo info; info.table_id = args.table_id; - if (is_replicated_database) - { - auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); - info.shard = getReplicatedDatabaseShardName(database); - info.replica = getReplicatedDatabaseReplicaName(database); - } if (!allow_uuid_macro) info.table_id.uuid = UUIDHelpers::Nil; zookeeper_path = args.getContext()->getMacros()->expand(zookeeper_path, info); diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index dd592600d18..8711162385f 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -10,6 +10,7 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes() { return { {"config_name", std::make_shared()}, + {"rule_type", std::make_shared()}, {"regexp", std::make_shared()}, {"function", std::make_shared()}, {"age", std::make_shared()}, @@ -85,6 +86,7 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co bool is_default = pattern.regexp == nullptr; String regexp; String function; + const String & rule_type = ruleTypeStr(pattern.rule_type); if (is_default) { @@ -107,6 +109,7 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co { size_t i = 0; res_columns[i++]->insert(config.first); + res_columns[i++]->insert(rule_type); res_columns[i++]->insert(regexp); res_columns[i++]->insert(function); res_columns[i++]->insert(retention.age); @@ -121,6 +124,7 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co { size_t i = 0; res_columns[i++]->insert(config.first); + res_columns[i++]->insert(rule_type); res_columns[i++]->insert(regexp); res_columns[i++]->insert(function); res_columns[i++]->insertDefault(); diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index 3577553be34..ec3841f79d7 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -100,3 +100,19 @@ def exec_query_with_retry(instance, query, retry_count=40, sleep_time=0.5, silen time.sleep(sleep_time) else: raise exception + +def csv_compare(result, expected): + csv_result = TSV(result) + csv_expected = TSV(expected) + mismatch = [] + max_len = len(csv_result) if len(csv_result) > len(csv_expected) else len(csv_expected) + for i in range(max_len): + if i >= len(csv_result): + mismatch.append("-[%d]=%s" % (i, csv_expected.lines[i])) + elif i >= len(csv_expected): + mismatch.append("+[%d]=%s" % (i, csv_result.lines[i])) + elif csv_expected.lines[i] != csv_result.lines[i]: + mismatch.append("-[%d]=%s" % (i, csv_expected.lines[i])) + mismatch.append("+[%d]=%s" % (i, csv_result.lines[i])) + + return "\n".join(mismatch) diff --git a/tests/integration/test_graphite_merge_tree/test.py b/tests/integration/test_graphite_merge_tree/test.py index 7628211551d..9e48f12f007 100644 --- a/tests/integration/test_graphite_merge_tree/test.py +++ b/tests/integration/test_graphite_merge_tree/test.py @@ -6,6 +6,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV +from helpers.test_tools import csv_compare cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', @@ -234,18 +235,19 @@ SELECT * FROM test.graphite; def test_system_graphite_retentions(graphite_table): expected = ''' -graphite_rollup \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] -graphite_rollup \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] -graphite_rollup ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite'] -graphite_rollup ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite'] -graphite_rollup ^five_min\\\\. 0 300 3 0 ['test'] ['graphite'] -graphite_rollup ^one_min avg 31536000 600 4 0 ['test'] ['graphite'] -graphite_rollup ^one_min avg 7776000 300 4 0 ['test'] ['graphite'] -graphite_rollup ^one_min avg 0 60 4 0 ['test'] ['graphite'] +graphite_rollup all \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] +graphite_rollup all \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] +graphite_rollup all ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite'] +graphite_rollup all ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite'] +graphite_rollup all ^five_min\\\\. 0 300 3 0 ['test'] ['graphite'] +graphite_rollup all ^one_min avg 31536000 600 4 0 ['test'] ['graphite'] +graphite_rollup all ^one_min avg 7776000 300 4 0 ['test'] ['graphite'] +graphite_rollup all ^one_min avg 0 60 4 0 ['test'] ['graphite'] ''' result = q('SELECT * from system.graphite_retentions') - assert TSV(result) == TSV(expected) + mismatch = csv_compare(result, expected) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n" q(''' DROP TABLE IF EXISTS test.graphite2; diff --git a/tests/integration/test_graphite_merge_tree_typed/__init__.py b/tests/integration/test_graphite_merge_tree_typed/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml b/tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml new file mode 100644 index 00000000000..c716540a61c --- /dev/null +++ b/tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml @@ -0,0 +1,120 @@ + + + + metric + timestamp + value + updated + + plain + \.count$ + sum + + + plain + \.max$ + max + + + plain + ^five_min\. + + 0 + 300 + + + 5184000 + 3600 + + + 31536000 + 14400 + + + + plain + ^one_min + avg + + 0 + 60 + + + 7776000 + 300 + + + 31536000 + 600 + + + + tagged + + avg + + 0 + 60 + + + 7776000 + 300 + + + 31536000 + 600 + + + + tag_list + retention=five_min + avg + + 0 + 300 + + + 5184000 + 3600 + + + 31536000 + 14400 + + + + tagged + ^for_taggged + avg + + 0 + 60 + + + 7776000 + 300 + + + 31536000 + 600 + + + + all + ^ten_min\. + sum + + 0 + 600 + + + 5184000 + 7200 + + + 31536000 + 28800 + + + + diff --git a/tests/integration/test_graphite_merge_tree_typed/configs/users.xml b/tests/integration/test_graphite_merge_tree_typed/configs/users.xml new file mode 100644 index 00000000000..66d0cd7e445 --- /dev/null +++ b/tests/integration/test_graphite_merge_tree_typed/configs/users.xml @@ -0,0 +1,8 @@ + + + + + 0 + + + diff --git a/tests/integration/test_graphite_merge_tree_typed/test.py b/tests/integration/test_graphite_merge_tree_typed/test.py new file mode 100644 index 00000000000..e26fd0d2e77 --- /dev/null +++ b/tests/integration/test_graphite_merge_tree_typed/test.py @@ -0,0 +1,580 @@ +import datetime +import os.path as p +import time + +import sys +import pytest +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV +from helpers.test_tools import csv_compare + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', + main_configs=['configs/graphite_rollup.xml'], + user_configs=["configs/users.xml"]) +q = instance.query + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + q('CREATE DATABASE test') + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture +def graphite_table(started_cluster): + q(''' +DROP TABLE IF EXISTS test.graphite; +CREATE TABLE test.graphite + (metric String, value Float64, timestamp UInt32, date Date, updated UInt32) + ENGINE = GraphiteMergeTree('graphite_rollup') + PARTITION BY toYYYYMM(date) + ORDER BY (metric, timestamp) + SETTINGS index_granularity=8192; +''') + + yield + + q('DROP TABLE test.graphite') + + +def test_rollup_versions_plain(graphite_table): + timestamp = int(time.time()) + rounded_timestamp = timestamp - timestamp % 60 + date = datetime.date.today().isoformat() + + # Insert rows with timestamps relative to the current time so that the + # first retention clause is active. + # Two parts are created. + q(''' +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('one_min.x1', 100, {timestamp}, '{date}', 1); +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('one_min.x1', 200, {timestamp}, '{date}', 2); +'''.format(timestamp=timestamp, date=date)) + + expected1 = '''\ +one_min.x1 100 {timestamp} {date} 1 +one_min.x1 200 {timestamp} {date} 2 +'''.format(timestamp=timestamp, date=date) + + assert TSV( + q('SELECT * FROM test.graphite ORDER BY updated') + ) == TSV(expected1) + + q('OPTIMIZE TABLE test.graphite') + + # After rollup only the row with max version is retained. + expected2 = '''\ +one_min.x1 200 {timestamp} {date} 2 +'''.format(timestamp=rounded_timestamp, date=date) + + assert TSV(q('SELECT * FROM test.graphite')) == TSV(expected2) + + +def test_rollup_versions_tagged(graphite_table): + timestamp = int(time.time()) + rounded_timestamp = timestamp - timestamp % 60 + date = datetime.date.today().isoformat() + + # Insert rows with timestamps relative to the current time so that the + # first retention clause is active. + # Two parts are created. + q(''' +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('x1?retention=one_min', 100, {timestamp}, '{date}', 1); +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('x1?retention=one_min', 200, {timestamp}, '{date}', 2); +'''.format(timestamp=timestamp, date=date)) + + expected1 = '''\ +x1?retention=one_min 100 {timestamp} {date} 1 +x1?retention=one_min 200 {timestamp} {date} 2 +'''.format(timestamp=timestamp, date=date) + + result = q('SELECT * FROM test.graphite ORDER BY metric, updated') + mismatch = csv_compare(result, expected1) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected1}\ndiff\n{mismatch}\n" + + q('OPTIMIZE TABLE test.graphite') + + # After rollup only the row with max version is retained. + expected2 = '''\ +x1?retention=one_min 200 {timestamp} {date} 2 +'''.format(timestamp=rounded_timestamp, date=date) + + result = q('SELECT * FROM test.graphite ORDER BY metric, updated') + mismatch = csv_compare(result, expected2) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected2}\ndiff\n{mismatch}\n" + + +def test_rollup_versions_all(graphite_table): + timestamp = int(time.time()) + rounded_timestamp = timestamp - timestamp % 600 + date = datetime.date.today().isoformat() + + # Insert rows with timestamps relative to the current time so that the + # first retention clause is active. + # Two parts are created. + q(''' +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('ten_min.x1', 100, {timestamp}, '{date}', 1); +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('ten_min.x1', 200, {timestamp}, '{date}', 2); +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('ten_min.x1?env=staging', 100, {timestamp}, '{date}', 1); +INSERT INTO test.graphite (metric, value, timestamp, date, updated) + VALUES ('ten_min.x1?env=staging', 200, {timestamp}, '{date}', 2); +'''.format(timestamp=timestamp, date=date)) + + expected1 = '''\ +ten_min.x1 100 {timestamp} {date} 1 +ten_min.x1 200 {timestamp} {date} 2 +ten_min.x1?env=staging 100 {timestamp} {date} 1 +ten_min.x1?env=staging 200 {timestamp} {date} 2 +'''.format(timestamp=timestamp, date=date) + + result = q('SELECT * FROM test.graphite ORDER BY metric, updated') + mismatch = csv_compare(result, expected1) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected1}\ndiff\n{mismatch}\n" + + q('OPTIMIZE TABLE test.graphite') + + # After rollup only the row with max version is retained. + expected2 = '''\ +ten_min.x1 200 {timestamp} {date} 2 +ten_min.x1?env=staging 200 {timestamp} {date} 2 +'''.format(timestamp=rounded_timestamp, date=date) + + result = q('SELECT * FROM test.graphite ORDER BY metric, updated') + mismatch = csv_compare(result, expected2) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected2}\ndiff\n{mismatch}\n" + + +def test_rollup_aggregation_plain(graphite_table): + # This query essentially emulates what rollup does. + result1 = q(''' +SELECT avg(v), max(upd) +FROM (SELECT timestamp, + argMax(value, (updated, number)) AS v, + max(updated) AS upd + FROM (SELECT 'one_min.x5' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(intDiv(number, 2)) AS updated, + number + FROM system.numbers LIMIT 1000000) + WHERE intDiv(timestamp, 600) * 600 = 1111444200 + GROUP BY timestamp) +''') + + expected1 = '''\ +999634.9918367347 499999 +''' + assert TSV(result1) == TSV(expected1) + + # Timestamp 1111111111 is in sufficiently distant past + # so that the last retention clause is active. + result2 = q(''' +INSERT INTO test.graphite + SELECT 'one_min.x' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, + toDate('2017-02-02') AS date, toUInt32(intDiv(number, 2)) AS updated + FROM (SELECT * FROM system.numbers LIMIT 1000000) + WHERE intDiv(timestamp, 600) * 600 = 1111444200; + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; +''') + + expected2 = '''\ +one_min.x 999634.9918367347 1111444200 2017-02-02 499999 +''' + + assert TSV(result2) == TSV(expected2) + + +def test_rollup_aggregation_tagged(graphite_table): + # This query essentially emulates what rollup does. + result1 = q(''' +SELECT avg(v), max(upd) +FROM (SELECT timestamp, + argMax(value, (updated, number)) AS v, + max(updated) AS upd + FROM (SELECT 'x?retention=one_min' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(intDiv(number, 2)) AS updated, + number + FROM system.numbers LIMIT 1000000) + WHERE intDiv(timestamp, 600) * 600 = 1111444200 + GROUP BY timestamp) +''') + + expected1 = '''\ +999634.9918367347 499999 +''' + assert TSV(result1) == TSV(expected1) + + # Timestamp 1111111111 is in sufficiently distant past + # so that the last retention clause is active. + result2 = q(''' +INSERT INTO test.graphite + SELECT 'x?retention=one_min' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, + toDate('2017-02-02') AS date, toUInt32(intDiv(number, 2)) AS updated + FROM (SELECT * FROM system.numbers LIMIT 1000000) + WHERE intDiv(timestamp, 600) * 600 = 1111444200; + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; +''') + + expected2 = '''\ +x?retention=one_min 999634.9918367347 1111444200 2017-02-02 499999 +''' + + assert TSV(result2) == TSV(expected2) + + +def test_rollup_aggregation_2_plain(graphite_table): + result = q(''' +INSERT INTO test.graphite + SELECT 'one_min.x' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 - intDiv(number, 3)) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(100 - number) AS updated + FROM (SELECT * FROM system.numbers LIMIT 50); + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; +''') + + expected = '''\ +one_min.x 24 1111110600 2017-02-02 100 +''' + + assert TSV(result) == TSV(expected) + + +def test_rollup_aggregation_2_tagged(graphite_table): + result = q(''' +INSERT INTO test.graphite + SELECT 'x?retention=one_min' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 - intDiv(number, 3)) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(100 - number) AS updated + FROM (SELECT * FROM system.numbers LIMIT 50); + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; +''') + + expected = '''\ +x?retention=one_min 24 1111110600 2017-02-02 100 +''' + + assert TSV(result) == TSV(expected) + + +def test_multiple_paths_and_versions_plain(graphite_table): + result = q(''' +INSERT INTO test.graphite + SELECT 'one_min.x' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + intDiv(number, 3) * 600) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(100 - number) AS updated + FROM (SELECT * FROM system.numbers LIMIT 50); + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; + + +INSERT INTO test.graphite + SELECT 'one_min.y' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + number * 600) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(100 - number) AS updated + FROM (SELECT * FROM system.numbers LIMIT 50); + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; +''') + + with open(p.join(p.dirname(__file__), + 'test_multiple_paths_and_versions.reference.plain') + ) as reference: + assert TSV(result) == TSV(reference) + + +def test_multiple_paths_and_versions_tagged(graphite_table): + result = q(''' +INSERT INTO test.graphite + SELECT 'x?retention=one_min' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + intDiv(number, 3) * 600) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(100 - number) AS updated + FROM (SELECT * FROM system.numbers LIMIT 50); + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; + + +INSERT INTO test.graphite + SELECT 'y?retention=one_min' AS metric, + toFloat64(number) AS value, + toUInt32(1111111111 + number * 600) AS timestamp, + toDate('2017-02-02') AS date, + toUInt32(100 - number) AS updated + FROM (SELECT * FROM system.numbers LIMIT 50); + +OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; + +SELECT * FROM test.graphite; +''') + + with open(p.join(p.dirname(__file__), + 'test_multiple_paths_and_versions.reference.tagged') + ) as reference: + assert TSV(result) == TSV(reference) + + +def test_multiple_output_blocks(graphite_table): + MERGED_BLOCK_SIZE = 8192 + + to_insert = '' + expected = '' + for i in range(2 * MERGED_BLOCK_SIZE + 1): + rolled_up_time = 1000000200 + 600 * i + + for j in range(3): + cur_time = rolled_up_time + 100 * j + to_insert += 'one_min.x1 {} {} 2001-09-09 1\n'.format( + 10 * j, cur_time + ) + to_insert += 'one_min.x1 {} {} 2001-09-09 2\n'.format( + 10 * (j + 1), cur_time + ) + + expected += 'one_min.x1 20 {} 2001-09-09 2\n'.format(rolled_up_time) + + q('INSERT INTO test.graphite FORMAT TSV', to_insert) + + result = q(''' +OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL; + +SELECT * FROM test.graphite; +''') + + assert TSV(result) == TSV(expected) + + +def test_paths_not_matching_any_pattern(graphite_table): + to_insert = '''\ +one_min.x1 100 1000000000 2001-09-09 1 +zzzzzzzz 100 1000000001 2001-09-09 1 +zzzzzzzz 200 1000000001 2001-09-09 2 +''' + + q('INSERT INTO test.graphite FORMAT TSV', to_insert) + + expected = '''\ +one_min.x1 100 999999600 2001-09-09 1 +zzzzzzzz 200 1000000001 2001-09-09 2 +''' + + result = q(''' +OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL; + +SELECT * FROM test.graphite; +''') + + assert TSV(result) == TSV(expected) + + +def test_rules_isolation(graphite_table): + to_insert = '''\ +one_min.x1 100 1000000000 2001-09-09 1 +for_taggged 100 1000000001 2001-09-09 1 +for_taggged 200 1000000001 2001-09-09 2 +one_min?env=staging 100 1000000001 2001-09-09 1 +one_min?env=staging 200 1000000001 2001-09-09 2 +''' + + q('INSERT INTO test.graphite FORMAT TSV', to_insert) + + expected = '''\ +for_taggged 200 1000000001 2001-09-09 2 +one_min.x1 100 999999600 2001-09-09 1 +one_min?env=staging 200 1000000001 2001-09-09 2 +''' + + result = q(''' +OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL; + +SELECT * FROM test.graphite; +''') + + result = q('SELECT * FROM test.graphite ORDER BY metric, updated') + mismatch = csv_compare(result, expected) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n" + + +def test_system_graphite_retentions(graphite_table): + expected = ''' +graphite_rollup plain \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] +graphite_rollup plain \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] +graphite_rollup plain ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite'] +graphite_rollup plain ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite'] +graphite_rollup plain ^five_min\\\\. 0 300 3 0 ['test'] ['graphite'] +graphite_rollup plain ^one_min avg 31536000 600 4 0 ['test'] ['graphite'] +graphite_rollup plain ^one_min avg 7776000 300 4 0 ['test'] ['graphite'] +graphite_rollup plain ^one_min avg 0 60 4 0 ['test'] ['graphite'] +graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 31536000 600 5 0 ['test'] ['graphite'] +graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 7776000 300 5 0 ['test'] ['graphite'] +graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 0 60 5 0 ['test'] ['graphite'] +graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 31536000 14400 6 0 ['test'] ['graphite'] +graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 5184000 3600 6 0 ['test'] ['graphite'] +graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 0 300 6 0 ['test'] ['graphite'] +graphite_rollup tagged ^for_taggged avg 31536000 600 7 0 ['test'] ['graphite'] +graphite_rollup tagged ^for_taggged avg 7776000 300 7 0 ['test'] ['graphite'] +graphite_rollup tagged ^for_taggged avg 0 60 7 0 ['test'] ['graphite'] +graphite_rollup all ^ten_min\\\\. sum 31536000 28800 8 0 ['test'] ['graphite'] +graphite_rollup all ^ten_min\\\\. sum 5184000 7200 8 0 ['test'] ['graphite'] +graphite_rollup all ^ten_min\\\\. sum 0 600 8 0 ['test'] ['graphite'] + ''' + result = q('SELECT * from system.graphite_retentions') + + mismatch = csv_compare(result, expected) + assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n" + + q(''' +DROP TABLE IF EXISTS test.graphite2; +CREATE TABLE test.graphite2 + (metric String, value Float64, timestamp UInt32, date Date, updated UInt32) + ENGINE = GraphiteMergeTree('graphite_rollup') + PARTITION BY toYYYYMM(date) + ORDER BY (metric, timestamp) + SETTINGS index_granularity=8192; + ''') + expected = ''' +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] +graphite_rollup ['test','test'] ['graphite','graphite2'] + ''' + result = q(''' + SELECT + config_name, + Tables.database, + Tables.table + FROM system.graphite_retentions + ''') + assert csv_compare(result, expected), f"got\n{result}\nwant\n{expected}" + + +def test_path_dangling_pointer(graphite_table): + q(''' +DROP TABLE IF EXISTS test.graphite2; +CREATE TABLE test.graphite2 + (metric String, value Float64, timestamp UInt32, date Date, updated UInt32) + ENGINE = GraphiteMergeTree('graphite_rollup') + PARTITION BY toYYYYMM(date) + ORDER BY (metric, timestamp) + SETTINGS index_granularity=1; + ''') + + path = 'abcd' * 4000000 # 16MB + q('INSERT INTO test.graphite2 FORMAT TSV', + "{}\t0.0\t0\t2018-01-01\t100\n".format(path)) + q('INSERT INTO test.graphite2 FORMAT TSV', + "{}\t0.0\t0\t2018-01-01\t101\n".format(path)) + for version in range(10): + q('INSERT INTO test.graphite2 FORMAT TSV', + "{}\t0.0\t0\t2018-01-01\t{}\n".format(path, version)) + + while True: + q('OPTIMIZE TABLE test.graphite2 PARTITION 201801 FINAL') + parts = int(q("SELECT count() FROM system.parts " + "WHERE active AND database='test' " + "AND table='graphite2'")) + if parts == 1: + break + print(('Parts', parts)) + + assert TSV( + q("SELECT value, timestamp, date, updated FROM test.graphite2") + ) == TSV("0\t0\t2018-01-01\t101\n") + + q('DROP TABLE test.graphite2') + + +def test_combined_rules(graphite_table): + # 1487970000 ~ Sat 25 Feb 00:00:00 MSK 2017 + to_insert = 'INSERT INTO test.graphite VALUES ' + expected_unmerged = '' + for i in range(384): + to_insert += "('five_min.count', {v}, {t}, toDate({t}), 1), ".format( + v=1, t=1487970000 + (i * 300) + ) + to_insert += "('five_min.max', {v}, {t}, toDate({t}), 1), ".format( + v=i, t=1487970000 + (i * 300) + ) + expected_unmerged += ("five_min.count\t{v1}\t{t}\n" + "five_min.max\t{v2}\t{t}\n").format( + v1=1, v2=i, + t=1487970000 + (i * 300) + ) + + q(to_insert) + assert TSV(q('SELECT metric, value, timestamp FROM test.graphite' + ' ORDER BY (timestamp, metric)')) == TSV(expected_unmerged) + + q('OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL') + expected_merged = ''' + five_min.count 48 1487970000 2017-02-25 1 + five_min.count 48 1487984400 2017-02-25 1 + five_min.count 48 1487998800 2017-02-25 1 + five_min.count 48 1488013200 2017-02-25 1 + five_min.count 48 1488027600 2017-02-25 1 + five_min.count 48 1488042000 2017-02-25 1 + five_min.count 48 1488056400 2017-02-26 1 + five_min.count 48 1488070800 2017-02-26 1 + five_min.max 47 1487970000 2017-02-25 1 + five_min.max 95 1487984400 2017-02-25 1 + five_min.max 143 1487998800 2017-02-25 1 + five_min.max 191 1488013200 2017-02-25 1 + five_min.max 239 1488027600 2017-02-25 1 + five_min.max 287 1488042000 2017-02-25 1 + five_min.max 335 1488056400 2017-02-26 1 + five_min.max 383 1488070800 2017-02-26 1 + ''' + assert TSV(q('SELECT * FROM test.graphite' + ' ORDER BY (metric, timestamp)')) == TSV(expected_merged) diff --git a/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain b/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain new file mode 100644 index 00000000000..0f10d11ed05 --- /dev/null +++ b/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain @@ -0,0 +1,84 @@ +one_min.x 0 1111110600 2017-02-02 100 +one_min.x 3 1111111200 2017-02-02 97 +one_min.x 6 1111111800 2017-02-02 94 +one_min.x 9 1111112400 2017-02-02 91 +one_min.x 12 1111113000 2017-02-02 88 +one_min.x 15 1111113600 2017-02-02 85 +one_min.x 18 1111114200 2017-02-02 82 +one_min.x 21 1111114800 2017-02-02 79 +one_min.x 24 1111115400 2017-02-02 76 +one_min.x 27 1111116000 2017-02-02 73 +one_min.x 30 1111116600 2017-02-02 70 +one_min.x 33 1111117200 2017-02-02 67 +one_min.x 36 1111117800 2017-02-02 64 +one_min.x 39 1111118400 2017-02-02 61 +one_min.x 42 1111119000 2017-02-02 58 +one_min.x 45 1111119600 2017-02-02 55 +one_min.x 48 1111120200 2017-02-02 52 +one_min.x 0 1111110600 2017-02-02 100 +one_min.x 3 1111111200 2017-02-02 97 +one_min.x 6 1111111800 2017-02-02 94 +one_min.x 9 1111112400 2017-02-02 91 +one_min.x 12 1111113000 2017-02-02 88 +one_min.x 15 1111113600 2017-02-02 85 +one_min.x 18 1111114200 2017-02-02 82 +one_min.x 21 1111114800 2017-02-02 79 +one_min.x 24 1111115400 2017-02-02 76 +one_min.x 27 1111116000 2017-02-02 73 +one_min.x 30 1111116600 2017-02-02 70 +one_min.x 33 1111117200 2017-02-02 67 +one_min.x 36 1111117800 2017-02-02 64 +one_min.x 39 1111118400 2017-02-02 61 +one_min.x 42 1111119000 2017-02-02 58 +one_min.x 45 1111119600 2017-02-02 55 +one_min.x 48 1111120200 2017-02-02 52 +one_min.y 0 1111110600 2017-02-02 100 +one_min.y 1 1111111200 2017-02-02 99 +one_min.y 2 1111111800 2017-02-02 98 +one_min.y 3 1111112400 2017-02-02 97 +one_min.y 4 1111113000 2017-02-02 96 +one_min.y 5 1111113600 2017-02-02 95 +one_min.y 6 1111114200 2017-02-02 94 +one_min.y 7 1111114800 2017-02-02 93 +one_min.y 8 1111115400 2017-02-02 92 +one_min.y 9 1111116000 2017-02-02 91 +one_min.y 10 1111116600 2017-02-02 90 +one_min.y 11 1111117200 2017-02-02 89 +one_min.y 12 1111117800 2017-02-02 88 +one_min.y 13 1111118400 2017-02-02 87 +one_min.y 14 1111119000 2017-02-02 86 +one_min.y 15 1111119600 2017-02-02 85 +one_min.y 16 1111120200 2017-02-02 84 +one_min.y 17 1111120800 2017-02-02 83 +one_min.y 18 1111121400 2017-02-02 82 +one_min.y 19 1111122000 2017-02-02 81 +one_min.y 20 1111122600 2017-02-02 80 +one_min.y 21 1111123200 2017-02-02 79 +one_min.y 22 1111123800 2017-02-02 78 +one_min.y 23 1111124400 2017-02-02 77 +one_min.y 24 1111125000 2017-02-02 76 +one_min.y 25 1111125600 2017-02-02 75 +one_min.y 26 1111126200 2017-02-02 74 +one_min.y 27 1111126800 2017-02-02 73 +one_min.y 28 1111127400 2017-02-02 72 +one_min.y 29 1111128000 2017-02-02 71 +one_min.y 30 1111128600 2017-02-02 70 +one_min.y 31 1111129200 2017-02-02 69 +one_min.y 32 1111129800 2017-02-02 68 +one_min.y 33 1111130400 2017-02-02 67 +one_min.y 34 1111131000 2017-02-02 66 +one_min.y 35 1111131600 2017-02-02 65 +one_min.y 36 1111132200 2017-02-02 64 +one_min.y 37 1111132800 2017-02-02 63 +one_min.y 38 1111133400 2017-02-02 62 +one_min.y 39 1111134000 2017-02-02 61 +one_min.y 40 1111134600 2017-02-02 60 +one_min.y 41 1111135200 2017-02-02 59 +one_min.y 42 1111135800 2017-02-02 58 +one_min.y 43 1111136400 2017-02-02 57 +one_min.y 44 1111137000 2017-02-02 56 +one_min.y 45 1111137600 2017-02-02 55 +one_min.y 46 1111138200 2017-02-02 54 +one_min.y 47 1111138800 2017-02-02 53 +one_min.y 48 1111139400 2017-02-02 52 +one_min.y 49 1111140000 2017-02-02 51 diff --git a/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged b/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged new file mode 100644 index 00000000000..e2c63ab3b22 --- /dev/null +++ b/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged @@ -0,0 +1,84 @@ +x?retention=one_min 0 1111110600 2017-02-02 100 +x?retention=one_min 3 1111111200 2017-02-02 97 +x?retention=one_min 6 1111111800 2017-02-02 94 +x?retention=one_min 9 1111112400 2017-02-02 91 +x?retention=one_min 12 1111113000 2017-02-02 88 +x?retention=one_min 15 1111113600 2017-02-02 85 +x?retention=one_min 18 1111114200 2017-02-02 82 +x?retention=one_min 21 1111114800 2017-02-02 79 +x?retention=one_min 24 1111115400 2017-02-02 76 +x?retention=one_min 27 1111116000 2017-02-02 73 +x?retention=one_min 30 1111116600 2017-02-02 70 +x?retention=one_min 33 1111117200 2017-02-02 67 +x?retention=one_min 36 1111117800 2017-02-02 64 +x?retention=one_min 39 1111118400 2017-02-02 61 +x?retention=one_min 42 1111119000 2017-02-02 58 +x?retention=one_min 45 1111119600 2017-02-02 55 +x?retention=one_min 48 1111120200 2017-02-02 52 +x?retention=one_min 0 1111110600 2017-02-02 100 +x?retention=one_min 3 1111111200 2017-02-02 97 +x?retention=one_min 6 1111111800 2017-02-02 94 +x?retention=one_min 9 1111112400 2017-02-02 91 +x?retention=one_min 12 1111113000 2017-02-02 88 +x?retention=one_min 15 1111113600 2017-02-02 85 +x?retention=one_min 18 1111114200 2017-02-02 82 +x?retention=one_min 21 1111114800 2017-02-02 79 +x?retention=one_min 24 1111115400 2017-02-02 76 +x?retention=one_min 27 1111116000 2017-02-02 73 +x?retention=one_min 30 1111116600 2017-02-02 70 +x?retention=one_min 33 1111117200 2017-02-02 67 +x?retention=one_min 36 1111117800 2017-02-02 64 +x?retention=one_min 39 1111118400 2017-02-02 61 +x?retention=one_min 42 1111119000 2017-02-02 58 +x?retention=one_min 45 1111119600 2017-02-02 55 +x?retention=one_min 48 1111120200 2017-02-02 52 +y?retention=one_min 0 1111110600 2017-02-02 100 +y?retention=one_min 1 1111111200 2017-02-02 99 +y?retention=one_min 2 1111111800 2017-02-02 98 +y?retention=one_min 3 1111112400 2017-02-02 97 +y?retention=one_min 4 1111113000 2017-02-02 96 +y?retention=one_min 5 1111113600 2017-02-02 95 +y?retention=one_min 6 1111114200 2017-02-02 94 +y?retention=one_min 7 1111114800 2017-02-02 93 +y?retention=one_min 8 1111115400 2017-02-02 92 +y?retention=one_min 9 1111116000 2017-02-02 91 +y?retention=one_min 10 1111116600 2017-02-02 90 +y?retention=one_min 11 1111117200 2017-02-02 89 +y?retention=one_min 12 1111117800 2017-02-02 88 +y?retention=one_min 13 1111118400 2017-02-02 87 +y?retention=one_min 14 1111119000 2017-02-02 86 +y?retention=one_min 15 1111119600 2017-02-02 85 +y?retention=one_min 16 1111120200 2017-02-02 84 +y?retention=one_min 17 1111120800 2017-02-02 83 +y?retention=one_min 18 1111121400 2017-02-02 82 +y?retention=one_min 19 1111122000 2017-02-02 81 +y?retention=one_min 20 1111122600 2017-02-02 80 +y?retention=one_min 21 1111123200 2017-02-02 79 +y?retention=one_min 22 1111123800 2017-02-02 78 +y?retention=one_min 23 1111124400 2017-02-02 77 +y?retention=one_min 24 1111125000 2017-02-02 76 +y?retention=one_min 25 1111125600 2017-02-02 75 +y?retention=one_min 26 1111126200 2017-02-02 74 +y?retention=one_min 27 1111126800 2017-02-02 73 +y?retention=one_min 28 1111127400 2017-02-02 72 +y?retention=one_min 29 1111128000 2017-02-02 71 +y?retention=one_min 30 1111128600 2017-02-02 70 +y?retention=one_min 31 1111129200 2017-02-02 69 +y?retention=one_min 32 1111129800 2017-02-02 68 +y?retention=one_min 33 1111130400 2017-02-02 67 +y?retention=one_min 34 1111131000 2017-02-02 66 +y?retention=one_min 35 1111131600 2017-02-02 65 +y?retention=one_min 36 1111132200 2017-02-02 64 +y?retention=one_min 37 1111132800 2017-02-02 63 +y?retention=one_min 38 1111133400 2017-02-02 62 +y?retention=one_min 39 1111134000 2017-02-02 61 +y?retention=one_min 40 1111134600 2017-02-02 60 +y?retention=one_min 41 1111135200 2017-02-02 59 +y?retention=one_min 42 1111135800 2017-02-02 58 +y?retention=one_min 43 1111136400 2017-02-02 57 +y?retention=one_min 44 1111137000 2017-02-02 56 +y?retention=one_min 45 1111137600 2017-02-02 55 +y?retention=one_min 46 1111138200 2017-02-02 54 +y?retention=one_min 47 1111138800 2017-02-02 53 +y?retention=one_min 48 1111139400 2017-02-02 52 +y?retention=one_min 49 1111140000 2017-02-02 51 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 2b391cd292e..a2e56fa0f1d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -21,7 +21,7 @@ CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `de CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `rule_type` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 8309b6bcb53..a930e7db3fc 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -32,6 +32,7 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (wal-dump) add_subdirectory (check-mysql-binlog) add_subdirectory (keeper-bench) + add_subdirectory (graphite-rollup) if (USE_NURAFT) add_subdirectory (keeper-data-dumper) diff --git a/utils/graphite-rollup/CMakeLists.txt b/utils/graphite-rollup/CMakeLists.txt new file mode 100644 index 00000000000..bd6a078fbd5 --- /dev/null +++ b/utils/graphite-rollup/CMakeLists.txt @@ -0,0 +1,23 @@ +add_executable(graphite-rollup-bench graphite-rollup-bench.cpp) +target_link_libraries( + graphite-rollup-bench + PRIVATE + clickhouse_storages_system + clickhouse_aggregate_functions + clickhouse_common_config + dbms +) +target_include_directories( + graphite-rollup-bench + PRIVATE + ${ClickHouse_SOURCE_DIR}/src ${CMAKE_BINARY_DIR}/src + ${ClickHouse_SOURCE_DIR}/base ${ClickHouse_SOURCE_DIR}/base/pcg-random + ${CMAKE_BINARY_DIR}/src/Core/include + ${POCO_INCLUDE_DIR} + ${ClickHouse_SOURCE_DIR}/contrib/double-conversion ${ClickHouse_SOURCE_DIR}/contrib/dragonbox/include + ${ClickHouse_SOURCE_DIR}/contrib/fmtlib/include + ${ClickHouse_SOURCE_DIR}/contrib/cityhash102/include + ${RE2_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/re2_st +) + +target_compile_definitions(graphite-rollup-bench PRIVATE RULES_DIR="${CMAKE_CURRENT_SOURCE_DIR}") diff --git a/utils/graphite-rollup/graphite-rollup-bench.cpp b/utils/graphite-rollup/graphite-rollup-bench.cpp new file mode 100644 index 00000000000..dabe0353b0f --- /dev/null +++ b/utils/graphite-rollup/graphite-rollup-bench.cpp @@ -0,0 +1,147 @@ +#include +#include +#include +#include +#include +#include +#include + +#include + +#include +#include +#include +#include +#include +#include +#include + +using namespace DB; + +static SharedContextHolder shared_context = Context::createShared(); + +std::vector loadMetrics(const std::string & metrics_file) +{ + std::vector metrics; + + FILE * stream; + char * line = nullptr; + size_t len = 0; + ssize_t nread; + + stream = fopen(metrics_file.c_str(), "r"); + if (stream == nullptr) + { + throw std::runtime_error(strerror(errno)); + } + + while ((nread = getline(&line, &len, stream)) != -1) + { + size_t l = strlen(line); + if (l > 0) + { + if (line[l - 1] == '\n') + { + line[l - 1] = '\0'; + l--; + } + if (l > 0) + { + metrics.push_back(StringRef(strdup(line), l)); + } + } + } + free(line); + if (ferror(stream)) + { + fclose(stream); + throw std::runtime_error(strerror(errno)); + } + + fclose(stream); + + return metrics; +} + +ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path) +{ + ConfigProcessor config_processor(config_path, true, true); + ConfigProcessor::LoadedConfig config = config_processor.loadConfig(false); + return config; +} + +void bench(const std::string & config_path, const std::string & metrics_file, size_t n, bool verbose) +{ + auto config = loadConfiguration(config_path); + + auto context = Context::createGlobal(shared_context.get()); + context->setConfig(config.configuration.get()); + + Graphite::Params params; + setGraphitePatternsFromConfig(context, "graphite_rollup", params); + + std::vector metrics = loadMetrics(metrics_file); + + std::vector durations(metrics.size()); + size_t j, i; + for (j = 0; j < n; j++) + { + for (i = 0; i < metrics.size(); i++) + { + auto start = std::chrono::high_resolution_clock::now(); + + auto rule = DB::Graphite::selectPatternForPath(params, metrics[i]); + (void)rule; + + auto end = std::chrono::high_resolution_clock::now(); + double duration = (duration_cast>(end - start)).count() * 1E9; + durations[i] += duration; + + if (j == 0 && verbose) + { + std::cout << metrics[i].data << ": rule with regexp '" << rule.second->regexp_str << "' found\n"; + } + } + } + + for (i = 0; i < metrics.size(); i++) + { + std::cout << metrics[i].data << " " << durations[i] / n << " ns\n"; + free(const_cast(static_cast(metrics[i].data))); + } +} + +int main(int argc, char ** argv) +{ + registerAggregateFunctions(); + + std::string config_file, metrics_file; + + using namespace std::literals; + + std::string config_default = RULES_DIR + "/rollup.xml"s; + std::string metrics_default = RULES_DIR + "/metrics.txt"s; + + namespace po = boost::program_options; + po::variables_map vm; + + po::options_description desc; + desc.add_options()("help,h", "produce help")( + "config,c", po::value()->default_value(config_default), "XML config with rollup rules")( + "metrics,m", po::value()->default_value(metrics_default), "metrcis files (one metric per line) for run benchmark")( + "verbose,V", po::bool_switch()->default_value(false), "verbose output (print found rule)"); + + po::parsed_options parsed = po::command_line_parser(argc, argv).options(desc).run(); + po::store(parsed, vm); + po::notify(vm); + + if (vm.count("help")) + { + std::cout << desc << '\n'; + exit(1); + } + + bench(vm["config"].as(), vm["metrics"].as(), 10000, vm["verbose"].as()); + + return 0; +} diff --git a/utils/graphite-rollup/metrics.txt b/utils/graphite-rollup/metrics.txt new file mode 100644 index 00000000000..199c3791310 --- /dev/null +++ b/utils/graphite-rollup/metrics.txt @@ -0,0 +1,11 @@ +test.sum +sum?env=test&tag=Fake3 +test.max +max?env=test&tag=Fake4 +test.min +min?env=test&tag=Fake5 +fake5?env=test&tag=Fake5 +test.p95 +p95?env=test&tag=FakeNo +default +default?env=test&tag=FakeNo diff --git a/utils/graphite-rollup/rollup-tag-list.xml b/utils/graphite-rollup/rollup-tag-list.xml new file mode 100644 index 00000000000..ef28f2089ad --- /dev/null +++ b/utils/graphite-rollup/rollup-tag-list.xml @@ -0,0 +1,167 @@ + + + + plain + \.sum$ + sum + + 0 + 60 + + + 86400 + 3600 + + + + tagged + ^((.*)|.)sum\? + sum + + 0 + 60 + + + 86400 + 3600 + + + + plain + \.max$ + max + + 0 + 60 + + + 86400 + 3600 + + + + tagged + ^((.*)|.)max\? + max + + 0 + 60 + + + 86400 + 3600 + + + + plain + \.min$ + min + + 0 + 60 + + + 86400 + 3600 + + + + tagged + ^((.*)|.)min\? + min + + 0 + 60 + + + 86400 + 3600 + + + + plain + \.fake1\..*\.Fake1\. + sum + + + tag_list + fake1;tag=Fake1 + sum + + + plain + \.fake2\..*\.Fake2\. + sum + + + tag_list + fake2;tag=Fake2 + sum + + + plain + \.fake3\..*\.Fake3\. + sum + + + tag_list + fake3;tag=Fake3 + sum + + + plain + \.fake4\..*\.Fake4\. + sum + + + tag_list + fake4;tag=Fake4 + sum + + + plain + \.fake5\..*\.Fake5\. + sum + + + tag_list + fake5;tag=Fake5 + sum + + + plain + \.fake6\..*\.Fake6\. + sum + + + tag_list + fake6;tag=Fake6 + sum + + + plain + \.fake7\..*\.Fake7\. + sum + + + tag_list + fake7;tag=Fake7 + sum + + + avg + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + diff --git a/utils/graphite-rollup/rollup-typed.xml b/utils/graphite-rollup/rollup-typed.xml new file mode 100644 index 00000000000..0b27d43ece9 --- /dev/null +++ b/utils/graphite-rollup/rollup-typed.xml @@ -0,0 +1,167 @@ + + + + plain + \.sum$ + sum + + 0 + 60 + + + 86400 + 3600 + + + + tagged + ^((.*)|.)sum\? + sum + + 0 + 60 + + + 86400 + 3600 + + + + plain + \.max$ + max + + 0 + 60 + + + 86400 + 3600 + + + + tagged + ^((.*)|.)max\? + max + + 0 + 60 + + + 86400 + 3600 + + + + plain + \.min$ + min + + 0 + 60 + + + 86400 + 3600 + + + + tagged + ^((.*)|.)min\? + min + + 0 + 60 + + + 86400 + 3600 + + + + plain + \.fake1\..*\.Fake1\. + sum + + + tagged + + sum + + + plain + \.fake2\..*\.Fake2\. + sum + + + tagged + + sum + + + plain + \.fake3\..*\.Fake3\. + sum + + + tagged + + sum + + + plain + \.fake4\..*\.Fake4\. + sum + + + tagged + + sum + + + plain + \.fake5\..*\.Fake5\. + sum + + + tagged + + sum + + + plain + \.fake6\..*\.Fake6\. + sum + + + tagged + + sum + + + plain + \.fake7\..*\.Fake7\. + sum + + + tagged + + sum + + + avg + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + diff --git a/utils/graphite-rollup/rollup.xml b/utils/graphite-rollup/rollup.xml new file mode 100644 index 00000000000..641b0130509 --- /dev/null +++ b/utils/graphite-rollup/rollup.xml @@ -0,0 +1,147 @@ + + + + \.sum$ + sum + + 0 + 60 + + + 86400 + 3600 + + + + ^((.*)|.)sum\? + sum + + 0 + 60 + + + 86400 + 3600 + + + + \.max$ + max + + 0 + 60 + + + 86400 + 3600 + + + + ^((.*)|.)max\? + max + + 0 + 60 + + + 86400 + 3600 + + + + \.min$ + min + + 0 + 60 + + + 86400 + 3600 + + + + ^((.*)|.)min\? + min + + 0 + 60 + + + 86400 + 3600 + + + + \.fake1\..*\.Fake1\. + sum + + + + sum + + + \.fake2\..*\.Fake2\. + sum + + + + sum + + + \.fake3\..*\.Fake3\. + sum + + + + sum + + + \.fake4\..*\.Fake4\. + sum + + + + sum + + + \.fake5\..*\.Fake5\. + sum + + + + sum + + + \.fake6\..*\.Fake6\. + sum + + + + sum + + + \.fake7\..*\.Fake7\. + sum + + + + sum + + + avg + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + From c92cfc38d44f73a77821e37b71f8d040d7f1b914 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Dec 2021 19:47:30 +0300 Subject: [PATCH 553/609] Fix possible crash in DataTypeAggregateFunction (#32287) --- src/DataTypes/DataTypeAggregateFunction.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypeAggregateFunction.cpp b/src/DataTypes/DataTypeAggregateFunction.cpp index d572da1ecd0..c65a30b80ac 100644 --- a/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/src/DataTypes/DataTypeAggregateFunction.cpp @@ -139,17 +139,20 @@ static DataTypePtr create(const ASTPtr & arguments) if (!arguments || arguments->children.empty()) throw Exception("Data type AggregateFunction requires parameters: " - "name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + "version(optionally), name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ASTPtr data_type_ast = arguments->children[0]; size_t argument_types_start_idx = 1; /* If aggregate function definition doesn't have version, it will have in AST children args [ASTFunction, types...] - in case * it is parametric, or [ASTIdentifier, types...] - otherwise. If aggregate function has version in AST, then it will be: - * [ASTLitearl, ASTFunction (or ASTIdentifier), types...]. + * [ASTLiteral, ASTFunction (or ASTIdentifier), types...]. */ if (auto * version_ast = arguments->children[0]->as()) { + if (arguments->children.size() < 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Data type AggregateFunction has version, but it requires at least one more parameter - name of aggregate function"); version = version_ast->value.safeGet(); data_type_ast = arguments->children[1]; argument_types_start_idx = 2; From fefecde2c471b29c0bcaeb548df03b4156e0123c Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 20:36:42 +0300 Subject: [PATCH 554/609] translate --- .../aggregate-functions/reference/sparkbar.md | 2 +- .../aggregate-functions/reference/sparkbar.md | 63 +++++++++++++++++++ 2 files changed, 64 insertions(+), 1 deletion(-) create mode 100644 docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md index adbe1d551ca..4c19d16ec60 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md @@ -12,7 +12,7 @@ If no interval is specified, then the minimum `x` is used as the interval start, **Syntax** ``` sql -sparkbar(width, min_x, max_x)(x, y) +sparkbar(width[, min_x, max_x])(x, y) ``` **Parameters** diff --git a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md new file mode 100644 index 00000000000..e58d78bc0e5 --- /dev/null +++ b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md @@ -0,0 +1,63 @@ +--- +toc_priority: 311 +toc_title: sparkbar +--- + +# sparkbar {#sparkbar} + +Функция строит гистограмму частот по заданным значениям `x` и частоте повторения этих значений `y` на интервале `[min_x, max_x]`. + +Если интервал для постоения не указан, то в качестве начального значения будет использовано минимальное `x`, а в качестве конечного — максимальное `x`. + +**Синтаксис** + +``` sql +sparkbar(width[, min_x, max_x])(x, y) +``` + +**Параметры** + +- `width` — Количество отрезков. Тип: [Integer](../../../sql-reference/data-types/int-uint.md). +- `min_x` — Начало интервала. Необязательный параметр. +- `max_x` — Конец интервала. Необязательный параметр. + +**Аргументы** + +- `x` — Поле со значениями. +- `y` — Поле с частотой значений. + +**Возвращаемые значения** + +- Гистограмма частот. + +**Пример** + +Запрос: + +``` sql +CREATE TABLE spark_bar_data (`cnt` UInt64,`event_date` Date) ENGINE = MergeTree ORDER BY event_date SETTINGS index_granularity = 8192; + +INSERT INTO spark_bar_data VALUES(1,'2020-01-01'),(4,'2020-01-02'),(5,'2020-01-03'),(2,'2020-01-04'),(3,'2020-01-05'),(7,'2020-01-06'),(6,'2020-01-07'),(8,'2020-01-08'),(2,'2020-01-11'); + +SELECT sparkbar(9)(event_date,cnt) FROM spark_bar_data; + +SELECT sparkbar(9,toDate('2020-01-01'),toDate('2020-01-10'))(event_date,cnt) FROM spark_bar_data; +``` + +Результат: + +``` text + +┌─sparkbar(9)(event_date, cnt)─┐ +│ │ +│ ▁▅▄▃██▅ ▁ │ +│ │ +└──────────────────────────────┘ + +┌─sparkbar(9, toDate('2020-01-01'), toDate('2020-01-10'))(event_date, cnt)─┐ +│ │ +│▁▄▄▂▅▇█▁ │ +│ │ +└──────────────────────────────────────────────────────────────────────────┘ +``` + From fb24e7181f982a897b14a57b1822f4b53096d152 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Dec 2021 18:27:06 +0000 Subject: [PATCH 555/609] Better --- programs/local/LocalServer.cpp | 64 +++++++++++++++++++++++---------- programs/main.cpp | 8 ++--- src/Functions/getFuzzerData.cpp | 46 ++---------------------- src/Functions/getFuzzerData.h | 48 +++++++++++++++++++++++++ 4 files changed, 98 insertions(+), 68 deletions(-) create mode 100644 src/Functions/getFuzzerData.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 17541b19b8a..a1c35cec97c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -41,6 +41,10 @@ #include #include +#if defined(FUZZING_MODE) + #include +#endif + namespace fs = std::filesystem; @@ -404,18 +408,28 @@ try ThreadStatus thread_status; setupSignalHandler(); -#ifdef FUZZING_MODE - static bool first_time = true; - if (first_time) - { -#endif std::cout << std::fixed << std::setprecision(3); std::cerr << std::fixed << std::setprecision(3); +#if defined(FUZZING_MODE) + static bool first_time = true; + if (first_time) + { + + if (queries_files.empty() && !config().has("query")) + { + std::cerr << "\033[31m" << "ClickHouse compiled in fuzzing mode." << "\033[0m" << std::endl; + std::cerr << "\033[31m" << "You have to provide a query with --query or --queries-file option." << "\033[0m" << std::endl; + std::cerr << "\033[31m" << "The query have to use function getFuzzerData() inside." << "\033[0m" << std::endl; + exit(1); + } + + is_interactive = false; +#else is_interactive = stdin_is_a_tty && (config().hasOption("interactive") || (!config().has("query") && !config().has("table-structure") && queries_files.empty())); - +#endif if (!is_interactive) { /// We will terminate process on error @@ -446,6 +460,7 @@ try #ifdef FUZZING_MODE first_time = false; + } #endif if (is_interactive && !delayed_interactive) @@ -664,7 +679,7 @@ void LocalServer::processConfig() } -static std::string getHelpHeader() +[[ maybe_unused ]] static std::string getHelpHeader() { return "usage: clickhouse-local [initial table definition] [--query ]\n" @@ -680,7 +695,7 @@ static std::string getHelpHeader() } -static std::string getHelpFooter() +[[ maybe_unused ]] static std::string getHelpFooter() { return "Example printing memory used by each Unix user:\n" @@ -691,11 +706,21 @@ static std::string getHelpFooter() } -void LocalServer::printHelpMessage(const OptionsDescription & options_description) +void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & options_description) { +#if defined(FUZZING_MODE) + std::cout << + "usage: clickhouse --query [--query-file ]\n" + + "ClickHouse is build with coverage guided fuzzer (libfuzzer) inside it.\n" + "You have to provide a query which contains getFuzzerData function.\n" + "This will take the data from fuzzing engine, pass it to getFuzzerData function and execute a query.\n" + "Each time the data will be different, and it will last until some segfault or sanitizer assertion is found. \n"; +#else std::cout << getHelpHeader() << "\n"; std::cout << options_description.main_description.value() << "\n"; std::cout << getHelpFooter() << "\n"; +#endif } @@ -793,8 +818,11 @@ int mainEntryClickHouseLocal(int argc, char ** argv) } } -#ifdef FUZZING_MODE -#include +#if defined(FUZZING_MODE) + +// #include + +// #endif std::optional fuzz_app; @@ -825,6 +853,9 @@ extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) return 0; } + + + extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) { try @@ -833,19 +864,14 @@ extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) if (size) --size; auto cur_str = String(reinterpret_cast(data), size); - // to clearly see the beginning and the end - std::cerr << '>' << cur_str << '<' << std::endl; + DB::FunctionGetFuzzerData::update(cur_str); fuzz_app->run(); + return 0; } catch (...) { - std::cerr << "Why here?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!?!" << std::endl; - std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 0; - //auto code = DB::getCurrentExceptionCode(); - //return code ? code : 1; + return 1; } - return 0; } #endif diff --git a/programs/main.cpp b/programs/main.cpp index b4b229d123b..cd416f57982 100644 --- a/programs/main.cpp +++ b/programs/main.cpp @@ -88,9 +88,10 @@ namespace using MainFunc = int (*)(int, char**); +#if !defined(FUZZING_MODE) /// Add an item here to register new application -[[maybe_unused]]std::pair clickhouse_applications[] = +std::pair clickhouse_applications[] = { #if ENABLE_CLICKHOUSE_LOCAL {"local", mainEntryClickHouseLocal}, @@ -141,7 +142,6 @@ using MainFunc = int (*)(int, char**); {"hash-binary", mainEntryClickHouseHashBinary}, }; -#ifndef FUZZING_MODE int printHelp(int, char **) { std::cerr << "Use one of the following commands:" << std::endl; @@ -149,9 +149,7 @@ int printHelp(int, char **) std::cerr << "clickhouse " << application.first << " [args] " << std::endl; return -1; } -#endif -#ifndef FUZZING_MODE bool isClickhouseApp(const std::string & app_suffix, std::vector & argv) { /// Use app if the first arg 'app' is passed (the arg should be quietly removed) @@ -350,7 +348,7 @@ bool inside_main = false; bool inside_main = true; #endif -#ifndef FUZZING_MODE +#if !defined(FUZZING_MODE) int main(int argc_, char ** argv_) { inside_main = true; diff --git a/src/Functions/getFuzzerData.cpp b/src/Functions/getFuzzerData.cpp index c01f575f0be..f516c871950 100644 --- a/src/Functions/getFuzzerData.cpp +++ b/src/Functions/getFuzzerData.cpp @@ -1,53 +1,11 @@ -#include -#include -#include -#include +#include namespace DB { -class FunctionGetFuzzerData : public IFunction -{ - inline static String fuzz_data; - -public: - static constexpr auto name = "getFuzzerData"; - - inline static FunctionPtr create(ContextPtr) { return create(); } - - static FunctionPtr create() - { - return std::make_shared(); - } - - inline String getName() const override { return name; } - - inline size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - inline bool isDeterministic() const override { return false; } - - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, - const DataTypePtr &, - size_t input_rows_count) const override - { - return DataTypeString().createColumnConst(input_rows_count, fuzz_data); - } - - static void update(const String & fuzz_data_) - { - fuzz_data = fuzz_data_; - } -}; void registerFunctionGetFuzzerData(FunctionFactory & factory) { factory.registerFunction(); - factory.registerAlias("get_fuzzer_data", FunctionGetFuzzerData::name, FunctionFactory::CaseInsensitive); } + } diff --git a/src/Functions/getFuzzerData.h b/src/Functions/getFuzzerData.h new file mode 100644 index 00000000000..06f11f28e70 --- /dev/null +++ b/src/Functions/getFuzzerData.h @@ -0,0 +1,48 @@ +#include +#include +#include +#include + +namespace DB +{ +class FunctionGetFuzzerData : public IFunction +{ + inline static String fuzz_data; + +public: + static constexpr auto name = "getFuzzerData"; + + inline static FunctionPtr create(ContextPtr) { return create(); } + + static FunctionPtr create() + { + return std::make_shared(); + } + + inline String getName() const override { return name; } + + inline size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + inline bool isDeterministic() const override { return false; } + + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, + const DataTypePtr &, + size_t input_rows_count) const override + { + return DataTypeString().createColumnConst(input_rows_count, fuzz_data); + } + + static void update(const String & fuzz_data_) + { + fuzz_data = fuzz_data_; + } +}; + +} From 3a5ba8fe44b3696eee561503416233c75b594f7d Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 21:48:09 +0300 Subject: [PATCH 556/609] Update docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../ru/sql-reference/aggregate-functions/reference/sparkbar.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md index e58d78bc0e5..28da1390522 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md @@ -7,7 +7,8 @@ toc_title: sparkbar Функция строит гистограмму частот по заданным значениям `x` и частоте повторения этих значений `y` на интервале `[min_x, max_x]`. -Если интервал для постоения не указан, то в качестве начального значения будет использовано минимальное `x`, а в качестве конечного — максимальное `x`. +Если интервал для построения не указан, то в качестве нижней границы интервала будет взято минимальное значение `x`, а в качестве верхней границы — максимальное значение `x`. + **Синтаксис** From 0d89a9b3a20430645bf15643f2c58223ceb53c3c Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 21:48:24 +0300 Subject: [PATCH 557/609] Update docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../ru/sql-reference/aggregate-functions/reference/sparkbar.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md index 28da1390522..05cf4d67b75 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md @@ -25,7 +25,8 @@ sparkbar(width[, min_x, max_x])(x, y) **Аргументы** - `x` — Поле со значениями. -- `y` — Поле с частотой значений. +- `y` — Поле с частотой повторения значений. + **Возвращаемые значения** From 3a38520bc74cb71fcdb5466803c78107dfc8457f Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 21:48:31 +0300 Subject: [PATCH 558/609] Update docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../ru/sql-reference/aggregate-functions/reference/sparkbar.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md index 05cf4d67b75..b66d710744e 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/sparkbar.md @@ -18,7 +18,8 @@ sparkbar(width[, min_x, max_x])(x, y) **Параметры** -- `width` — Количество отрезков. Тип: [Integer](../../../sql-reference/data-types/int-uint.md). +- `width` — Количество столбцов гистограммы. Тип: [Integer](../../../sql-reference/data-types/int-uint.md). + - `min_x` — Начало интервала. Необязательный параметр. - `max_x` — Конец интервала. Необязательный параметр. From 6854bb46f59f472e561d66a23c1eeee74e91c440 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 6 Dec 2021 21:48:53 +0300 Subject: [PATCH 559/609] Update docs/en/sql-reference/aggregate-functions/reference/sparkbar.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../en/sql-reference/aggregate-functions/reference/sparkbar.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md index 4c19d16ec60..47c696129c7 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md +++ b/docs/en/sql-reference/aggregate-functions/reference/sparkbar.md @@ -5,7 +5,8 @@ toc_title: sparkbar # sparkbar {#sparkbar} -The function plots a frequency histogram for values `x` and the repetition rate of these `y` values over the interval `[min_x, max_x]`. +The function plots a frequency histogram for values `x` and the repetition rate `y` of these values over the interval `[min_x, max_x]`. + If no interval is specified, then the minimum `x` is used as the interval start, and the maximum `x` — as the interval end. From eab6f0ba492f1922d4943a6bc0ff9fca7eb8dd02 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 6 Dec 2021 23:35:29 +0300 Subject: [PATCH 560/609] Update FormatFactory.cpp --- src/Formats/FormatFactory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 4539a0d6e6a..7788b9d115b 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -312,7 +312,7 @@ String FormatFactory::getContentType( throw Exception(ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT, "Format {} is not suitable for output (with processors)", name); auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context); - + Block empty_block; RowOutputFormatParams empty_params; WriteBufferFromOwnString empty_buffer; From 80a146816ca837b892002775854fcd9fac1353c7 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Dec 2021 21:34:52 +0000 Subject: [PATCH 561/609] More comments and style --- CMakeLists.txt | 4 +++ docker/packager/other/fuzzer.sh | 4 --- programs/local/CMakeLists.txt | 10 ------- programs/local/LocalServer.cpp | 50 ++++++++++++++------------------- 4 files changed, 25 insertions(+), 43 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7a3991bc93c..bc0f119e3f6 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -149,6 +149,10 @@ if (ENABLE_FUZZING) set (ENABLE_JEMALLOC 0) set (ENABLE_CHECK_HEAVY_BUILDS 1) set (GLIBC_COMPATIBILITY OFF) + + # For codegen_select_fuzzer + set (ENABLE_PROTOBUF 1) + set (USE_INTERNAL_PROTOBUF_LIBRARY 1) endif() # Global libraries diff --git a/docker/packager/other/fuzzer.sh b/docker/packager/other/fuzzer.sh index 4c208ca7e31..431352f1126 100755 --- a/docker/packager/other/fuzzer.sh +++ b/docker/packager/other/fuzzer.sh @@ -31,10 +31,6 @@ do mv "$FUZZER_PATH" /output/fuzzers done -ninja clickhouse-local -LOCAL_PATH=$(find ./programs -name clickhouse) -strip --strip-unneeded "$LOCAL_PATH" -mv "$LOCAL_PATH" /output/fuzzers tar -zcvf /output/fuzzers.tar.gz /output/fuzzers rm -rf /output/fuzzers diff --git a/programs/local/CMakeLists.txt b/programs/local/CMakeLists.txt index 4ac8ad5d30d..da466f725b3 100644 --- a/programs/local/CMakeLists.txt +++ b/programs/local/CMakeLists.txt @@ -22,14 +22,4 @@ if (ENABLE_FUZZING) add_compile_definitions(FUZZING_MODE=1) set (WITH_COVERAGE ON) target_link_libraries(clickhouse-local-lib PRIVATE ${LIB_FUZZING_ENGINE}) - #add_executable(fuzz-clickhouse-local LocalServer.cpp ${SRCS}) - #[[target_link_libraries(fuzz-clickhouse-local PRIVATE - dbms - ${LIB_FUZZING_ENGINE} - loggers - clickhouse_functions - clickhouse_aggregate_functions - clickhouse_storages_system - clickhouse_table_functions - readpassphrase)]] endif () diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index a1c35cec97c..561b0588787 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -710,7 +710,9 @@ void LocalServer::printHelpMessage([[maybe_unused]] const OptionsDescription & o { #if defined(FUZZING_MODE) std::cout << - "usage: clickhouse --query [--query-file ]\n" + "usage: clickhouse -- \n" + "Note: It is important not to use only one letter keys with single dash for \n" + "for clickhouse-local arguments. It may work incorrectly.\n" "ClickHouse is build with coverage guided fuzzer (libfuzzer) inside it.\n" "You have to provide a query which contains getFuzzerData function.\n" @@ -820,10 +822,6 @@ int mainEntryClickHouseLocal(int argc, char ** argv) #if defined(FUZZING_MODE) -// #include - -// #endif - std::optional fuzz_app; extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) @@ -831,8 +829,11 @@ extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) int & argc = *pargc; char ** argv = *pargv; - // position of delimiter "--" that separates arguments - // of clickhouse-local and fuzzer + /// As a user you can add flags to clickhouse binary in fuzzing mode as follows + /// clickhouse -- + + /// Calculate the position of delimiter "--" that separates arguments + /// of clickhouse-local and libfuzzer int pos_delim = argc; for (int i = 0; i < argc; ++i) { @@ -843,35 +844,26 @@ extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) } } + /// Initialize clickhouse-local app fuzz_app.emplace(); fuzz_app->init(pos_delim, argv); - for (int i = pos_delim + 1; i < argc; ++i) - std::swap(argv[i], argv[i - pos_delim]); - argc -= pos_delim; - if (argc == 0) // no delimiter provided - ++argc; + + /// We will leave clickhouse-local specific arguments as is, because libfuzzer will ignore + /// all keys starting with -- return 0; } - - extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try { - try - { - // inappropriate symbol for fuzzing at the end - if (size) - --size; - auto cur_str = String(reinterpret_cast(data), size); - - DB::FunctionGetFuzzerData::update(cur_str); - fuzz_app->run(); - return 0; - } - catch (...) - { - return 1; - } + auto input = String(reinterpret_cast(data), size); + DB::FunctionGetFuzzerData::update(input); + fuzz_app->run(); + return 0; +} +catch (...) +{ + return 1; } #endif From 0a5df82c2f92d94e01009320652da3ec892460f8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 6 Dec 2021 21:40:44 +0000 Subject: [PATCH 562/609] Bump From dc5707bdb5b2479712b95ca6a9804fa04c0ee2e5 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 7 Dec 2021 10:47:17 +0300 Subject: [PATCH 563/609] Update backport.py --- utils/github/backport.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/github/backport.py b/utils/github/backport.py index a28a1510694..9227dbf4108 100644 --- a/utils/github/backport.py +++ b/utils/github/backport.py @@ -74,7 +74,7 @@ class Backport: # First pass. Find all must-backports for label in pr['labels']['nodes']: - if label['name'] == 'pr-bugfix' or label['name'] == 'pr-must-backport': + if label['name'] == 'pr-must-backport': backport_map[pr['number']] = branch_set.copy() continue matched = RE_MUST_BACKPORT.match(label['name']) From 9df664e1c99a8fbe021c7545d8af63d52427e581 Mon Sep 17 00:00:00 2001 From: dongyifeng Date: Tue, 7 Dec 2021 16:09:39 +0800 Subject: [PATCH 564/609] fix bug when remove unneeded columns in subquery (#32289) --- src/Interpreters/TreeRewriter.cpp | 6 +++++- .../0_stateless/02131_remove_columns_in_subquery.reference | 1 + .../0_stateless/02131_remove_columns_in_subquery.sql | 1 + 3 files changed, 7 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02131_remove_columns_in_subquery.reference create mode 100644 tests/queries/0_stateless/02131_remove_columns_in_subquery.sql diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index d864bb54b2e..6b3a50d88e2 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -465,9 +465,13 @@ void removeUnneededColumnsFromSelectClause(const ASTSelectQuery * select_query, ASTFunction * func = elem->as(); /// Never remove untuple. It's result column may be in required columns. - /// It is not easy to analyze untuple here, because types were not calculated yes. + /// It is not easy to analyze untuple here, because types were not calculated yet. if (func && func->name == "untuple") new_elements.push_back(elem); + + /// removing aggregation can change number of rows, so `count()` result in outer sub-query would be wrong + if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name) && !select_query->groupBy()) + new_elements.push_back(elem); } } diff --git a/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference b/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02131_remove_columns_in_subquery.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql b/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql new file mode 100644 index 00000000000..f9ca2269aad --- /dev/null +++ b/tests/queries/0_stateless/02131_remove_columns_in_subquery.sql @@ -0,0 +1 @@ +select count(1) from (SELECT 1 AS a, count(1) FROM numbers(5)) From c4c517bb8a5ed2cff70c18dae437e785de483d66 Mon Sep 17 00:00:00 2001 From: vxider Date: Tue, 7 Dec 2021 08:14:00 +0000 Subject: [PATCH 565/609] rename window functions --- src/Functions/FunctionsWindow.cpp | 14 ++-- src/Functions/FunctionsWindow.h | 20 +++--- src/Storages/WindowView/StorageWindowView.cpp | 48 ++++++------- src/Storages/WindowView/StorageWindowView.h | 32 ++++----- ...7_window_view_parser_inner_table.reference | 20 +++--- .../01047_window_view_parser_inner_table.sql | 20 +++--- .../01048_window_view_parser.reference | 28 ++++---- .../0_stateless/01048_window_view_parser.sql | 28 ++++---- ...049_window_view_window_functions.reference | 68 +++++++++---------- .../01049_window_view_window_functions.sql | 68 +++++++++---------- .../01050_window_view_parser_tumble.sql | 14 ++-- .../01051_window_view_parser_hop.sql | 14 ++-- .../01052_window_view_proc_tumble_to_now.sql | 2 +- .../01053_window_view_proc_hop_to_now.sql | 2 +- .../01054_window_view_proc_tumble_to.sql | 2 +- .../01055_window_view_proc_hop_to.sql | 2 +- .../01056_window_view_proc_hop_watch.py | 2 +- ...window_view_event_tumble_to_strict_asc.sql | 2 +- ...58_window_view_event_hop_to_strict_asc.sql | 2 +- ..._window_view_event_hop_watch_strict_asc.py | 2 +- .../01060_window_view_event_tumble_to_asc.sql | 2 +- .../01061_window_view_event_hop_to_asc.sql | 2 +- .../01062_window_view_event_hop_watch_asc.py | 2 +- ...63_window_view_event_tumble_to_bounded.sql | 2 +- ...01064_window_view_event_hop_to_bounded.sql | 2 +- ...065_window_view_event_hop_watch_bounded.py | 2 +- ...ew_event_tumble_to_strict_asc_lateness.sql | 2 +- ...ndow_view_event_tumble_to_asc_lateness.sql | 2 +- ..._view_event_tumble_to_bounded_lateness.sql | 2 +- .../01069_window_view_proc_tumble_watch.py | 2 +- 30 files changed, 205 insertions(+), 205 deletions(-) diff --git a/src/Functions/FunctionsWindow.cpp b/src/Functions/FunctionsWindow.cpp index a26faac304d..be336aa9a7c 100644 --- a/src/Functions/FunctionsWindow.cpp +++ b/src/Functions/FunctionsWindow.cpp @@ -116,7 +116,7 @@ namespace template <> struct WindowImpl { - static constexpr auto name = "TUMBLE"; + static constexpr auto name = "tumble"; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -213,7 +213,7 @@ struct WindowImpl template <> struct WindowImpl { - static constexpr auto name = "TUMBLE_START"; + static constexpr auto name = "tumbleStart"; static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -257,7 +257,7 @@ struct WindowImpl template <> struct WindowImpl { - static constexpr auto name = "TUMBLE_END"; + static constexpr auto name = "tumbleEnd"; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -285,7 +285,7 @@ struct WindowImpl template <> struct WindowImpl { - static constexpr auto name = "HOP"; + static constexpr auto name = "hop"; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -417,7 +417,7 @@ struct WindowImpl template <> struct WindowImpl { - static constexpr auto name = "WINDOW_ID"; + static constexpr auto name = "windowID"; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -569,7 +569,7 @@ struct WindowImpl template <> struct WindowImpl { - static constexpr auto name = "HOP_START"; + static constexpr auto name = "hopStart"; static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { @@ -612,7 +612,7 @@ struct WindowImpl template <> struct WindowImpl { - static constexpr auto name = "HOP_END"; + static constexpr auto name = "hopEnd"; [[maybe_unused]] static DataTypePtr getReturnType(const ColumnsWithTypeAndName & arguments, const String & function_name) { diff --git a/src/Functions/FunctionsWindow.h b/src/Functions/FunctionsWindow.h index 37acb660751..be4513225cf 100644 --- a/src/Functions/FunctionsWindow.h +++ b/src/Functions/FunctionsWindow.h @@ -9,25 +9,25 @@ namespace DB /** Window functions: * - * TUMBLE(time_attr, interval [, timezone]) + * tumble(time_attr, interval [, timezone]) * - * TUMBLE_START(window_id) + * tumbleStart(window_id) * - * TUMBLE_START(time_attr, interval [, timezone]) + * tumbleStart(time_attr, interval [, timezone]) * - * TUMBLE_END(window_id) + * tumbleEnd(window_id) * - * TUMBLE_END(time_attr, interval [, timezone]) + * tumbleEnd(time_attr, interval [, timezone]) * - * HOP(time_attr, hop_interval, window_interval [, timezone]) + * hop(time_attr, hop_interval, window_interval [, timezone]) * - * HOP_START(window_id) + * hopStart(window_id) * - * HOP_START(time_attr, hop_interval, window_interval [, timezone]) + * hopStart(time_attr, hop_interval, window_interval [, timezone]) * - * HOP_END(window_id) + * hopEnd(window_id) * - * HOP_END(time_attr, hop_interval, window_interval [, timezone]) + * hopEnd(time_attr, hop_interval, window_interval [, timezone]) * */ enum WindowFunctionName diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index 915e775ff14..51f2a37aa8f 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -62,7 +62,7 @@ namespace ErrorCodes namespace { - /// Fetch all window info and replace TUMPLE or HOP node names with WINDOW_ID + /// Fetch all window info and replace tumble or hop node names with windowID struct FetchQueryInfoMatcher { using Visitor = InDepthNodeVisitor; @@ -85,10 +85,10 @@ namespace { if (auto * t = ast->as()) { - if (t->name == "TUMBLE" || t->name == "HOP") + if (t->name == "tumble" || t->name == "hop") { - data.is_tumble = t->name == "TUMBLE"; - data.is_hop = t->name == "HOP"; + data.is_tumble = t->name == "tumble"; + data.is_hop = t->name == "hop"; auto temp_node = t->clone(); temp_node->setAlias(""); if (startsWith(t->arguments->children[0]->getColumnName(), "toDateTime")) @@ -98,7 +98,7 @@ namespace if (!data.window_function) { data.serialized_window_function = serializeAST(*temp_node); - t->name = "WINDOW_ID"; + t->name = "windowID"; data.window_id_name = t->getColumnName(); data.window_id_alias = t->alias; data.window_function = t->clone(); @@ -109,14 +109,14 @@ namespace { if (serializeAST(*temp_node) != data.serialized_window_function) throw Exception("WINDOW VIEW only support ONE WINDOW FUNCTION", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW); - t->name = "WINDOW_ID"; + t->name = "windowID"; } } } } }; - /// Replace WINDOW_ID node name with either TUMBLE or HOP. + /// Replace windowID node name with either tumble or hop. struct ReplaceWindowIdMatcher { public: @@ -132,15 +132,15 @@ namespace { if (auto * t = ast->as()) { - if (t->name == "WINDOW_ID") + if (t->name == "windowID") t->name = data.window_name; } } }; - /// GROUP BY TUMBLE(now(), INTERVAL '5' SECOND) + /// GROUP BY tumble(now(), INTERVAL '5' SECOND) /// will become - /// GROUP BY TUMBLE(____timestamp, INTERVAL '5' SECOND) + /// GROUP BY tumble(____timestamp, INTERVAL '5' SECOND) struct ReplaceFunctionNowData { using TypeToVisit = ASTFunction; @@ -151,7 +151,7 @@ namespace void visit(ASTFunction & node, ASTPtr & node_ptr) { - if (node.name == "WINDOW_ID" || node.name == "TUMBLE" || node.name == "HOP") + if (node.name == "windowID" || node.name == "tumble" || node.name == "hop") { if (const auto * t = node.arguments->children[0]->as(); t && t->name == "now") @@ -188,8 +188,8 @@ namespace { if (auto * t = ast->as()) { - if (t->name == "HOP" || t->name == "TUMBLE") - t->name = "WINDOW_ID"; + if (t->name == "hop" || t->name == "tumble") + t->name = "windowID"; } } }; @@ -221,12 +221,12 @@ namespace { if (node.name == "tuple") { - /// tuple(WINDOW_ID(timestamp, toIntervalSecond('5'))) + /// tuple(windowID(timestamp, toIntervalSecond('5'))) return; } else { - /// WINDOW_ID(timestamp, toIntervalSecond('5')) -> identifier. + /// windowID(timestamp, toIntervalSecond('5')) -> identifier. /// and other... node_ptr = std::make_shared(node.getColumnName()); } @@ -351,14 +351,14 @@ static size_t getWindowIDColumnPosition(const Block & header) auto position = -1; for (const auto & column : header.getColumnsWithTypeAndName()) { - if (startsWith(column.name, "WINDOW_ID")) + if (startsWith(column.name, "windowID")) { position = header.getPositionByName(column.name); break; } } if (position < 0) - throw Exception("Not found column WINDOW_ID", ErrorCodes::LOGICAL_ERROR); + throw Exception("Not found column windowID", ErrorCodes::LOGICAL_ERROR); return position; } @@ -631,7 +631,7 @@ std::shared_ptr StorageWindowView::getInnerTableCreateQuery( time_now_visitor.visit(node); function_now_timezone = time_now_data.now_timezone; } - /// TUMBLE/HOP -> WINDOW_ID + /// tumble/hop -> windowID func_window_visitor.visit(node); to_identifier_visitor.visit(node); new_storage->set(field, node); @@ -960,7 +960,7 @@ StorageWindowView::StorageWindowView( select_table_id = StorageID(select_database_name, select_table_name); DatabaseCatalog::instance().addDependency(select_table_id, table_id_); - /// Extract all info from query; substitute Function_TUMPLE and Function_HOP with Function_WINDOW_ID. + /// Extract all info from query; substitute Function_tumble and Function_hop with Function_windowID. auto inner_query = innerQueryParser(select_query->as()); // Parse mergeable query @@ -971,13 +971,13 @@ StorageWindowView::StorageWindowView( if (is_time_column_func_now) window_id_name = func_now_data.window_id_name; - // Parse final query (same as mergeable query but has TUMBLE/HOP instead of WINDOW_ID) + // Parse final query (same as mergeable query but has tumble/hop instead of windowID) final_query = mergeable_query->clone(); ReplaceWindowIdMatcher::Data final_query_data; if (is_tumble) - final_query_data.window_name = "TUMBLE"; + final_query_data.window_name = "tumble"; else - final_query_data.window_name = "HOP"; + final_query_data.window_name = "hop"; ReplaceWindowIdMatcher::Visitor(final_query_data).visit(final_query); is_watermark_strictly_ascending = query.is_watermark_strictly_ascending; @@ -989,9 +989,9 @@ StorageWindowView::StorageWindowView( eventTimeParser(query); if (is_tumble) - window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "TUMBLE"); + window_column_name = std::regex_replace(window_id_name, std::regex("windowID"), "tumble"); else - window_column_name = std::regex_replace(window_id_name, std::regex("WINDOW_ID"), "HOP"); + window_column_name = std::regex_replace(window_id_name, std::regex("windowID"), "hop"); auto generate_inner_table_name = [](const StorageID & storage_id) { diff --git a/src/Storages/WindowView/StorageWindowView.h b/src/Storages/WindowView/StorageWindowView.h index 08f24816d72..aaa9f7093e7 100644 --- a/src/Storages/WindowView/StorageWindowView.h +++ b/src/Storages/WindowView/StorageWindowView.h @@ -22,11 +22,11 @@ using ASTPtr = std::shared_ptr; * [ENGINE [db.]name] * [WATERMARK strategy] [ALLOWED_LATENESS interval_function] * AS SELECT ... - * GROUP BY [TUBLE/HOP(...)] + * GROUP BY [tumble/hop(...)] * * - only stores data that has not been triggered yet; * - fire_task checks if there is a window ready to be fired - * (each window result is fired in one output at the end of TUMBLE/HOP window interval); + * (each window result is fired in one output at the end of tumble/hop window interval); * - intermediate data is stored in inner table with * AggregatingMergeTree engine by default, but any other -MergeTree * engine might be used as inner table engine; @@ -35,24 +35,24 @@ using ASTPtr = std::shared_ptr; * Here function in GROUP BY clause results in a "window_id" * represented as Tuple(DateTime, DateTime) - lower and upper bounds of the window. * Function might be one of the following: - * 1. TUMBLE(time_attr, interval [, timezone]) + * 1. tumble(time_attr, interval [, timezone]) * - non-overlapping, continuous windows with a fixed duration (interval); * - example: - * SELECT TUMBLE(toDateTime('2021-01-01 00:01:45'), INTERVAL 10 SECOND) + * SELECT tumble(toDateTime('2021-01-01 00:01:45'), INTERVAL 10 SECOND) * results in ('2021-01-01 00:01:40','2021-01-01 00:01:50') - * 2. HOP(time_attr, hop_interval, window_interval [, timezone]) + * 2. hop(time_attr, hop_interval, window_interval [, timezone]) * - sliding window; * - has a fixed duration (window_interval parameter) and hops by a * specified hop interval (hop_interval parameter); * If the hop_interval is smaller than the window_interval, hopping windows * are overlapping. Thus, records can be assigned to multiple windows. * - example: - * SELECT HOP(toDateTime('2021-01-01 00:00:45'), INTERVAL 3 SECOND, INTERVAL 10 SECOND) + * SELECT hop(toDateTime('2021-01-01 00:00:45'), INTERVAL 3 SECOND, INTERVAL 10 SECOND) * results in ('2021-01-01 00:00:38','2021-01-01 00:00:48') * * DateTime value can be used with the following functions to find out start/end of the window: - * - TUMPLE_START(time_attr, interval [, timezone]), TUMPLE_END(time_attr, interval [, timezone]) - * - HOP_START(time_attr, hop_interval, window_interval [, timezone]), HOP_END(time_attr, hop_interval, window_interval [, timezone]) + * - tumbleStart(time_attr, interval [, timezone]), tumbleEnd(time_attr, interval [, timezone]) + * - hopStart(time_attr, hop_interval, window_interval [, timezone]), hopEnd(time_attr, hop_interval, window_interval [, timezone]) * * * Time processing options. @@ -61,8 +61,8 @@ using ASTPtr = std::shared_ptr; * - produces results based on the time of the local machine; * - example: * CREATE WINDOW VIEW test.wv TO test.dst - * AS SELECT count(number), TUMBLE_START(w_id) as w_start FROM test.mt - * GROUP BY TUMBLE(now(), INTERVAL '5' SECOND) as w_id + * AS SELECT count(number), tumbleStart(w_id) as w_start FROM test.mt + * GROUP BY tumble(now(), INTERVAL '5' SECOND) as w_id * * 2. event time * - produces results based on the time that is contained in every record; @@ -79,7 +79,7 @@ using ASTPtr = std::shared_ptr; * CREATE WINDOW VIEW test.wv TO test.dst * WATERMARK=STRICTLY_ASCENDING * AS SELECT count(number) FROM test.mt - * GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND); + * GROUP BY tumble(timestamp, INTERVAL '5' SECOND); * (where `timestamp` is a DateTime column in test.mt) * * @@ -90,8 +90,8 @@ using ASTPtr = std::shared_ptr; * - Can be enabled by using ALLOWED_LATENESS=INTERVAL, like this: * CREATE WINDOW VIEW test.wv TO test.dst * WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND - * AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM test.mt - * GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND) AS wid; + * AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM test.mt + * GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid; * * - Instead of firing at the end of windows, WINDOW VIEW will fire * immediately when encountering late events; @@ -150,11 +150,11 @@ public: private: Poco::Logger * log; - /// Stored query, e.g. SELECT * FROM * GROUP BY TUMBLE(now(), *) + /// Stored query, e.g. SELECT * FROM * GROUP BY tumble(now(), *) ASTPtr select_query; - /// Used to generate the mergeable state of select_query, e.g. SELECT * FROM * GROUP BY WINDOW_ID(____timestamp, *) + /// Used to generate the mergeable state of select_query, e.g. SELECT * FROM * GROUP BY windowID(____timestamp, *) ASTPtr mergeable_query; - /// Used to fetch the mergeable state and generate the final result. e.g. SELECT * FROM * GROUP BY TUMBLE(____timestamp, *) + /// Used to fetch the mergeable state and generate the final result. e.g. SELECT * FROM * GROUP BY tumble(____timestamp, *) ASTPtr final_query; ContextMutablePtr window_view_context; diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference index 19ebe5e0dbc..77f48f2832c 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.reference @@ -1,22 +1,22 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(____timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'))`\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nORDER BY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---PARTITION--- -CREATE TABLE test_01047.`.inner.wv`\n(\n `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `WINDOW_ID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01047.`.inner.wv`\n(\n `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPARTITION BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY `windowID(____timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql index f332ec57b7f..777c5ae2a5a 100644 --- a/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql +++ b/tests/queries/0_stateless/01047_window_view_parser_inner_table.sql @@ -12,31 +12,31 @@ SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), TUMBLE_END(wid) AS count FROM test_01047.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) as wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY tumble(timestamp, INTERVAL '1' SECOND) AS SELECT count(a), tumbleEnd(wid) AS count FROM test_01047.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) as wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), b) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (TUMBLE(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY TUMBLE(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (tumble(timestamp, INTERVAL '1' SECOND), plus(a, b)) PRIMARY KEY tumble(timestamp, INTERVAL '1' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---PARTITION---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, TUMBLE(now(), INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, tumble(now(), INTERVAL '1' SECOND) AS wid FROM test_01047.mt GROUP BY wid; SHOW CREATE TABLE test_01047.`.inner.wv`; @@ -44,31 +44,31 @@ SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, HOP_END(wid) FROM test_01047.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01047.mt GROUP BY wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), b) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY (hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND), plus(a, b)) PRIMARY KEY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS SELECT count(a) AS count FROM test_01047.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE test_01047.`.inner.wv`; SELECT '||---PARTITION---'; DROP TABLE IF EXISTS test_01047.wv; DROP TABLE IF EXISTS test_01047.`.inner.wv`; -CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, HOP_END(wid) FROM test_01047.mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; +CREATE WINDOW VIEW test_01047.wv ENGINE AggregatingMergeTree ORDER BY wid PARTITION BY wid AS SELECT count(a) AS count, hopEnd(wid) FROM test_01047.mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) as wid; SHOW CREATE TABLE test_01047.`.inner.wv`; DROP TABLE test_01047.wv; diff --git a/tests/queries/0_stateless/01048_window_view_parser.reference b/tests/queries/0_stateless/01048_window_view_parser.reference index 47ed39fc1d8..6625313f572 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.reference +++ b/tests/queries/0_stateless/01048_window_view_parser.reference @@ -1,26 +1,26 @@ ---TUMBLE--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(1))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(\'1\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(\'1\'), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 ---HOP--- ||---WINDOW COLUMN NAME--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3))`)\nSETTINGS index_granularity = 8192 ||---WINDOW COLUMN ALIAS--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`)\nSETTINGS index_granularity = 8192 ||---IDENTIFIER--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `b` Int32,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `b` Int32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, b)\nSETTINGS index_granularity = 8192 ||---FUNCTION--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `plus(a, b)` Int64,\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 ||---TimeZone--- -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY tuple(`WINDOW_ID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 -CREATE TABLE test_01048.`.inner.wv`\n(\n `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`WINDOW_ID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')` UInt32,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`\nORDER BY tuple(`windowID(timestamp, toIntervalSecond(1), toIntervalSecond(3), \'Asia/Shanghai\')`)\nSETTINGS index_granularity = 8192 +CREATE TABLE test_01048.`.inner.wv`\n(\n `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))` UInt32,\n `plus(a, b)` Int64,\n `count(a)` AggregateFunction(count, Int32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`\nORDER BY (`windowID(timestamp, toIntervalSecond(\'1\'), toIntervalSecond(\'3\'))`, `plus(a, b)`)\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01048_window_view_parser.sql b/tests/queries/0_stateless/01048_window_view_parser.sql index e7dc4b324f6..3f57f6fbd91 100644 --- a/tests/queries/0_stateless/01048_window_view_parser.sql +++ b/tests/queries/0_stateless/01048_window_view_parser.sql @@ -11,71 +11,71 @@ CREATE TABLE test_01048.mt(a Int32, b Int32, timestamp DateTime) ENGINE=MergeTre SELECT '---TUMBLE---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, TUMBLE_END(wid) as wend FROM test_01048.mt GROUP BY TUMBLE(timestamp, INTERVAL 1 SECOND) as wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, tumbleEnd(wid) as wend FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL 1 SECOND) as wid; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01048.mt GROUP BY wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND) AS wid FROM test_01048.mt GROUP BY wid; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, tumble(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE test_01048.`.inner.wv`; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, b; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) AS wid, b; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, tumble(timestamp, INTERVAL '1' SECOND) AS wid; SHOW CREATE TABLE test_01048.`.inner.wv`; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND) AS wid, plus(a, b); SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---TimeZone---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, TUMBLE(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM test_01048.mt GROUP BY wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, tumble(timestamp, INTERVAL '1' SECOND, 'Asia/Shanghai') AS wid FROM test_01048.mt GROUP BY wid; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '---HOP---'; SELECT '||---WINDOW COLUMN NAME---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND) as wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, hopEnd(wid) as wend FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND) as wid; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---WINDOW COLUMN ALIAS---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01048.mt GROUP BY wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid FROM test_01048.mt GROUP BY wid; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---IDENTIFIER---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY b, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE test_01048.`.inner.wv`; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, b; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---FUNCTION---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY plus(a, b) as _type, hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; SHOW CREATE TABLE test_01048.`.inner.wv`; SELECT '||---TimeZone---'; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, HOP_END(wid) as wend FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'Asia/Shanghai') as wid; +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count, hopEnd(wid) as wend FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'Asia/Shanghai') as wid; SHOW CREATE TABLE test_01048.`.inner.wv`; DROP TABLE IF EXISTS test_01048.wv; -CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); +CREATE WINDOW VIEW test_01048.wv AS SELECT count(a) AS count FROM test_01048.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid, plus(a, b); SHOW CREATE TABLE test_01048.`.inner.wv`; DROP TABLE test_01048.wv; diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.reference b/tests/queries/0_stateless/01049_window_view_window_functions.reference index e8813db5a7d..2d49664b280 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.reference +++ b/tests/queries/0_stateless/01049_window_view_window_functions.reference @@ -1,69 +1,69 @@ -- { echo } -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, 'US/Samoa'); ('2020-01-09 12:00:01','2020-01-09 12:00:02') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, 'US/Samoa'); ('2020-01-09 12:00:00','2020-01-09 12:01:00') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' HOUR, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' HOUR, 'US/Samoa'); ('2020-01-09 12:00:00','2020-01-09 13:00:00') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); ('2020-01-09 00:00:00','2020-01-10 00:00:00') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, 'US/Samoa'); ('2020-01-06','2020-01-13') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' MONTH, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' MONTH, 'US/Samoa'); ('2020-01-01','2020-02-01') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' QUARTER, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' QUARTER, 'US/Samoa'); ('2020-01-01','2020-04-01') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' YEAR, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' YEAR, 'US/Samoa'); ('2020-01-01','2021-01-01') -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); ('2020-01-09 00:00:00','2020-01-10 00:00:00') -SELECT TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT tumbleStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); 2020-01-09 00:00:00 -SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(tumbleStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-09 00:00:00 -SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(tumbleStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-09 00:00:00 -SELECT TUMBLE_START(TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); +SELECT tumbleStart(tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); 2020-01-09 00:00:00 -SELECT TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT tumbleEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); 2020-01-10 00:00:00 -SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(tumbleEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-10 00:00:00 -SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(tumbleEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-10 00:00:00 -SELECT TUMBLE_END(TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); +SELECT tumbleEnd(tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); 2020-01-10 00:00:00 -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'US/Samoa'); ('2020-01-09 11:59:59','2020-01-09 12:00:02') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, INTERVAL 3 MINUTE, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, INTERVAL 3 MINUTE, 'US/Samoa'); ('2020-01-09 11:58:00','2020-01-09 12:01:00') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 HOUR, INTERVAL 3 HOUR, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 HOUR, INTERVAL 3 HOUR, 'US/Samoa'); ('2020-01-09 10:00:00','2020-01-09 13:00:00') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 DAY, INTERVAL 3 DAY, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 DAY, INTERVAL 3 DAY, 'US/Samoa'); ('2020-01-07 00:00:00','2020-01-10 00:00:00') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, INTERVAL 3 WEEK, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, INTERVAL 3 WEEK, 'US/Samoa'); ('2019-12-23','2020-01-13') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MONTH, INTERVAL 3 MONTH, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MONTH, INTERVAL 3 MONTH, 'US/Samoa'); ('2019-11-01','2020-02-01') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 QUARTER, INTERVAL 3 QUARTER, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 QUARTER, INTERVAL 3 QUARTER, 'US/Samoa'); ('2019-07-01','2020-04-01') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 YEAR, INTERVAL 3 YEAR, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 YEAR, INTERVAL 3 YEAR, 'US/Samoa'); ('2018-01-01','2021-01-01') -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); ('2020-01-07 00:00:00','2020-01-10 00:00:00') -SELECT HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); +SELECT hopStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); 2020-01-07 00:00:00 -SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(hopStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-07 00:00:00 -SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(hopStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-07 00:00:00 -SELECT HOP_START(HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); +SELECT hopStart(hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); 2020-01-07 00:00:00 -SELECT HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); +SELECT hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); 2020-01-10 00:00:00 -SELECT toDateTime(HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-10 00:00:00 -SELECT toDateTime(HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); 2020-01-10 00:00:00 -SELECT HOP_END(HOP(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); +SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); 2019-01-10 00:00:00 diff --git a/tests/queries/0_stateless/01049_window_view_window_functions.sql b/tests/queries/0_stateless/01049_window_view_window_functions.sql index 4c98f9445e1..617019bd2c6 100644 --- a/tests/queries/0_stateless/01049_window_view_window_functions.sql +++ b/tests/queries/0_stateless/01049_window_view_window_functions.sql @@ -1,38 +1,38 @@ -- { echo } -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' HOUR, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' MONTH, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' QUARTER, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' YEAR, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' HOUR, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' MONTH, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' QUARTER, 'US/Samoa'); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' YEAR, 'US/Samoa'); -SELECT TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); -SELECT TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); -SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT toDateTime(TUMBLE_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT TUMBLE_START(TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); -SELECT TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); -SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT toDateTime(TUMBLE_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT TUMBLE_END(TUMBLE(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); +SELECT tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT tumbleStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT toDateTime(tumbleStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(tumbleStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT tumbleStart(tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); +SELECT tumbleEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'); +SELECT toDateTime(tumbleEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(tumbleEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT tumbleEnd(tumble(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, 'US/Samoa')); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, INTERVAL 3 MINUTE, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 HOUR, INTERVAL 3 HOUR, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 DAY, INTERVAL 3 DAY, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, INTERVAL 3 WEEK, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MONTH, INTERVAL 3 MONTH, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 QUARTER, INTERVAL 3 QUARTER, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 YEAR, INTERVAL 3 YEAR, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 SECOND, INTERVAL 3 SECOND, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MINUTE, INTERVAL 3 MINUTE, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 HOUR, INTERVAL 3 HOUR, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 DAY, INTERVAL 3 DAY, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 WEEK, INTERVAL 3 WEEK, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 MONTH, INTERVAL 3 MONTH, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 QUARTER, INTERVAL 3 QUARTER, 'US/Samoa'); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL 1 YEAR, INTERVAL 3 YEAR, 'US/Samoa'); -SELECT HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); -SELECT HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); -SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT toDateTime(HOP_START(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT HOP_START(HOP(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); -SELECT HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); -SELECT toDateTime(HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT toDateTime(HOP_END(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); -SELECT HOP_END(HOP(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); +SELECT hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); +SELECT hopStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); +SELECT toDateTime(hopStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(hopStart(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT hopStart(hop(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); +SELECT hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'); +SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT toDateTime(hopEnd(toDateTime('2020-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa'), 'US/Samoa'); +SELECT hopEnd(hop(toDateTime('2019-01-09 12:00:01', 'US/Samoa'), INTERVAL '1' DAY, INTERVAL '3' DAY, 'US/Samoa')); diff --git a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql index 6837036263c..54f9ed00cbe 100644 --- a/tests/queries/0_stateless/01050_window_view_parser_tumble.sql +++ b/tests/queries/0_stateless/01050_window_view_parser_tumble.sql @@ -6,28 +6,28 @@ CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---WATERMARK---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---With w_end---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(TUMBLE(timestamp, INTERVAL '3' SECOND)) AS w_start, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(tumble(timestamp, INTERVAL '3' SECOND)) AS w_start, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WithOut w_end---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WITH---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(wid) AS w_end, date_time FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(wid) AS w_end, date_time FROM mt GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---WHERE---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid; SELECT '---ORDER_BY---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY TUMBLE(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; +CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY tumble(timestamp, INTERVAL '3' SECOND) AS wid ORDER BY w_start; SELECT '---With now---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), TUMBLE_START(wid) AS w_start, TUMBLE_END(TUMBLE(now(), INTERVAL '3' SECOND)) AS w_end FROM mt GROUP BY TUMBLE(now(), INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), tumbleStart(wid) AS w_start, tumbleEnd(tumble(now(), INTERVAL '3' SECOND)) AS w_end FROM mt GROUP BY tumble(now(), INTERVAL '3' SECOND) AS wid; diff --git a/tests/queries/0_stateless/01051_window_view_parser_hop.sql b/tests/queries/0_stateless/01051_window_view_parser_hop.sql index df0729108d0..0f705d5c911 100644 --- a/tests/queries/0_stateless/01051_window_view_parser_hop.sql +++ b/tests/queries/0_stateless/01051_window_view_parser_hop.sql @@ -6,28 +6,28 @@ CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); SELECT '---WATERMARK---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv WATERMARK=INTERVAL '1' SECOND AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---With w_end---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WithOut w_end---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WITH---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), HOP_START(wid) AS w_start, HOP_END(wid) AS w_end, date_time FROM mt GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv AS WITH toDateTime('2018-01-01 00:00:00') AS date_time SELECT count(a), hopStart(wid) AS w_start, hopEnd(wid) AS w_end, date_time FROM mt GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---WHERE---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid; SELECT '---ORDER_BY---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start FROM mt WHERE a != 1 GROUP BY HOP(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; +CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start FROM mt WHERE a != 1 GROUP BY hop(timestamp, INTERVAL '3' SECOND, INTERVAL '5' SECOND) AS wid ORDER BY w_start; SELECT '---With now---'; DROP TABLE IF EXISTS wv NO DELAY; -CREATE WINDOW VIEW wv AS SELECT count(a), HOP_START(wid) AS w_start, HOP_END(HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND)) as w_end FROM mt GROUP BY HOP(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; +CREATE WINDOW VIEW wv AS SELECT count(a), hopStart(wid) AS w_start, hopEnd(hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND)) as w_end FROM mt GROUP BY hop(now(), INTERVAL '1' SECOND, INTERVAL '3' SECOND) AS wid; diff --git a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql index 169a66e7bc6..2d01e1205b2 100644 --- a/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql +++ b/tests/queries/0_stateless/01052_window_view_proc_tumble_to_now.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(now('US/Samoa'), INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY tumble(now('US/Samoa'), INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); SELECT sleep(3); diff --git a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql index c39bab21cb1..9f3dc3ca89e 100644 --- a/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql +++ b/tests/queries/0_stateless/01053_window_view_proc_hop_to_now.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS wv; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(now('US/Samoa'), INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY hop(now('US/Samoa'), INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1); SELECT sleep(3); diff --git a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql index f229969603b..86b7ab89150 100644 --- a/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql +++ b/tests/queries/0_stateless/01054_window_view_proc_tumble_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, now('US/Samoa') + 1); SELECT sleep(3); diff --git a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql index b75cc33e741..1da497092c5 100644 --- a/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql +++ b/tests/queries/0_stateless/01055_window_view_proc_hop_to.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst AS SELECT count(a) AS count FROM mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, now('US/Samoa') + 1); SELECT sleep(3); diff --git a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py index df83615d507..02e97ee7a17 100755 --- a/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py +++ b/tests/queries/0_stateless/01056_window_view_proc_hop_watch.py @@ -34,7 +34,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('CREATE TABLE 01056_window_view_proc_hop_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY HOP(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;") + client1.send("CREATE WINDOW VIEW 01056_window_view_proc_hop_watch.wv AS SELECT count(a) AS count FROM 01056_window_view_proc_hop_watch.mt GROUP BY hop(timestamp, INTERVAL '1' SECOND, INTERVAL '1' SECOND, 'US/Samoa') AS wid;") client1.expect(prompt) client1.send('WATCH 01056_window_view_proc_hop_watch.wv') diff --git a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql index 4883e006e85..de738662817 100644 --- a/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql +++ b/tests/queries/0_stateless/01057_window_view_event_tumble_to_strict_asc.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) as w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) as w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql index 944fd9939b4..c9846cbd7cd 100644 --- a/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql +++ b/tests/queries/0_stateless/01058_window_view_event_hop_to_strict_asc.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, hopEnd(wid) as w_end FROM mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py index 1052b44965c..638182ac216 100755 --- a/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py +++ b/tests/queries/0_stateless/01059_window_view_event_hop_watch_strict_asc.py @@ -30,7 +30,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send("CREATE TABLE db_01059_event_hop_watch_strict_asc.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()") client1.expect(prompt) - client1.send("CREATE WINDOW VIEW db_01059_event_hop_watch_strict_asc.wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, HOP_END(wid) as w_end FROM db_01059_event_hop_watch_strict_asc.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid;") + client1.send("CREATE WINDOW VIEW db_01059_event_hop_watch_strict_asc.wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(a) AS count, hopEnd(wid) as w_end FROM db_01059_event_hop_watch_strict_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid;") client1.expect(prompt) client1.send('WATCH db_01059_event_hop_watch_strict_asc.wv') diff --git a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql index 18b17fd3d2b..cb27e881870 100644 --- a/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql +++ b/tests/queries/0_stateless/01060_window_view_event_tumble_to_asc.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql index 2cf98d6b08f..c021bd1d4a1 100644 --- a/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql +++ b/tests/queries/0_stateless/01061_window_view_event_hop_to_asc.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py index 8541c7ee064..6be3e08665c 100755 --- a/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py +++ b/tests/queries/0_stateless/01062_window_view_event_hop_watch_asc.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('CREATE TABLE 01062_window_view_event_hop_watch_asc.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv WATERMARK=ASCENDING AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid") + client1.send("CREATE WINDOW VIEW 01062_window_view_event_hop_watch_asc.wv WATERMARK=ASCENDING AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM 01062_window_view_event_hop_watch_asc.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid") client1.expect(prompt) client1.send('WATCH 01062_window_view_event_hop_watch_asc.wv') diff --git a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql index 37757fd77b3..6b17d04517a 100644 --- a/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql +++ b/tests/queries/0_stateless/01063_window_view_event_tumble_to_bounded.sql @@ -7,7 +7,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK = INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK = INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql index 5f148900905..2f4b1c13d47 100644 --- a/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql +++ b/tests/queries/0_stateless/01064_window_view_event_hop_to_bounded.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS wv NO DELAY; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:01'); diff --git a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py index adae1073c80..b828c5116da 100755 --- a/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py +++ b/tests/queries/0_stateless/01065_window_view_event_hop_watch_bounded.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('CREATE TABLE test.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, HOP_END(wid) AS w_end FROM test.mt GROUP BY HOP(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid") + client1.send("CREATE WINDOW VIEW test.wv WATERMARK=INTERVAL '2' SECOND AS SELECT count(a) AS count, hopEnd(wid) AS w_end FROM test.mt GROUP BY hop(timestamp, INTERVAL '2' SECOND, INTERVAL '3' SECOND, 'US/Samoa') AS wid") client1.expect(prompt) client1.send('WATCH test.wv') diff --git a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql index 1c55b70f3aa..37830d506d6 100644 --- a/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql +++ b/tests/queries/0_stateless/01066_window_view_event_tumble_to_strict_asc_lateness.sql @@ -8,7 +8,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=STRICTLY_ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); diff --git a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql index 11409203d4c..eb57d9b6b15 100644 --- a/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql +++ b/tests/queries/0_stateless/01067_window_view_event_tumble_to_asc_lateness.sql @@ -8,7 +8,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); diff --git a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql index 74a095c632f..bc6d3a30947 100644 --- a/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql +++ b/tests/queries/0_stateless/01068_window_view_event_tumble_to_bounded_lateness.sql @@ -8,7 +8,7 @@ DROP TABLE IF EXISTS `.inner.wv`; CREATE TABLE dst(count UInt64, w_end DateTime) Engine=MergeTree ORDER BY tuple(); CREATE TABLE mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple(); -CREATE WINDOW VIEW wv TO dst WATERMARK=INTERVAL '2' SECOND ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, TUMBLE_END(wid) AS w_end FROM mt GROUP BY TUMBLE(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; +CREATE WINDOW VIEW wv TO dst WATERMARK=INTERVAL '2' SECOND ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND, 'US/Samoa') AS wid; INSERT INTO mt VALUES (1, '1990/01/01 12:00:00'); INSERT INTO mt VALUES (1, '1990/01/01 12:00:02'); diff --git a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py index 6f4f1795857..adab2988e39 100755 --- a/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py +++ b/tests/queries/0_stateless/01069_window_view_proc_tumble_watch.py @@ -32,7 +32,7 @@ with client(name='client1>', log=log) as client1, client(name='client2>', log=lo client1.send('CREATE TABLE 01069_window_view_proc_tumble_watch.mt(a Int32, timestamp DateTime) ENGINE=MergeTree ORDER BY tuple()') client1.expect(prompt) - client1.send("CREATE WINDOW VIEW 01069_window_view_proc_tumble_watch.wv AS SELECT count(a) AS count FROM 01069_window_view_proc_tumble_watch.mt GROUP BY TUMBLE(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid;") + client1.send("CREATE WINDOW VIEW 01069_window_view_proc_tumble_watch.wv AS SELECT count(a) AS count FROM 01069_window_view_proc_tumble_watch.mt GROUP BY tumble(timestamp, INTERVAL '1' SECOND, 'US/Samoa') AS wid;") client1.expect(prompt) client1.send('WATCH 01069_window_view_proc_tumble_watch.wv') From 9b9d5243596eb4e9a361da1a7b6a221eb42b3b9b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Dec 2021 12:12:20 +0300 Subject: [PATCH 566/609] Revert "Add a test with 20000 mutations in one query" --- .../02125_many_mutations.reference | 6 --- .../0_stateless/02125_many_mutations.sh | 49 ------------------- 2 files changed, 55 deletions(-) delete mode 100644 tests/queries/0_stateless/02125_many_mutations.reference delete mode 100755 tests/queries/0_stateless/02125_many_mutations.sh diff --git a/tests/queries/0_stateless/02125_many_mutations.reference b/tests/queries/0_stateless/02125_many_mutations.reference deleted file mode 100644 index c98d8221c7f..00000000000 --- a/tests/queries/0_stateless/02125_many_mutations.reference +++ /dev/null @@ -1,6 +0,0 @@ -0 0 -1 1 -20000 -0 -0 20000 -1 20001 diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh deleted file mode 100755 index 727cc9d6213..00000000000 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/env bash -# Tags: long - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x" -$CLICKHOUSE_CLIENT -q "insert into many_mutations values (0, 0), (1, 1)" -$CLICKHOUSE_CLIENT -q "system stop merges many_mutations" - -$CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x" - -job() -{ - for i in {1..1000} - do - $CLICKHOUSE_CLIENT -q "alter table many_mutations update y = y + 1 where 1" - done -} - -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & -job & - -wait - -$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" -$CLICKHOUSE_CLIENT -q "system start merges many_mutations" -$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" -$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" -$CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x" From 5717458f0c9d0df237b7aca2e826e509c4a18820 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Dec 2021 12:12:41 +0300 Subject: [PATCH 567/609] Revert "Revert "Add a test with 20000 mutations in one query"" --- .../02125_many_mutations.reference | 6 +++ .../0_stateless/02125_many_mutations.sh | 49 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 tests/queries/0_stateless/02125_many_mutations.reference create mode 100755 tests/queries/0_stateless/02125_many_mutations.sh diff --git a/tests/queries/0_stateless/02125_many_mutations.reference b/tests/queries/0_stateless/02125_many_mutations.reference new file mode 100644 index 00000000000..c98d8221c7f --- /dev/null +++ b/tests/queries/0_stateless/02125_many_mutations.reference @@ -0,0 +1,6 @@ +0 0 +1 1 +20000 +0 +0 20000 +1 20001 diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh new file mode 100755 index 00000000000..727cc9d6213 --- /dev/null +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash +# Tags: long + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "create table many_mutations (x UInt32, y UInt32) engine = MergeTree order by x" +$CLICKHOUSE_CLIENT -q "insert into many_mutations values (0, 0), (1, 1)" +$CLICKHOUSE_CLIENT -q "system stop merges many_mutations" + +$CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x" + +job() +{ + for i in {1..1000} + do + $CLICKHOUSE_CLIENT -q "alter table many_mutations update y = y + 1 where 1" + done +} + +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & +job & + +wait + +$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" +$CLICKHOUSE_CLIENT -q "system start merges many_mutations" +$CLICKHOUSE_CLIENT -q "optimize table many_mutations final" +$CLICKHOUSE_CLIENT -q "select count() from system.mutations where database = currentDatabase() and table = 'many_mutations' and not is_done" +$CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x" From deb6a06ce40b4a0f0efe60c1ca146971b70218b3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Dec 2021 12:18:26 +0300 Subject: [PATCH 568/609] Update 02125_many_mutations.sh --- tests/queries/0_stateless/02125_many_mutations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh index 727cc9d6213..603713e5e9f 100755 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: long +# Tags: long, no-tsan, no-debug, no-asan, no-msan, no-ubsan CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 03dc76da153f6bc4169acec7961b3a4355777b1f Mon Sep 17 00:00:00 2001 From: vxider Date: Tue, 7 Dec 2021 10:05:19 +0000 Subject: [PATCH 569/609] add window view doc(en) --- .../functions/window-functions.md | 114 +++++++++++++++++ .../sql-reference/statements/create/view.md | 118 +++++++++++++++++- 2 files changed, 231 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql-reference/functions/window-functions.md diff --git a/docs/en/sql-reference/functions/window-functions.md b/docs/en/sql-reference/functions/window-functions.md new file mode 100644 index 00000000000..66f33f512a7 --- /dev/null +++ b/docs/en/sql-reference/functions/window-functions.md @@ -0,0 +1,114 @@ +--- +toc_priority: 68 +toc_title: Window +--- + +# Window Functions {#window-functions} + +Window functions indicate the lower and upper window bound of records in WindowView. The functions for working with WindowView are listed below. + +## tumble {#window-functions-tumble} + +A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (interval). + +``` sql +tumble(time_attr, interval [, timezone]) +``` + +**Arguments** +- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type. +- `interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The lower and upper bound of the tumble window. + +Type: `Tuple(DateTime, DateTime)` + +**Example** + +Query: + +``` sql +SELECT tumble(now(), toIntervalDay('1')) +``` + +Result: + +``` text +┌─tumble(now(), toIntervalDay('1'))─────────────┐ +│ ['2020-01-01 00:00:00','2020-01-02 00:00:00'] │ +└───────────────────────────────────────────────┘ +``` + +## hop {#window-functions-hop} + +A hopping time window has a fixed duration (`window_interval`) and hops by a specified hop interval (`hop_interval`). If the `hop_interval` is smaller than the `window_interval`, hopping windows are overlapping. Thus, records can be assigned to multiple windows. + +``` sql +hop(time_attr, hop_interval, window_interval [, timezone]) +``` + +**Arguments** + +- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type. +- `hop_interval` - Hop interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number. +- `window_interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number. +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The lower and upper bound of the hop window. Since hop windows are + overlapped, the function only returns the bound of the **first** window when + hop function is used **without** `WINDOW VIEW`. + +Type: `Tuple(DateTime, DateTime)` + +**Example** + +Query: + +``` sql +SELECT hop(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) +``` + +Result: + +``` text +┌─hop(now(), toIntervalSecond('1'), toIntervalSecond('2'))──┐ +│ ('2020-01-14 16:58:22','2020-01-14 16:58:24') │ +└───────────────────────────────────────────────────────────┘ +``` + +## tumbleStart {#window-functions-tumblestart} + +Indicate the lower bound of a tumble function. + +``` sql +tumbleStart(time_attr, interval [, timezone]); +``` + +## tumbleEnd {#window-functions-tumbleend} + +Indicate the upper bound of a tumble function. + +``` sql +tumbleEnd(time_attr, interval [, timezone]); +``` + +## hopStart {#window-functions-hopstart} + +Indicate the lower bound of a hop function. + +``` sql +hopStart(time_attr, hop_interval, window_interval [, timezone]); +``` + +## hopEnd {#window-functions-hopend} + +Indicate the upper bound of a hop function. + +``` sql +hopEnd(time_attr, hop_interval, window_interval [, timezone]); +``` \ No newline at end of file diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index ec34c57a4cd..f67cc62f7db 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -5,7 +5,7 @@ toc_title: VIEW # CREATE VIEW {#create-view} -Creates a new view. Views can be [normal](#normal), [materialized](#materialized) and [live](#live-view) (the latter is an experimental feature). +Creates a new view. Views can be [normal](#normal), [materialized](#materialized), [live](#live-view), and [window](#window-view) (live view and window view are experimental features). ## Normal View {#normal} @@ -243,3 +243,119 @@ Most common uses of live view tables include: **See Also** - [ALTER LIVE VIEW](../alter/view.md#alter-live-view) + +## Window View [Experimental] {#window-view} + +!!! important "Important" + This is an experimental feature that may change in backwards-incompatible ways in the future releases. + Enable usage of window views and `WATCH` query using [allow_experimental_window_view](../../../operations/settings/settings.md#allow-experimental-window-view) setting. Input the command `set allow_experimental_window_view = 1`. + +``` sql +CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY window_function +``` + +Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table and can push the processing result to a specified table or push notifications using the WATCH query. + +Creating a window view is similar to creating `MATERIALIZED VIEW`. Window view needs an inner storage engine to store intermediate data. The inner storage will use `AggregatingMergeTree` as the default engine. + +### Window Functions {#window-view-windowfunctions} + +[WindowFunctions](../../functions/window-functions.md) are used to indicate the lower and upper window bound of records. The window view needs to be used with a window function. + +### TIME ATTRIBUTES {#window-view-timeattributes} + +Window view supports **processing time** and **event time** process. + +**Processing time** allows window view to produce results based on the local machine's time and is used by default. It is the most straightforward notion of time but does not provide determinism. The processing time attribute can be defined by setting the `time_attr` of the window function to a table column or using the function `now()`. The following query creates a window view with processing time. + +``` sql +CREATE WINDOW VIEW wv AS SELECT count(number), tumbleStart(w_id) as w_start from date GROUP BY tumble(now(), INTERVAL '5' SECOND) as w_id +``` + +**Event time** is the time that each individual event occurred on its producing device. This time is typically embedded within the records when it is generated. Event time processing allows for consistent results even in case of out-of-order events or late events. Window view supports event time processing by using `WATERMARK` syntax. + +Window view provides three watermark strategies. + +* `STRICTLY_ASCENDING`: Emits a watermark of the maximum observed timestamp so far. Rows that have a timestamp smaller to the max timestamp are not late. +* `ASCENDING`: Emits a watermark of the maximum observed timestamp so far minus 1. Rows that have a timestamp equal and smaller to the max timestamp are not late. +* `BOUNDED`: WATERMARK=INTERVAL. Emits watermarks, which are the maximum observed timestamp minus the specified delay. + +The following queries are examples of creating a window view with `WATERMARK`. + +``` sql +CREATE WINDOW VIEW wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); +CREATE WINDOW VIEW wv WATERMARK=ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); +CREATE WINDOW VIEW wv WATERMARK=INTERVAL '3' SECOND AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); +``` + +By default, the window will be fired when the watermark comes, and elements that arrived behind the watermark will be dropped. Window view supports late event processing by setting `ALLOWED_LATENESS=INTERVAL`. An example of lateness handling is: + +``` sql +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM test.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid; +``` + +Note that elements emitted by a late firing should be treated as updated results of a previous computation. Instead of firing at the end of windows, the window view will fire immediately when the late event arrives. Thus, it will result in multiple outputs for the same window. Users need to take these duplicated results into account or deduplicate them. + +### Monitoring New Windows{#window-view-monitoring} + +Window view supports the `WATCH` query to constantly append the processing results to the console or use `TO` syntax to output the results to a table. + +``` sql +WATCH [db.]name [LIMIT n] +``` + +`WATCH` query acts similar as in `LIVE VIEW`. A `LIMIT` can be specified to set the number of updates to receive before terminating the query. + +### Settings {#window-view-settings} + +- `window_view_clean_interval`: The clean interval of window view in seconds to free outdated data. The system will retain the windows that have not been fully triggered according to the system time or `WATERMARK` configuration, and the other data will be deleted. +- `window_view_heartbeat_interval`: The heartbeat interval in seconds to indicate the watch query is alive. + +### Example {#window-view-example} + +Suppose we need to count the number of click logs per 10 seconds in a log table called `data`, and its table structure is: + +``` sql +CREATE TABLE data ( `id` UInt64, `timestamp` DateTime) ENGINE = Memory; +``` + +First, we create a window view with tumble window of 10 seconds interval: + +``` sql +CREATE WINDOW VIEW wv as select count(id), tumbleStart(w_id) as window_start from data group by tumble(timestamp, INTERVAL '10' SECOND) as w_id +``` + +Then, we use the `WATCH` query to get the results. + +``` sql +WATCH wv +``` + +When logs are inserted into table `data`, + +``` sql +INSERT INTO data VALUES(1,now()) +``` + +The `WATCH` query should print the results as follows: + +``` text +┌─count(id)─┬────────window_start─┐ +│ 1 │ 2020-01-14 16:56:40 │ +└───────────┴─────────────────────┘ +``` + +Alternatively, we can attach the output to another table using `TO` syntax. + +``` sql +CREATE WINDOW VIEW wv TO dst AS SELECT count(id), tumbleStart(w_id) as window_start FROM data GROUP BY tumble(timestamp, INTERVAL '10' SECOND) as w_id +``` + +Additional examples can be found among stateful tests of ClickHouse (they are named `*window_view*` there). + +### Window View Usage {#window-view-usage} + +The window view is useful in the following scenarios: + +* **Monitoring**: Aggregate and calculate the metrics logs by time, and output the results to a target table. The dashboard can use the target table as a source table. +* **Analyzing**: Automatically aggregate and preprocess data in the time window. This can be useful when analyzing a large number of logs. The preprocessing eliminates repeated calculations in multiple queries and reduces query latency. From c7f0a400d812e4a97f3f3d225e28a2db9fcef7aa Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Dec 2021 10:31:48 +0000 Subject: [PATCH 570/609] Typo --- programs/local/LocalServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 561b0588787..1f27072f142 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -830,7 +830,7 @@ extern "C" int LLVMFuzzerInitialize(int * pargc, char *** pargv) char ** argv = *pargv; /// As a user you can add flags to clickhouse binary in fuzzing mode as follows - /// clickhouse -- + /// clickhouse -- /// Calculate the position of delimiter "--" that separates arguments /// of clickhouse-local and libfuzzer From ed17afbdb312b65d24ab383ca5cc59f841792825 Mon Sep 17 00:00:00 2001 From: vxider Date: Tue, 7 Dec 2021 10:31:49 +0000 Subject: [PATCH 571/609] rename window function to window view function --- .../functions/window-functions.md | 114 ------------------ .../sql-reference/statements/create/view.md | 8 +- 2 files changed, 4 insertions(+), 118 deletions(-) delete mode 100644 docs/en/sql-reference/functions/window-functions.md diff --git a/docs/en/sql-reference/functions/window-functions.md b/docs/en/sql-reference/functions/window-functions.md deleted file mode 100644 index 66f33f512a7..00000000000 --- a/docs/en/sql-reference/functions/window-functions.md +++ /dev/null @@ -1,114 +0,0 @@ ---- -toc_priority: 68 -toc_title: Window ---- - -# Window Functions {#window-functions} - -Window functions indicate the lower and upper window bound of records in WindowView. The functions for working with WindowView are listed below. - -## tumble {#window-functions-tumble} - -A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (interval). - -``` sql -tumble(time_attr, interval [, timezone]) -``` - -**Arguments** -- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type. -- `interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. -- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). - -**Returned values** - -- The lower and upper bound of the tumble window. - -Type: `Tuple(DateTime, DateTime)` - -**Example** - -Query: - -``` sql -SELECT tumble(now(), toIntervalDay('1')) -``` - -Result: - -``` text -┌─tumble(now(), toIntervalDay('1'))─────────────┐ -│ ['2020-01-01 00:00:00','2020-01-02 00:00:00'] │ -└───────────────────────────────────────────────┘ -``` - -## hop {#window-functions-hop} - -A hopping time window has a fixed duration (`window_interval`) and hops by a specified hop interval (`hop_interval`). If the `hop_interval` is smaller than the `window_interval`, hopping windows are overlapping. Thus, records can be assigned to multiple windows. - -``` sql -hop(time_attr, hop_interval, window_interval [, timezone]) -``` - -**Arguments** - -- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type. -- `hop_interval` - Hop interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number. -- `window_interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number. -- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). - -**Returned values** - -- The lower and upper bound of the hop window. Since hop windows are - overlapped, the function only returns the bound of the **first** window when - hop function is used **without** `WINDOW VIEW`. - -Type: `Tuple(DateTime, DateTime)` - -**Example** - -Query: - -``` sql -SELECT hop(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) -``` - -Result: - -``` text -┌─hop(now(), toIntervalSecond('1'), toIntervalSecond('2'))──┐ -│ ('2020-01-14 16:58:22','2020-01-14 16:58:24') │ -└───────────────────────────────────────────────────────────┘ -``` - -## tumbleStart {#window-functions-tumblestart} - -Indicate the lower bound of a tumble function. - -``` sql -tumbleStart(time_attr, interval [, timezone]); -``` - -## tumbleEnd {#window-functions-tumbleend} - -Indicate the upper bound of a tumble function. - -``` sql -tumbleEnd(time_attr, interval [, timezone]); -``` - -## hopStart {#window-functions-hopstart} - -Indicate the lower bound of a hop function. - -``` sql -hopStart(time_attr, hop_interval, window_interval [, timezone]); -``` - -## hopEnd {#window-functions-hopend} - -Indicate the upper bound of a hop function. - -``` sql -hopEnd(time_attr, hop_interval, window_interval [, timezone]); -``` \ No newline at end of file diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index f67cc62f7db..aa6b82360e0 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -251,22 +251,22 @@ Most common uses of live view tables include: Enable usage of window views and `WATCH` query using [allow_experimental_window_view](../../../operations/settings/settings.md#allow-experimental-window-view) setting. Input the command `set allow_experimental_window_view = 1`. ``` sql -CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY window_function +CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY window_view_function ``` Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table and can push the processing result to a specified table or push notifications using the WATCH query. Creating a window view is similar to creating `MATERIALIZED VIEW`. Window view needs an inner storage engine to store intermediate data. The inner storage will use `AggregatingMergeTree` as the default engine. -### Window Functions {#window-view-windowfunctions} +### Window View Functions {#window-view-windowviewfunctions} -[WindowFunctions](../../functions/window-functions.md) are used to indicate the lower and upper window bound of records. The window view needs to be used with a window function. +[Window view functions](../../functions/window-view-functions.md) are used to indicate the lower and upper window bound of records. The window view needs to be used with a window view function. ### TIME ATTRIBUTES {#window-view-timeattributes} Window view supports **processing time** and **event time** process. -**Processing time** allows window view to produce results based on the local machine's time and is used by default. It is the most straightforward notion of time but does not provide determinism. The processing time attribute can be defined by setting the `time_attr` of the window function to a table column or using the function `now()`. The following query creates a window view with processing time. +**Processing time** allows window view to produce results based on the local machine's time and is used by default. It is the most straightforward notion of time but does not provide determinism. The processing time attribute can be defined by setting the `time_attr` of the window view function to a table column or using the function `now()`. The following query creates a window view with processing time. ``` sql CREATE WINDOW VIEW wv AS SELECT count(number), tumbleStart(w_id) as w_start from date GROUP BY tumble(now(), INTERVAL '5' SECOND) as w_id From 6c16348faa59805ebf44b4bdd92675eee5a2ad17 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 7 Dec 2021 13:32:26 +0300 Subject: [PATCH 572/609] Fix division by zero in avgWeighted with Decimal argument (#32303) * fix division by zero * Update src/AggregateFunctions/AggregateFunctionAvg.h Co-authored-by: Dmitry Novik * remove trash * Update AggregateFunctionAvg.h Co-authored-by: Dmitry Novik --- src/AggregateFunctions/AggregateFunctionAvg.h | 35 +++++-------------- .../AggregateFunctionAvgWeighted.cpp | 6 ++-- .../01668_avg_weighted_ubsan.reference | 13 +++++++ .../0_stateless/01668_avg_weighted_ubsan.sql | 4 +++ 4 files changed, 28 insertions(+), 30 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAvg.h b/src/AggregateFunctions/AggregateFunctionAvg.h index e2a9220f113..8ca0ae1dac2 100644 --- a/src/AggregateFunctions/AggregateFunctionAvg.h +++ b/src/AggregateFunctions/AggregateFunctionAvg.h @@ -20,6 +20,7 @@ namespace DB { + struct Settings; template constexpr bool DecimalOrExtendedInt = @@ -42,39 +43,19 @@ struct AvgFraction /// Invoked only is either Numerator or Denominator are Decimal. Float64 NO_SANITIZE_UNDEFINED divideIfAnyDecimal(UInt32 num_scale, UInt32 denom_scale [[maybe_unused]]) const { - if constexpr (is_decimal && is_decimal) - { - // According to the docs, num(S1) / denom(S2) would have scale S1 - - if constexpr (std::is_same_v && std::is_same_v) - ///Special case as Decimal256 / Decimal128 = compile error (as Decimal128 is not parametrized by a wide - ///int), but an __int128 instead - return DecimalUtils::convertTo( - numerator / (denominator.template convertTo()), num_scale); - else - return DecimalUtils::convertTo(numerator / denominator, num_scale); - } - - /// Numerator is always casted to Float64 to divide correctly if the denominator is not Float64. - Float64 num_converted; - + Float64 numerator_float; if constexpr (is_decimal) - num_converted = DecimalUtils::convertTo(numerator, num_scale); + numerator_float = DecimalUtils::convertTo(numerator, num_scale); else - num_converted = static_cast(numerator); /// all other types, including extended integral. - - std::conditional_t, - Float64, Denominator> denom_converted; + numerator_float = numerator; + Float64 denominator_float; if constexpr (is_decimal) - denom_converted = DecimalUtils::convertTo(denominator, denom_scale); - else if constexpr (DecimalOrExtendedInt) - /// no way to divide Float64 and extended integral type without an explicit cast. - denom_converted = static_cast(denominator); + denominator_float = DecimalUtils::convertTo(denominator, denom_scale); else - denom_converted = denominator; /// can divide on float, no cast required. + denominator_float = denominator; - return num_converted / denom_converted; + return numerator_float / denominator_float; } Float64 NO_SANITIZE_UNDEFINED divide() const diff --git a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp index b7fdb3460e3..ab6fdc8fd7e 100644 --- a/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp +++ b/src/AggregateFunctions/AggregateFunctionAvgWeighted.cpp @@ -82,17 +82,17 @@ createAggregateFunctionAvgWeighted(const std::string & name, const DataTypes & a const bool left_decimal = isDecimal(data_type); const bool right_decimal = isDecimal(data_type_weight); + /// We multiply value by weight, so actual scale of numerator is + if (left_decimal && right_decimal) ptr.reset(create(*data_type, *data_type_weight, argument_types, - getDecimalScale(*data_type), getDecimalScale(*data_type_weight))); + getDecimalScale(*data_type) + getDecimalScale(*data_type_weight), getDecimalScale(*data_type_weight))); else if (left_decimal) ptr.reset(create(*data_type, *data_type_weight, argument_types, getDecimalScale(*data_type))); else if (right_decimal) ptr.reset(create(*data_type, *data_type_weight, argument_types, - // numerator is not decimal, so its scale is 0 - 0, getDecimalScale(*data_type_weight))); + getDecimalScale(*data_type_weight), getDecimalScale(*data_type_weight))); else ptr.reset(create(*data_type, *data_type_weight, argument_types)); diff --git a/tests/queries/0_stateless/01668_avg_weighted_ubsan.reference b/tests/queries/0_stateless/01668_avg_weighted_ubsan.reference index ec064f61ba7..a8921b27cff 100644 --- a/tests/queries/0_stateless/01668_avg_weighted_ubsan.reference +++ b/tests/queries/0_stateless/01668_avg_weighted_ubsan.reference @@ -1 +1,14 @@ -0 +nan +nan +1 +2 +3 +4 +5 +6 +7 +8 +9 +nan +nan diff --git a/tests/queries/0_stateless/01668_avg_weighted_ubsan.sql b/tests/queries/0_stateless/01668_avg_weighted_ubsan.sql index 24e7dc0cb90..1c31c23eaee 100644 --- a/tests/queries/0_stateless/01668_avg_weighted_ubsan.sql +++ b/tests/queries/0_stateless/01668_avg_weighted_ubsan.sql @@ -1 +1,5 @@ SELECT round(avgWeighted(x, y)) FROM (SELECT 1023 AS x, 1000000000 AS y UNION ALL SELECT 10 AS x, -9223372036854775808 AS y); +select avgWeighted(number, toDecimal128(number, 9)) from numbers(0); +SELECT avgWeighted(a, toDecimal64(c, 9)) OVER (PARTITION BY c) FROM (SELECT number AS a, number AS c FROM numbers(10)); +select avg(toDecimal128(number, 9)) from numbers(0); +select avgWeighted(number, toDecimal128(0, 9)) from numbers(10); From 7e036d31e976625f4ce8f0e60b6bb7d9fed6f4c8 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Dec 2021 11:29:46 +0000 Subject: [PATCH 573/609] One more style related commit --- src/Functions/getFuzzerData.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Functions/getFuzzerData.h b/src/Functions/getFuzzerData.h index 06f11f28e70..635ca2bdce9 100644 --- a/src/Functions/getFuzzerData.h +++ b/src/Functions/getFuzzerData.h @@ -1,3 +1,5 @@ +#pragma once + #include #include #include From 877716b2b005401bcc81d10ad1e48a4c9b9d9b90 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 7 Dec 2021 15:46:19 +0300 Subject: [PATCH 574/609] Update 02125_many_mutations.sh --- tests/queries/0_stateless/02125_many_mutations.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02125_many_mutations.sh b/tests/queries/0_stateless/02125_many_mutations.sh index 603713e5e9f..7a89e5f7c4f 100755 --- a/tests/queries/0_stateless/02125_many_mutations.sh +++ b/tests/queries/0_stateless/02125_many_mutations.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select x, y from many_mutations order by x" job() { - for i in {1..1000} + for _ in {1..1000} do $CLICKHOUSE_CLIENT -q "alter table many_mutations update y = y + 1 where 1" done From afcc3b6f965ce032f68631ca72596423a914bbce Mon Sep 17 00:00:00 2001 From: vxider Date: Tue, 7 Dec 2021 13:35:30 +0000 Subject: [PATCH 575/609] add doc window-view-functions --- .../functions/window-view-functions.md | 114 ++++++++++++++++++ 1 file changed, 114 insertions(+) create mode 100644 docs/en/sql-reference/functions/window-view-functions.md diff --git a/docs/en/sql-reference/functions/window-view-functions.md b/docs/en/sql-reference/functions/window-view-functions.md new file mode 100644 index 00000000000..5684e93bd88 --- /dev/null +++ b/docs/en/sql-reference/functions/window-view-functions.md @@ -0,0 +1,114 @@ +--- +toc_priority: 68 +toc_title: Window View +--- + +# Window View Functions {#window-view-functions} + +Window functions indicate the lower and upper window bound of records in WindowView. The functions for working with WindowView are listed below. + +## tumble {#window-view-functions-tumble} + +A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (interval). + +``` sql +tumble(time_attr, interval [, timezone]) +``` + +**Arguments** +- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type. +- `interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The lower and upper bound of the tumble window. + +Type: `Tuple(DateTime, DateTime)` + +**Example** + +Query: + +``` sql +SELECT tumble(now(), toIntervalDay('1')) +``` + +Result: + +``` text +┌─tumble(now(), toIntervalDay('1'))─────────────┐ +│ ['2020-01-01 00:00:00','2020-01-02 00:00:00'] │ +└───────────────────────────────────────────────┘ +``` + +## hop {#window-view-functions-hop} + +A hopping time window has a fixed duration (`window_interval`) and hops by a specified hop interval (`hop_interval`). If the `hop_interval` is smaller than the `window_interval`, hopping windows are overlapping. Thus, records can be assigned to multiple windows. + +``` sql +hop(time_attr, hop_interval, window_interval [, timezone]) +``` + +**Arguments** + +- `time_attr` - Date and time. [DateTime](../../sql-reference/data-types/datetime.md) data type. +- `hop_interval` - Hop interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number. +- `window_interval` - Window interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. Should be a positive number. +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) (optional). + +**Returned values** + +- The lower and upper bound of the hop window. Since hop windows are + overlapped, the function only returns the bound of the **first** window when + hop function is used **without** `WINDOW VIEW`. + +Type: `Tuple(DateTime, DateTime)` + +**Example** + +Query: + +``` sql +SELECT hop(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) +``` + +Result: + +``` text +┌─hop(now(), toIntervalSecond('1'), toIntervalSecond('2'))──┐ +│ ('2020-01-14 16:58:22','2020-01-14 16:58:24') │ +└───────────────────────────────────────────────────────────┘ +``` + +## tumbleStart {#window-view-functions-tumblestart} + +Indicate the lower bound of a tumble function. + +``` sql +tumbleStart(time_attr, interval [, timezone]); +``` + +## tumbleEnd {#window-view-functions-tumbleend} + +Indicate the upper bound of a tumble function. + +``` sql +tumbleEnd(time_attr, interval [, timezone]); +``` + +## hopStart {#window-view-functions-hopstart} + +Indicate the lower bound of a hop function. + +``` sql +hopStart(time_attr, hop_interval, window_interval [, timezone]); +``` + +## hopEnd {#window-view-functions-hopend} + +Indicate the upper bound of a hop function. + +``` sql +hopEnd(time_attr, hop_interval, window_interval [, timezone]); +``` \ No newline at end of file From ef7ae7cb7571a95dd3966d89d4851e50b33df7af Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 7 Dec 2021 16:42:16 +0300 Subject: [PATCH 576/609] fix MATERIALIZE COLUMN in case when data type of expression is not equal to data type of column --- src/Interpreters/MutationsInterpreter.cpp | 5 ++- .../02131_materialize_column_cast.reference | 14 ++++++++ .../02131_materialize_column_cast.sql | 35 +++++++++++++++++++ 3 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02131_materialize_column_cast.reference create mode 100644 tests/queries/0_stateless/02131_materialize_column_cast.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 471ad67d4e7..ecf79c03445 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -569,7 +569,10 @@ ASTPtr MutationsInterpreter::prepare(bool dry_run) stages.emplace_back(context); const auto & column = columns_desc.get(command.column_name); - stages.back().column_to_updated.emplace(column.name, column.default_desc.expression->clone()); + auto materialized_column = makeASTFunction( + "_CAST", column.default_desc.expression->clone(), std::make_shared(column.type->getName())); + + stages.back().column_to_updated.emplace(column.name, materialized_column); } else if (command.type == MutationCommand::MATERIALIZE_INDEX) { diff --git a/tests/queries/0_stateless/02131_materialize_column_cast.reference b/tests/queries/0_stateless/02131_materialize_column_cast.reference new file mode 100644 index 00000000000..8c94b2ade9c --- /dev/null +++ b/tests/queries/0_stateless/02131_materialize_column_cast.reference @@ -0,0 +1,14 @@ +1_1_1_0_2 i Int32 +1_1_1_0_2 s LowCardinality(String) +=========== +1_1_1_0_2 i Int32 +1_1_1_0_2 s LowCardinality(String) +2_3_3_0 i Int32 +2_3_3_0 s LowCardinality(String) +=========== +1_1_1_0_4 i Int32 +1_1_1_0_4 s LowCardinality(String) +2_3_3_0_4 i Int32 +2_3_3_0_4 s LowCardinality(String) +1 1 +2 2 diff --git a/tests/queries/0_stateless/02131_materialize_column_cast.sql b/tests/queries/0_stateless/02131_materialize_column_cast.sql new file mode 100644 index 00000000000..3bfeaf5baeb --- /dev/null +++ b/tests/queries/0_stateless/02131_materialize_column_cast.sql @@ -0,0 +1,35 @@ +DROP TABLE IF EXISTS t_materialize_column; + +CREATE TABLE t_materialize_column (i Int32) +ENGINE = MergeTree ORDER BY i PARTITION BY i +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO t_materialize_column VALUES (1); + +ALTER TABLE t_materialize_column ADD COLUMN s LowCardinality(String) DEFAULT toString(i); +ALTER TABLE t_materialize_column MATERIALIZE COLUMN s SETTINGS mutations_sync = 2; + +SELECT name, column, type FROM system.parts_columns +WHERE table = 't_materialize_column' AND database = currentDatabase() AND active +ORDER BY name, column; + +SELECT '==========='; + +INSERT INTO t_materialize_column (i) VALUES (2); + +SELECT name, column, type FROM system.parts_columns +WHERE table = 't_materialize_column' AND database = currentDatabase() AND active +ORDER BY name, column; + +SELECT '==========='; + +ALTER TABLE t_materialize_column ADD INDEX s_bf (s) TYPE bloom_filter(0.01) GRANULARITY 1; +ALTER TABLE t_materialize_column MATERIALIZE INDEX s_bf SETTINGS mutations_sync = 2; + +SELECT name, column, type FROM system.parts_columns +WHERE table = 't_materialize_column' AND database = currentDatabase() AND active +ORDER BY name, column; + +SELECT * FROM t_materialize_column ORDER BY i; + +DROP TABLE t_materialize_column; From b41552492e9496edcbd3a9917a4613b98efb68ca Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Dec 2021 15:28:09 +0000 Subject: [PATCH 577/609] Done --- utils/graphite-rollup/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/graphite-rollup/CMakeLists.txt b/utils/graphite-rollup/CMakeLists.txt index bd6a078fbd5..3cc0d3e756f 100644 --- a/utils/graphite-rollup/CMakeLists.txt +++ b/utils/graphite-rollup/CMakeLists.txt @@ -9,7 +9,7 @@ target_link_libraries( ) target_include_directories( graphite-rollup-bench - PRIVATE + SYSTEM PRIVATE ${ClickHouse_SOURCE_DIR}/src ${CMAKE_BINARY_DIR}/src ${ClickHouse_SOURCE_DIR}/base ${ClickHouse_SOURCE_DIR}/base/pcg-random ${CMAKE_BINARY_DIR}/src/Core/include From 76f91c2c6cd8e324e0342a2fb096ac64ad368d48 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 7 Dec 2021 16:50:57 +0000 Subject: [PATCH 578/609] Done --- src/Interpreters/MutationsInterpreter.cpp | 4 ++-- .../02132_empty_mutation_livelock.reference | 2 ++ .../0_stateless/02132_empty_mutation_livelock.sql | 12 ++++++++++++ 3 files changed, 16 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02132_empty_mutation_livelock.reference create mode 100644 tests/queries/0_stateless/02132_empty_mutation_livelock.sql diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 471ad67d4e7..d7443ca6edf 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -231,8 +231,8 @@ bool isStorageTouchedByMutations( PullingPipelineExecutor executor(io.pipeline); Block block; - while (!block.rows()) - executor.pull(block); + while (executor.pull(block)) {} + if (!block.rows()) return false; else if (block.rows() != 1) diff --git a/tests/queries/0_stateless/02132_empty_mutation_livelock.reference b/tests/queries/0_stateless/02132_empty_mutation_livelock.reference new file mode 100644 index 00000000000..e58e9764b39 --- /dev/null +++ b/tests/queries/0_stateless/02132_empty_mutation_livelock.reference @@ -0,0 +1,2 @@ +100 +100 diff --git a/tests/queries/0_stateless/02132_empty_mutation_livelock.sql b/tests/queries/0_stateless/02132_empty_mutation_livelock.sql new file mode 100644 index 00000000000..186199d4e13 --- /dev/null +++ b/tests/queries/0_stateless/02132_empty_mutation_livelock.sql @@ -0,0 +1,12 @@ +drop table if exists a8x; + +set empty_result_for_aggregation_by_empty_set=1; +create table a8x ENGINE = MergeTree ORDER BY tuple() settings min_bytes_for_wide_part=0 as SELECT number FROM system.numbers limit 100; + +select count() from a8x; + +set mutations_sync=1; +alter table a8x update number=0 WHERE number=-3; + +select count() from a8x; +drop table if exists a8x; From 4f46ac6b30f40939e3004f4ad683a776143b2cb7 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 7 Dec 2021 19:55:55 +0300 Subject: [PATCH 579/609] Remove LeaderElection (#32140) * remove LeaderElection * try fix tests * Update test.py * Update test.py --- src/Storages/MergeTree/LeaderElection.h | 170 ++++++------------ .../PartMovesBetweenShardsOrchestrator.cpp | 1 + .../ReplicatedMergeTreeCleanupThread.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.cpp | 7 - src/Storages/StorageReplicatedMergeTree.cpp | 60 +++---- src/Storages/StorageReplicatedMergeTree.h | 15 +- .../test_backward_compatibility/test.py | 15 +- .../test.py | 3 + 8 files changed, 90 insertions(+), 183 deletions(-) diff --git a/src/Storages/MergeTree/LeaderElection.h b/src/Storages/MergeTree/LeaderElection.h index ccc5fada537..afaf2e7e841 100644 --- a/src/Storages/MergeTree/LeaderElection.h +++ b/src/Storages/MergeTree/LeaderElection.h @@ -1,9 +1,6 @@ #pragma once -#include -#include #include -#include #include #include #include @@ -17,135 +14,74 @@ namespace zkutil * * But then we decided to get rid of leader election, so every replica can become leader. * For now, every replica can become leader if there is no leader among replicas with old version. - * - * It's tempting to remove this class at all, but we have to maintain it, - * to maintain compatibility when replicas with different versions work on the same cluster - * (this is allowed for short time period during cluster update). - * - * Replicas with new versions creates ephemeral sequential nodes with values like "replica_name (multiple leaders Ok)". - * If the first node belongs to a replica with new version, then all replicas with new versions become leaders. */ -class LeaderElection + +void checkNoOldLeaders(Poco::Logger * log, ZooKeeper & zookeeper, const String path) { -public: - using LeadershipHandler = std::function; + /// Previous versions (before 21.12) used to create ephemeral sequential node path/leader_election- + /// Replica with the lexicographically smallest node name becomes leader (before 20.6) or enables multi-leader mode (since 20.6) + constexpr auto persistent_multiple_leaders = "leader_election-0"; /// Less than any sequential node + constexpr auto suffix = " (multiple leaders Ok)"; + constexpr auto persistent_identifier = "all (multiple leaders Ok)"; - /** handler is called when this instance become leader. - * - * identifier - if not empty, must uniquely (within same path) identify participant of leader election. - * It means that different participants of leader election have different identifiers - * and existence of more than one ephemeral node with same identifier indicates an error. - */ - LeaderElection( - DB::BackgroundSchedulePool & pool_, - const std::string & path_, - ZooKeeper & zookeeper_, - LeadershipHandler handler_, - const std::string & identifier_) - : pool(pool_), path(path_), zookeeper(zookeeper_), handler(handler_), identifier(identifier_ + suffix) - , log_name("LeaderElection (" + path + ")") - , log(&Poco::Logger::get(log_name)) + size_t num_tries = 1000; + while (num_tries--) { - task = pool.createTask(log_name, [this] { threadFunction(); }); - createNode(); - } - - void shutdown() - { - if (shutdown_called) + Strings potential_leaders; + Coordination::Error code = zookeeper.tryGetChildren(path, potential_leaders); + /// NOTE zookeeper_path/leader_election node must exist now, but maybe we will remove it in future versions. + if (code == Coordination::Error::ZNONODE) return; + else if (code != Coordination::Error::ZOK) + throw KeeperException(code, path); - shutdown_called = true; - task->deactivate(); - } + Coordination::Requests ops; - ~LeaderElection() - { - releaseNode(); - } - -private: - static inline constexpr auto suffix = " (multiple leaders Ok)"; - DB::BackgroundSchedulePool & pool; - DB::BackgroundSchedulePool::TaskHolder task; - std::string path; - ZooKeeper & zookeeper; - LeadershipHandler handler; - std::string identifier; - std::string log_name; - Poco::Logger * log; - - EphemeralNodeHolderPtr node; - std::string node_name; - - std::atomic shutdown_called {false}; - - void createNode() - { - shutdown_called = false; - node = EphemeralNodeHolder::createSequential(fs::path(path) / "leader_election-", zookeeper, identifier); - - std::string node_path = node->getPath(); - node_name = node_path.substr(node_path.find_last_of('/') + 1); - - task->activateAndSchedule(); - } - - void releaseNode() - { - shutdown(); - node = nullptr; - } - - void threadFunction() - { - bool success = false; - - try + if (potential_leaders.empty()) { - Strings children = zookeeper.getChildren(path); - std::sort(children.begin(), children.end()); - - auto my_node_it = std::lower_bound(children.begin(), children.end(), node_name); - if (my_node_it == children.end() || *my_node_it != node_name) - throw Poco::Exception("Assertion failed in LeaderElection"); - - String value = zookeeper.get(path + "/" + children.front()); - - if (value.ends_with(suffix)) - { - handler(); + /// Ensure that no leaders appeared and enable persistent multi-leader mode + /// May fail with ZNOTEMPTY + ops.emplace_back(makeRemoveRequest(path, 0)); + ops.emplace_back(makeCreateRequest(path, "", zkutil::CreateMode::Persistent)); + /// May fail with ZNODEEXISTS + ops.emplace_back(makeCreateRequest(fs::path(path) / persistent_multiple_leaders, persistent_identifier, zkutil::CreateMode::Persistent)); + } + else + { + if (potential_leaders.front() == persistent_multiple_leaders) return; + + /// Ensure that current leader supports multi-leader mode and make it persistent + auto current_leader = fs::path(path) / potential_leaders.front(); + Coordination::Stat leader_stat; + String identifier; + if (!zookeeper.tryGet(current_leader, identifier, &leader_stat)) + { + LOG_INFO(log, "LeaderElection: leader suddenly changed, will retry"); + continue; } - if (my_node_it == children.begin()) - throw Poco::Exception("Assertion failed in LeaderElection"); + if (!identifier.ends_with(suffix)) + throw Poco::Exception(fmt::format("Found leader replica ({}) with too old version (< 20.6). Stop it before upgrading", identifier)); - /// Watch for the node in front of us. - --my_node_it; - std::string get_path_value; - if (!zookeeper.tryGetWatch(path + "/" + *my_node_it, get_path_value, nullptr, task->getWatchCallback())) - task->schedule(); - - success = true; - } - catch (const KeeperException & e) - { - DB::tryLogCurrentException(log); - - if (e.code == Coordination::Error::ZSESSIONEXPIRED) - return; - } - catch (...) - { - DB::tryLogCurrentException(log); + /// Version does not matter, just check that it still exists. + /// May fail with ZNONODE + ops.emplace_back(makeCheckRequest(current_leader, leader_stat.version)); + /// May fail with ZNODEEXISTS + ops.emplace_back(makeCreateRequest(fs::path(path) / persistent_multiple_leaders, persistent_identifier, zkutil::CreateMode::Persistent)); } - if (!success) - task->scheduleAfter(10 * 1000); + Coordination::Responses res; + code = zookeeper.tryMulti(ops, res); + if (code == Coordination::Error::ZOK) + return; + else if (code == Coordination::Error::ZNOTEMPTY || code == Coordination::Error::ZNODEEXISTS || code == Coordination::Error::ZNONODE) + LOG_INFO(log, "LeaderElection: leader suddenly changed or new node appeared, will retry"); + else + KeeperMultiException::check(code, ops, res); } -}; -using LeaderElectionPtr = std::shared_ptr; + throw Poco::Exception("Cannot check that no old leaders exist"); +} } diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index b3a17250549..4d18adc1dfc 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 5731092f2a8..ff37a341205 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -2,9 +2,9 @@ #include #include #include +#include #include -#include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 3bb592dcdcb..0cc6955ff72 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -197,11 +197,6 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() updateQuorumIfWeHavePart(); - if (storage_settings->replicated_can_become_leader) - storage.enterLeaderElection(); - else - LOG_INFO(log, "Will not enter leader election because replicated_can_become_leader=0"); - /// Anything above can throw a KeeperException if something is wrong with ZK. /// Anything below should not throw exceptions. @@ -380,8 +375,6 @@ void ReplicatedMergeTreeRestartingThread::partialShutdown() LOG_TRACE(log, "Waiting for threads to finish"); - storage.exitLeaderElection(); - storage.queue_updating_task->deactivate(); storage.mutations_updating_task->deactivate(); storage.mutations_finalizing_task->deactivate(); diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 7f600fc054c..852e2b10e6c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -32,6 +32,7 @@ #include #include #include +#include #include @@ -3400,53 +3401,29 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n } -void StorageReplicatedMergeTree::enterLeaderElection() +void StorageReplicatedMergeTree::startBeingLeader() { - auto callback = [this]() + if (!getSettings()->replicated_can_become_leader) { - LOG_INFO(log, "Became leader"); - - is_leader = true; - merge_selecting_task->activateAndSchedule(); - }; - - try - { - leader_election = std::make_shared( - getContext()->getSchedulePool(), - fs::path(zookeeper_path) / "leader_election", - *current_zookeeper, /// current_zookeeper lives for the lifetime of leader_election, - /// since before changing `current_zookeeper`, `leader_election` object is destroyed in `partialShutdown` method. - callback, - replica_name); - } - catch (...) - { - leader_election = nullptr; - throw; + LOG_INFO(log, "Will not enter leader election because replicated_can_become_leader=0"); + return; } + + zkutil::checkNoOldLeaders(log, *current_zookeeper, fs::path(zookeeper_path) / "leader_election"); + + LOG_INFO(log, "Became leader"); + is_leader = true; + merge_selecting_task->activateAndSchedule(); } -void StorageReplicatedMergeTree::exitLeaderElection() +void StorageReplicatedMergeTree::stopBeingLeader() { - if (!leader_election) + if (!is_leader) return; - /// Shut down the leader election thread to avoid suddenly becoming the leader again after - /// we have stopped the merge_selecting_thread, but before we have deleted the leader_election object. - leader_election->shutdown(); - - if (is_leader) - { - LOG_INFO(log, "Stopped being leader"); - - is_leader = false; - merge_selecting_task->deactivate(); - } - - /// Delete the node in ZK only after we have stopped the merge_selecting_thread - so that only one - /// replica assigns merges at any given time. - leader_election = nullptr; + LOG_INFO(log, "Stopped being leader"); + is_leader = false; + merge_selecting_task->deactivate(); } ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(ContextPtr local_context) @@ -4109,10 +4086,12 @@ void StorageReplicatedMergeTree::startup() assert(prev_ptr == nullptr); getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); + startBeingLeader(); + /// In this thread replica will be activated. restarting_thread.start(); - /// Wait while restarting_thread initializes LeaderElection (and so on) or makes first attempt to do it + /// Wait while restarting_thread finishing initialization startup_event.wait(); startBackgroundMovesIfNeeded(); @@ -4145,6 +4124,7 @@ void StorageReplicatedMergeTree::shutdown() fetcher.blocker.cancelForever(); merger_mutator.merges_blocker.cancelForever(); parts_mover.moves_blocker.cancelForever(); + stopBeingLeader(); restarting_thread.shutdown(); background_operations_assignee.finish(); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 65daf82a633..bcd364df30e 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include @@ -320,7 +319,6 @@ private: * It can be false only when old ClickHouse versions are working on the same cluster, because now we allow multiple leaders. */ std::atomic is_leader {false}; - zkutil::LeaderElectionPtr leader_election; InterserverIOEndpointPtr data_parts_exchange_endpoint; @@ -514,15 +512,10 @@ private: bool processQueueEntry(ReplicatedMergeTreeQueue::SelectedEntryPtr entry); - /// Postcondition: - /// either leader_election is fully initialized (node in ZK is created and the watching thread is launched) - /// or an exception is thrown and leader_election is destroyed. - void enterLeaderElection(); - - /// Postcondition: - /// is_leader is false, merge_selecting_thread is stopped, leader_election is nullptr. - /// leader_election node in ZK is either deleted, or the session is marked expired. - void exitLeaderElection(); + /// Start being leader (if not disabled by setting). + /// Since multi-leaders are allowed, it just sets is_leader flag. + void startBeingLeader(); + void stopBeingLeader(); /** Selects the parts to merge and writes to the log. */ diff --git a/tests/integration/test_backward_compatibility/test.py b/tests/integration/test_backward_compatibility/test.py index 71aedb78e5b..a8f4968956c 100644 --- a/tests/integration/test_backward_compatibility/test.py +++ b/tests/integration/test_backward_compatibility/test.py @@ -11,13 +11,14 @@ node2 = cluster.add_instance('node2', main_configs=['configs/wide_parts_only.xml def start_cluster(): try: cluster.start() - for i, node in enumerate([node1, node2]): - node.query_with_retry( - '''CREATE TABLE t(date Date, id UInt32) - ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}') - PARTITION BY toYYYYMM(date) - ORDER BY id'''.format(i)) - + create_query = '''CREATE TABLE t(date Date, id UInt32) + ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/t', '{}') + PARTITION BY toYYYYMM(date) + ORDER BY id''' + node1.query(create_query.format(1)) + node1.query("DETACH TABLE t") # stop being leader + node2.query(create_query.format(2)) + node1.query("ATTACH TABLE t") yield cluster finally: diff --git a/tests/integration/test_version_update_after_mutation/test.py b/tests/integration/test_version_update_after_mutation/test.py index 8d38234ccdd..3c22f2ed380 100644 --- a/tests/integration/test_version_update_after_mutation/test.py +++ b/tests/integration/test_version_update_after_mutation/test.py @@ -36,6 +36,8 @@ def test_mutate_and_upgrade(start_cluster): node1.query("ALTER TABLE mt DELETE WHERE id = 2", settings={"mutations_sync": "2"}) node2.query("SYSTEM SYNC REPLICA mt", timeout=15) + node2.query("DETACH TABLE mt") # stop being leader + node1.query("DETACH TABLE mt") # stop being leader node1.restart_with_latest_version(signal=9) node2.restart_with_latest_version(signal=9) @@ -83,6 +85,7 @@ def test_upgrade_while_mutation(start_cluster): node3.query("SYSTEM STOP MERGES mt1") node3.query("ALTER TABLE mt1 DELETE WHERE id % 2 == 0") + node3.query("DETACH TABLE mt1") # stop being leader node3.restart_with_latest_version(signal=9) # checks for readonly From 785d79531de5cac486bf7b00b6718452074dc42b Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 8 Dec 2021 15:27:32 +0300 Subject: [PATCH 580/609] Update StorageReplicatedMergeTree.cpp --- src/Storages/StorageReplicatedMergeTree.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 852e2b10e6c..89506184354 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4086,14 +4086,15 @@ void StorageReplicatedMergeTree::startup() assert(prev_ptr == nullptr); getContext()->getInterserverIOHandler().addEndpoint(data_parts_exchange_ptr->getId(replica_path), data_parts_exchange_ptr); - startBeingLeader(); - /// In this thread replica will be activated. restarting_thread.start(); /// Wait while restarting_thread finishing initialization startup_event.wait(); + /// Restarting thread has initialized replication queue, replica can become leader now + startBeingLeader(); + startBackgroundMovesIfNeeded(); part_moves_between_shards_orchestrator.start(); From 398d2f5984af4f32054eff08191d64784b65c473 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Wed, 8 Dec 2021 12:36:55 +0000 Subject: [PATCH 581/609] Fix --- src/Storages/MergeTree/tests/gtest_executor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/tests/gtest_executor.cpp b/src/Storages/MergeTree/tests/gtest_executor.cpp index a208e7dc233..d2895215ebe 100644 --- a/src/Storages/MergeTree/tests/gtest_executor.cpp +++ b/src/Storages/MergeTree/tests/gtest_executor.cpp @@ -147,7 +147,7 @@ TEST(Executor, RemoveTasksStress) for (size_t j = 0; j < tasks_kinds; ++j) executor->removeTasksCorrespondingToStorage({"test", std::to_string(j)}); - ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundMergesAndMutationsPoolTask], 0); - executor->wait(); + + ASSERT_EQ(CurrentMetrics::values[CurrentMetrics::BackgroundMergesAndMutationsPoolTask], 0); } From eb759c83f6f899b046fd9a4eb3d16312a9f91a55 Mon Sep 17 00:00:00 2001 From: vxider Date: Wed, 8 Dec 2021 14:43:26 +0000 Subject: [PATCH 582/609] window view docs improvement --- .../functions/window-view-functions.md | 18 ++- .../sql-reference/statements/create/view.md | 8 +- .../functions/window-view-functions.md | 112 +++++++++++++++++ .../sql-reference/statements/create/view.md | 119 +++++++++++++++++- 4 files changed, 242 insertions(+), 15 deletions(-) create mode 100644 docs/zh/sql-reference/functions/window-view-functions.md diff --git a/docs/en/sql-reference/functions/window-view-functions.md b/docs/en/sql-reference/functions/window-view-functions.md index 5684e93bd88..3f560aa96b9 100644 --- a/docs/en/sql-reference/functions/window-view-functions.md +++ b/docs/en/sql-reference/functions/window-view-functions.md @@ -5,11 +5,11 @@ toc_title: Window View # Window View Functions {#window-view-functions} -Window functions indicate the lower and upper window bound of records in WindowView. The functions for working with WindowView are listed below. +Window view functions return the inclusive lower and exclusive upper bound of the corresponding window. The functions for working with WindowView are listed below: ## tumble {#window-view-functions-tumble} -A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (interval). +A tumbling time window assigns records to non-overlapping, continuous windows with a fixed duration (`interval`). ``` sql tumble(time_attr, interval [, timezone]) @@ -22,7 +22,7 @@ tumble(time_attr, interval [, timezone]) **Returned values** -- The lower and upper bound of the tumble window. +- The inclusive lower and exclusive upper bound of the corresponding tumbling window. Type: `Tuple(DateTime, DateTime)` @@ -59,9 +59,7 @@ hop(time_attr, hop_interval, window_interval [, timezone]) **Returned values** -- The lower and upper bound of the hop window. Since hop windows are - overlapped, the function only returns the bound of the **first** window when - hop function is used **without** `WINDOW VIEW`. +- The inclusive lower and exclusive upper bound of the corresponding hopping window. Since one record can be assigned to multiple hop windows, the function only returns the bound of the **first** window when hop function is used **without** `WINDOW VIEW`. Type: `Tuple(DateTime, DateTime)` @@ -83,7 +81,7 @@ Result: ## tumbleStart {#window-view-functions-tumblestart} -Indicate the lower bound of a tumble function. +Returns the inclusive lower bound of the corresponding tumbling window. ``` sql tumbleStart(time_attr, interval [, timezone]); @@ -91,7 +89,7 @@ tumbleStart(time_attr, interval [, timezone]); ## tumbleEnd {#window-view-functions-tumbleend} -Indicate the upper bound of a tumble function. +Returns the exclusive upper bound of the corresponding tumbling window. ``` sql tumbleEnd(time_attr, interval [, timezone]); @@ -99,7 +97,7 @@ tumbleEnd(time_attr, interval [, timezone]); ## hopStart {#window-view-functions-hopstart} -Indicate the lower bound of a hop function. +Returns the inclusive lower bound of the corresponding hopping window. ``` sql hopStart(time_attr, hop_interval, window_interval [, timezone]); @@ -107,7 +105,7 @@ hopStart(time_attr, hop_interval, window_interval [, timezone]); ## hopEnd {#window-view-functions-hopend} -Indicate the upper bound of a hop function. +Returns the exclusive upper bound of the corresponding hopping window. ``` sql hopEnd(time_attr, hop_interval, window_interval [, timezone]); diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index aa6b82360e0..464de02eac6 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -254,13 +254,13 @@ Most common uses of live view tables include: CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY window_view_function ``` -Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table and can push the processing result to a specified table or push notifications using the WATCH query. +Window view can aggregate data by time window and output the results when the window is ready to fire. It stores the partial aggregation results in an inner(or specified) table to reduce latency and can push the processing result to a specified table or push notifications using the WATCH query. Creating a window view is similar to creating `MATERIALIZED VIEW`. Window view needs an inner storage engine to store intermediate data. The inner storage will use `AggregatingMergeTree` as the default engine. ### Window View Functions {#window-view-windowviewfunctions} -[Window view functions](../../functions/window-view-functions.md) are used to indicate the lower and upper window bound of records. The window view needs to be used with a window view function. +[Window view functions](../../functions/window-view-functions.md) are used to get the lower and upper window bound of records. The window view needs to be used with a window view function. ### TIME ATTRIBUTES {#window-view-timeattributes} @@ -274,13 +274,13 @@ CREATE WINDOW VIEW wv AS SELECT count(number), tumbleStart(w_id) as w_start from **Event time** is the time that each individual event occurred on its producing device. This time is typically embedded within the records when it is generated. Event time processing allows for consistent results even in case of out-of-order events or late events. Window view supports event time processing by using `WATERMARK` syntax. -Window view provides three watermark strategies. +Window view provides three watermark strategies: * `STRICTLY_ASCENDING`: Emits a watermark of the maximum observed timestamp so far. Rows that have a timestamp smaller to the max timestamp are not late. * `ASCENDING`: Emits a watermark of the maximum observed timestamp so far minus 1. Rows that have a timestamp equal and smaller to the max timestamp are not late. * `BOUNDED`: WATERMARK=INTERVAL. Emits watermarks, which are the maximum observed timestamp minus the specified delay. -The following queries are examples of creating a window view with `WATERMARK`. +The following queries are examples of creating a window view with `WATERMARK`: ``` sql CREATE WINDOW VIEW wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); diff --git a/docs/zh/sql-reference/functions/window-view-functions.md b/docs/zh/sql-reference/functions/window-view-functions.md new file mode 100644 index 00000000000..86425bc78e2 --- /dev/null +++ b/docs/zh/sql-reference/functions/window-view-functions.md @@ -0,0 +1,112 @@ +--- +toc_priority: 68 +toc_title: Window View +--- + +# Window View 函数{#window-view-han-shu} + +Window view函数用于获取窗口的起始(包含边界)和结束时间(不包含边界)。系统支持的window view函数如下: + +## tumble {#window-view-functions-tumble} + +tumble窗口是连续的、不重叠的固定大小(`interval`)时间窗口。 + +``` sql +tumble(time_attr, interval [, timezone]) +``` + +**参数** +- `time_attr` - [DateTime](../../sql-reference/data-types/datetime.md)类型的时间数据。 +- `interval` - [Interval](../../sql-reference/data-types/special-data-types/interval.md)类型的窗口大小。 +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) 类型的时区(可选参数). + +**返回值** + +- tumble窗口的开始(包含边界)和结束时间(不包含边界) + +类型: `Tuple(DateTime, DateTime)` + +**示例** + +查询: + +``` sql +SELECT tumble(now(), toIntervalDay('1')) +``` + +结果: + +``` text +┌─tumble(now(), toIntervalDay('1'))─────────────┐ +│ ['2020-01-01 00:00:00','2020-01-02 00:00:00'] │ +└───────────────────────────────────────────────┘ +``` + +## hop {#window-view-functions-hop} + +hop窗口是一个固定大小(`window_interval`)的时间窗口,并按照一个固定的滑动间隔(`hop_interval`)滑动。当滑动间隔小于窗口大小时,滑动窗口间存在重叠,此时一个数据可能存在于多个窗口。 + +``` sql +hop(time_attr, hop_interval, window_interval [, timezone]) +``` + +**参数** + +- `time_attr` - [DateTime](../../sql-reference/data-types/datetime.md)类型的时间数据。 +- `hop_interval` - Hop interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md)类型的滑动间隔,需要大于0。 +- `window_interval` - [Interval](../../sql-reference/data-types/special-data-types/interval.md)类型的窗口大小,需要大于0。 +- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) 类型的时区(可选参数)。 + +**返回值** + +- hop窗口的开始(包含边界)和结束时间(不包含边界)。由于一个数据可能存在于多个窗口,脱离window view单独调用该函数时只返回第一个窗口数据。 + +类型: `Tuple(DateTime, DateTime)` + +**示例** + +查询: + +``` sql +SELECT hop(now(), INTERVAL '1' SECOND, INTERVAL '2' SECOND) +``` + +结果: + +``` text +┌─hop(now(), toIntervalSecond('1'), toIntervalSecond('2'))──┐ +│ ('2020-01-14 16:58:22','2020-01-14 16:58:24') │ +└───────────────────────────────────────────────────────────┘ +``` + +## tumbleStart {#window-view-functions-tumblestart} + +返回tumble窗口的开始时间(包含边界)。 + +``` sql +tumbleStart(time_attr, interval [, timezone]); +``` + +## tumbleEnd {#window-view-functions-tumbleend} + +返回tumble窗口的结束时间(不包含边界)。 + +``` sql +tumbleEnd(time_attr, interval [, timezone]); +``` + +## hopStart {#window-view-functions-hopstart} + +返回hop窗口的开始时间(包含边界)。 + +``` sql +hopStart(time_attr, hop_interval, window_interval [, timezone]); +``` + +## hopEnd {#window-view-functions-hopend} + +返回hop窗口的结束时间(不包含边界)。 + +``` sql +hopEnd(time_attr, hop_interval, window_interval [, timezone]); +``` \ No newline at end of file diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index da69860f068..ed64b578150 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -5,7 +5,7 @@ toc_title: VIEW # CREATE VIEW {#create-view} -创建一个新视图。 有两种类型的视图:普通视图和物化视图。 +创建一个新视图。 有两种类型的视图:普通视图,物化视图,Live视图和Window视图。 ## Normal {#normal} @@ -241,3 +241,120 @@ Code: 60. DB::Exception: Received from localhost:9000. DB::Exception: Table defa - 使用定期刷新从系统表中查看指标。 [原始文章](https://clickhouse.com/docs/en/sql-reference/statements/create/view/) + +## Window View [Experimental] {#window-view} + +!!! important "重要" + 这是一项试验性功能,可能会在未来版本中以向后不兼容的方式进行更改。 + 通过[allow_experimental_window_view](../../../operations/settings/settings.md#allow-experimental-window-view)启用window view以及`WATCH`语句。输入命令 + `set allow_experimental_window_view = 1`。 + +``` sql +CREATE WINDOW VIEW [IF NOT EXISTS] [db.]table_name [TO [db.]table_name] [ENGINE = engine] [WATERMARK = strategy] [ALLOWED_LATENESS = interval_function] AS SELECT ... GROUP BY window_view_function +``` + +Window view可以通过时间窗口聚合数据,并在满足窗口触发条件时自动触发对应窗口计算。其通过将计算状态保存降低处理延迟,支持将处理结果输出至目标表或通过`WATCH`语句输出至终端。 + +创建window view的方式和创建物化视图类似。Window view使用默认为`AggregatingMergeTree`的内部存储引擎存储计算中间状态。 + +### Window View 函数{#window-view-han-shu} + +[Window view函数](../../functions/window-view-functions.md)用于获取窗口的起始和结束时间。Window view需要和window view函数配合使用。 + +### 时间属性{#window-view-shi-jian-shu-xing} + +Window view 支持**处理时间**和**事件时间**两种时间类型。 + +**处理时间**为默认时间类型,该模式下window view使用本地机器时间计算窗口数据。“处理时间”时间类型计算简单,但具有不确定性。该模式下时间可以为window view函数的第一个参数`time_attr`,或通过函数`now()`使用当前机器时间。下面的例子展示了使用“处理时间”创建的window view的例子。 + +``` sql +CREATE WINDOW VIEW wv AS SELECT count(number), tumbleStart(w_id) as w_start from date GROUP BY tumble(now(), INTERVAL '5' SECOND) as w_id +``` + +**事件时间** 是事件真实发生的时间,该时间往往在事件发生时便嵌入数据记录。事件时间处理提供较高的确定性,可以处理乱序数据以及迟到数据。Window view 通过水位线(`WATERMARK`)启用事件时间处理。 + +Window view提供如下三种水位线策略: + +* `STRICTLY_ASCENDING`: 提交观测到的最大时间作为水位线,小于最大观测时间的数据不算迟到。 +* `ASCENDING`: 提交观测到的最大时间减1作为水位线。小于或等于最大观测时间的数据不算迟到。 +* `BOUNDED`: WATERMARK=INTERVAL. 提交最大观测时间减去固定间隔(`INTERVAL`)做为水位线。 + +以下为使用`WATERMARK`创建window view的示例: + +``` sql +CREATE WINDOW VIEW wv WATERMARK=STRICTLY_ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); +CREATE WINDOW VIEW wv WATERMARK=ASCENDING AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); +CREATE WINDOW VIEW wv WATERMARK=INTERVAL '3' SECOND AS SELECT count(number) FROM date GROUP BY tumble(timestamp, INTERVAL '5' SECOND); +``` + +通常,窗口会在水位线到达时触发,水位线到达之后的数据会被丢弃。Window view可以通过设置`ALLOWED_LATENESS=INTERVAL`来开启迟到消息处理。示例如下: + +``` sql +CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTERVAL '2' SECOND AS SELECT count(a) AS count, tumbleEnd(wid) AS w_end FROM test.mt GROUP BY tumble(timestamp, INTERVAL '5' SECOND) AS wid; +``` + +需要注意的是,迟到消息需要更新之前的处理结果。与在窗口结束时触发不同,迟到消息到达时window view会立即触发计算。因此,会导致同一个窗口输出多次计算结果。用户需要注意这种情况,并消除重复结果。 + +### 新窗口监控{#window-view-xin-chuang-kou-jian-kong} + +Window view可以通过`WATCH`语句将处理结果推送至终端,或通过`TO`语句将结果推送至数据表。 + +``` sql +WATCH [db.]name [LIMIT n] +``` + +`WATCH`语句和`LIVE VIEW`中的类似。支持设置`LIMIT`参数,输出消息数目达到`LIMIT`限制时结束查询。 + +### 设置{#window-view-she-zhi} + +- `window_view_clean_interval`: window view清除过期数据间隔(单位为秒)。系统会定期清除过期数据,尚未触发的窗口数据不会被清除。 +- `window_view_heartbeat_interval`: 用于判断watch查询活跃的心跳时间间隔。 + +### 示例{#window-view-shi-li} + +假设我们需要每10秒统计一次`data`表中的点击日志,且`data`表的结构如下: + +``` sql +CREATE TABLE data ( `id` UInt64, `timestamp` DateTime) ENGINE = Memory; +``` + +首先,使用10秒大小的tumble函数创建window view。 + +``` sql +CREATE WINDOW VIEW wv as select count(id), tumbleStart(w_id) as window_start from data group by tumble(timestamp, INTERVAL '10' SECOND) as w_id +``` + +随后,我们使用`WATCH`语句获取计算结果。 + +``` sql +WATCH wv +``` + +当日志插入表`data`时, + +``` sql +INSERT INTO data VALUES(1,now()) +``` + +`WATCH`语句会输出如下结果: + +``` text +┌─count(id)─┬────────window_start─┐ +│ 1 │ 2020-01-14 16:56:40 │ +└───────────┴─────────────────────┘ +``` + +或者,我们可以通过`TO`关键字将处理结果输出至另一张表。 + +``` sql +CREATE WINDOW VIEW wv TO dst AS SELECT count(id), tumbleStart(w_id) as window_start FROM data GROUP BY tumble(timestamp, INTERVAL '10' SECOND) as w_id +``` + +ClickHouse测试中提供了更多的示例(以`*window_view*`命名)。 + +### Window View 使用场景{#window-view-shi-yong-chang-jing} + +Window view 在以下场景有用: + +* **监控**: 以时间维度聚合及处理数据,并将处理结果输出至目标表。用户可通过目标表获取并操作计算结果。 +* **分析**: 以时间维度进行数据分析. 当数据源非常庞大时,window view可以减少重复全表查询的计算量。 From 03927f5fc6b5991aa125fdf9de0ab5f899be3ea2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 8 Dec 2021 10:58:06 +0100 Subject: [PATCH 583/609] Revert "graphite: split tagged/plain rollup rules (for merges perfomance)" This reverts commit 303552f51507178fae443ddf822485031b6f7d9f. --- base/base/StringRef.h | 5 +- src/CMakeLists.txt | 1 - src/Common/tests/gtest_global_context.cpp | 7 - src/Common/tests/gtest_global_context.h | 6 +- src/Processors/Merges/Algorithms/Graphite.cpp | 493 --------------- src/Processors/Merges/Algorithms/Graphite.h | 37 +- .../GraphiteRollupSortedAlgorithm.cpp | 59 +- .../GraphiteRollupSortedAlgorithm.h | 10 + .../Algorithms/tests/gtest_graphite.cpp | 597 ------------------ .../MergeTree/registerStorageMergeTree.cpp | 175 +++++ src/Storages/System/StorageSystemGraphite.cpp | 4 - tests/integration/helpers/test_tools.py | 16 - .../test_graphite_merge_tree/test.py | 20 +- .../__init__.py | 0 .../configs/graphite_rollup.xml | 120 ---- .../configs/users.xml | 8 - .../test_graphite_merge_tree_typed/test.py | 580 ----------------- ...ultiple_paths_and_versions.reference.plain | 84 --- ...ltiple_paths_and_versions.reference.tagged | 84 --- .../02117_show_create_table_system.reference | 2 +- utils/CMakeLists.txt | 1 - utils/graphite-rollup/CMakeLists.txt | 23 - .../graphite-rollup/graphite-rollup-bench.cpp | 147 ----- utils/graphite-rollup/metrics.txt | 11 - utils/graphite-rollup/rollup-tag-list.xml | 167 ----- utils/graphite-rollup/rollup-typed.xml | 167 ----- utils/graphite-rollup/rollup.xml | 147 ----- 27 files changed, 266 insertions(+), 2705 deletions(-) delete mode 100644 src/Common/tests/gtest_global_context.cpp delete mode 100644 src/Processors/Merges/Algorithms/Graphite.cpp delete mode 100644 src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp delete mode 100644 tests/integration/test_graphite_merge_tree_typed/__init__.py delete mode 100644 tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml delete mode 100644 tests/integration/test_graphite_merge_tree_typed/configs/users.xml delete mode 100644 tests/integration/test_graphite_merge_tree_typed/test.py delete mode 100644 tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain delete mode 100644 tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged delete mode 100644 utils/graphite-rollup/CMakeLists.txt delete mode 100644 utils/graphite-rollup/graphite-rollup-bench.cpp delete mode 100644 utils/graphite-rollup/metrics.txt delete mode 100644 utils/graphite-rollup/rollup-tag-list.xml delete mode 100644 utils/graphite-rollup/rollup-typed.xml delete mode 100644 utils/graphite-rollup/rollup.xml diff --git a/base/base/StringRef.h b/base/base/StringRef.h index 98c322320a5..d0184dbc24c 100644 --- a/base/base/StringRef.h +++ b/base/base/StringRef.h @@ -48,10 +48,7 @@ struct StringRef std::string toString() const { return std::string(data, size); } explicit operator std::string() const { return toString(); } - - std::string_view toView() const { return std::string_view(data, size); } - - constexpr explicit operator std::string_view() const { return std::string_view(data, size); } + constexpr explicit operator std::string_view() const { return {data, size}; } }; /// Here constexpr doesn't implicate inline, see https://www.viva64.com/en/w/v1043/ diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 1f7a2700e5a..5f4ebaaa895 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -547,7 +547,6 @@ if (ENABLE_TESTS AND USE_GTEST) clickhouse_parsers clickhouse_storages_system dbms - clickhouse_common_config clickhouse_common_zookeeper string_utils) diff --git a/src/Common/tests/gtest_global_context.cpp b/src/Common/tests/gtest_global_context.cpp deleted file mode 100644 index 19ba3cdc269..00000000000 --- a/src/Common/tests/gtest_global_context.cpp +++ /dev/null @@ -1,7 +0,0 @@ -#include "gtest_global_context.h" - -const ContextHolder & getContext() -{ - static ContextHolder holder; - return holder; -} diff --git a/src/Common/tests/gtest_global_context.h b/src/Common/tests/gtest_global_context.h index 7756be7ce9b..9bd7c2490d6 100644 --- a/src/Common/tests/gtest_global_context.h +++ b/src/Common/tests/gtest_global_context.h @@ -18,4 +18,8 @@ struct ContextHolder ContextHolder(ContextHolder &&) = default; }; -const ContextHolder & getContext(); +inline const ContextHolder & getContext() +{ + static ContextHolder holder; + return holder; +} diff --git a/src/Processors/Merges/Algorithms/Graphite.cpp b/src/Processors/Merges/Algorithms/Graphite.cpp deleted file mode 100644 index 38d3fa30b42..00000000000 --- a/src/Processors/Merges/Algorithms/Graphite.cpp +++ /dev/null @@ -1,493 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -#include - -#include - -using namespace std::literals; - -namespace DB::ErrorCodes -{ - extern const int NOT_IMPLEMENTED; - extern const int BAD_ARGUMENTS; - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int NO_ELEMENTS_IN_CONFIG; - } - -namespace DB::Graphite -{ -static std::unordered_map ruleTypeMap = -{ - { RuleTypeAll, "all" }, - { RuleTypePlain, "plain" }, - { RuleTypeTagged, "tagged"}, - { RuleTypeTagList, "tag_list"} -}; - -const String & ruleTypeStr(RuleType rule_type) -{ - try - { - return ruleTypeMap.at(rule_type); - } - catch (...) - { - throw Exception("invalid rule type: " + std::to_string(rule_type), DB::ErrorCodes::BAD_ARGUMENTS); - } -} - -RuleType ruleType(const String & s) -{ - if (s == "all") - return RuleTypeAll; - else if (s == "plain") - return RuleTypePlain; - else if (s == "tagged") - return RuleTypeTagged; - else if (s == "tag_list") - return RuleTypeTagList; - else - throw Exception("invalid rule type: " + s, DB::ErrorCodes::BAD_ARGUMENTS); -} - -static const Graphite::Pattern undef_pattern = -{ /// empty pattern for selectPatternForPath - .rule_type = RuleTypeAll, - .regexp = nullptr, - .regexp_str = "", - .function = nullptr, - .retentions = Graphite::Retentions(), - .type = undef_pattern.TypeUndef, -}; - -inline static const Patterns & selectPatternsForMetricType(const Graphite::Params & params, const StringRef path) -{ - if (params.patterns_typed) - { - std::string_view path_view = path.toView(); - if (path_view.find("?"sv) == path_view.npos) - return params.patterns_plain; - else - return params.patterns_tagged; - } - else - { - return params.patterns; - } -} - -Graphite::RollupRule selectPatternForPath( - const Graphite::Params & params, - const StringRef path) -{ - const Graphite::Pattern * first_match = &undef_pattern; - - const Patterns & patterns_check = selectPatternsForMetricType(params, path); - - for (const auto & pattern : patterns_check) - { - if (!pattern.regexp) - { - /// Default pattern - if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll) - { - /// There is only default pattern for both retention and aggregation - return std::pair(&pattern, &pattern); - } - if (pattern.type != first_match->type) - { - if (first_match->type == first_match->TypeRetention) - { - return std::pair(first_match, &pattern); - } - if (first_match->type == first_match->TypeAggregation) - { - return std::pair(&pattern, first_match); - } - } - } - else - { - if (pattern.regexp->match(path.data, path.size)) - { - /// General pattern with matched path - if (pattern.type == pattern.TypeAll) - { - /// Only for not default patterns with both function and retention parameters - return std::pair(&pattern, &pattern); - } - if (first_match->type == first_match->TypeUndef) - { - first_match = &pattern; - continue; - } - if (pattern.type != first_match->type) - { - if (first_match->type == first_match->TypeRetention) - { - return std::pair(first_match, &pattern); - } - if (first_match->type == first_match->TypeAggregation) - { - return std::pair(&pattern, first_match); - } - } - } - } - } - - return {nullptr, nullptr}; -} - -/** Is used to order Graphite::Retentions by age and precision descending. - * Throws exception if not both age and precision are less or greater then another. - */ -static bool compareRetentions(const Retention & a, const Retention & b) -{ - if (a.age > b.age && a.precision > b.precision) - { - return true; - } - else if (a.age < b.age && a.precision < b.precision) - { - return false; - } - String error_msg = "age and precision should only grow up: " - + std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs " - + std::to_string(b.age) + ":" + std::to_string(b.precision); - throw Exception( - error_msg, - DB::ErrorCodes::BAD_ARGUMENTS); -} - -bool operator==(const Retention & a, const Retention & b) -{ - return a.age == b.age && a.precision == b.precision; -} - -std::ostream & operator<<(std::ostream & stream, const Retentions & a) -{ - stream << "{ "; - for (size_t i = 0; i < a.size(); i++) - { - if (i > 0) - stream << ","; - stream << " { age = " << a[i].age << ", precision = " << a[i].precision << " }"; - } - stream << " }"; - - return stream; -} - -bool operator==(const Pattern & a, const Pattern & b) -{ - // equal - // Retentions retentions; /// Must be ordered by 'age' descending. - if (a.type != b.type || a.regexp_str != b.regexp_str || a.rule_type != b.rule_type) - return false; - - if (a.function == nullptr) - { - if (b.function != nullptr) - return false; - } - else if (b.function == nullptr) - { - return false; - } - else if (a.function->getName() != b.function->getName()) - { - return false; - } - - return a.retentions == b.retentions; -} - -std::ostream & operator<<(std::ostream & stream, const Pattern & a) -{ - stream << "{ rule_type = " << ruleTypeStr(a.rule_type); - if (!a.regexp_str.empty()) - stream << ", regexp = '" << a.regexp_str << "'"; - if (a.function != nullptr) - stream << ", function = " << a.function->getName(); - if (!a.retentions.empty()) - { - stream << ",\n retentions = {\n"; - for (size_t i = 0; i < a.retentions.size(); i++) - { - stream << " { " << a.retentions[i].age << ", " << a.retentions[i].precision << " }"; - if (i < a.retentions.size() - 1) - stream << ","; - stream << "\n"; - } - stream << " }\n"; - } - else - stream << " "; - - stream << "}"; - return stream; -} - -std::string buildTaggedRegex(std::string regexp_str) -{ - /* - * tags list in format (for name or any value can use regexp, alphabet sorting not needed) - * spaces are not stiped and used as tag and value part - * name must be first (if used) - * - * tag1=value1; tag2=VALUE2_REGEX;tag3=value3 - * or - * name;tag1=value1;tag2=VALUE2_REGEX;tag3=value3 - * or for one tag - * tag1=value1 - * - * Resulting regex against metric like - * name?tag1=value1&tag2=value2 - * - * So, - * - * name - * produce - * name\? - * - * tag2=val2 - * produce - * [\?&]tag2=val2(&.*)?$ - * - * nam.* ; tag1=val1 ; tag2=val2 - * produce - * nam.*\?(.*&)?tag1=val1&(.*&)?tag2=val2(&.*)?$ - */ - - std::vector tags; - - splitInto<';'>(tags, regexp_str); - /* remove empthy elements */ - using namespace std::string_literals; - tags.erase(std::remove(tags.begin(), tags.end(), ""s), tags.end()); - if (tags[0].find('=') == tags[0].npos) - { - if (tags.size() == 1) /* only name */ - return "^" + tags[0] + "\\?"; - /* start with name value */ - regexp_str = "^" + tags[0] + "\\?(.*&)?"; - tags.erase(std::begin(tags)); - } - else - regexp_str = "[\\?&]"; - - std::sort(std::begin(tags), std::end(tags)); /* sorted tag keys */ - regexp_str += fmt::format( - "{}{}", - fmt::join(tags, "&(.*&)?"), - "(&.*)?$" /* close regex */ - ); - - return regexp_str; -} - -/** Read the settings for Graphite rollup from config. - * Example - * - * - * Path - * - * click_cost - * any - * - * 0 - * 3600 - * - * - * 86400 - * 60 - * - * - * - * max - * - * 0 - * 60 - * - * - * 3600 - * 300 - * - * - * 86400 - * 3600 - * - * - * - */ -static const Pattern & -appendGraphitePattern( - const Poco::Util::AbstractConfiguration & config, - const String & config_element, Patterns & patterns, - bool default_rule, - ContextPtr context) -{ - Pattern pattern; - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_element, keys); - - for (const auto & key : keys) - { - if (key == "regexp") - { - pattern.regexp_str = config.getString(config_element + ".regexp"); - } - else if (key == "function") - { - String aggregate_function_name_with_params = config.getString(config_element + ".function"); - String aggregate_function_name; - Array params_row; - getAggregateFunctionNameAndParametersArray( - aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context); - - /// TODO Not only Float64 - AggregateFunctionProperties properties; - pattern.function = AggregateFunctionFactory::instance().get( - aggregate_function_name, {std::make_shared()}, params_row, properties); - } - else if (key == "rule_type") - { - String rule_type = config.getString(config_element + ".rule_type"); - pattern.rule_type = ruleType(rule_type); - } - else if (startsWith(key, "retention")) - { - pattern.retentions.emplace_back(Graphite::Retention{ - .age = config.getUInt(config_element + "." + key + ".age"), - .precision = config.getUInt(config_element + "." + key + ".precision")}); - } - else - throw Exception("Unknown element in config: " + key, DB::ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - } - - if (!pattern.regexp_str.empty()) - { - if (pattern.rule_type == RuleTypeTagList) - { - // construct tagged regexp - pattern.regexp_str = buildTaggedRegex(pattern.regexp_str); - pattern.rule_type = RuleTypeTagged; - } - pattern.regexp = std::make_shared(pattern.regexp_str); - } - - if (!pattern.function && pattern.retentions.empty()) - throw Exception( - "At least one of an aggregate function or retention rules is mandatory for rollup patterns in GraphiteMergeTree", - DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - if (default_rule && pattern.rule_type != RuleTypeAll) - { - throw Exception( - "Default must have rule_type all for rollup patterns in GraphiteMergeTree", - DB::ErrorCodes::BAD_ARGUMENTS); - } - - if (!pattern.function) - { - pattern.type = pattern.TypeRetention; - } - else if (pattern.retentions.empty()) - { - pattern.type = pattern.TypeAggregation; - } - else - { - pattern.type = pattern.TypeAll; - } - - if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll - if (pattern.function->allocatesMemoryInArena()) - throw Exception( - "Aggregate function " + pattern.function->getName() + " isn't supported in GraphiteMergeTree", DB::ErrorCodes::NOT_IMPLEMENTED); - - /// retention should be in descending order of age. - if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll - std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions); - - patterns.emplace_back(pattern); - return patterns.back(); -} - -void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params) -{ - const auto & config = context->getConfigRef(); - - if (!config.has(config_element)) - throw Exception("No '" + config_element + "' element in configuration file", ErrorCodes::NO_ELEMENTS_IN_CONFIG); - - params.config_name = config_element; - params.path_column_name = config.getString(config_element + ".path_column_name", "Path"); - params.time_column_name = config.getString(config_element + ".time_column_name", "Time"); - params.value_column_name = config.getString(config_element + ".value_column_name", "Value"); - params.version_column_name = config.getString(config_element + ".version_column_name", "Timestamp"); - - params.patterns_typed = false; - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_element, keys); - - for (const auto & key : keys) - { - if (startsWith(key, "pattern")) - { - if (appendGraphitePattern(config, config_element + "." + key, params.patterns, false, context).rule_type != RuleTypeAll) - params.patterns_typed = true; - } - else if (key == "default") - { - /// See below. - } - else if (key == "path_column_name" || key == "time_column_name" || key == "value_column_name" || key == "version_column_name") - { - /// See above. - } - else - throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - } - - if (config.has(config_element + ".default")) - appendGraphitePattern(config, config_element + "." + ".default", params.patterns, true, context); - - for (const auto & pattern : params.patterns) - { - if (pattern.rule_type == RuleTypeAll) - { - if (params.patterns_typed) - { - params.patterns_plain.push_back(pattern); - params.patterns_tagged.push_back(pattern); - } - } - else if (pattern.rule_type == RuleTypePlain) - { - params.patterns_plain.push_back(pattern); - } - else if (pattern.rule_type == RuleTypeTagged) - { - params.patterns_tagged.push_back(pattern); - } - else - { - throw Exception("Unhandled rule_type in config: " + ruleTypeStr(pattern.rule_type), ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); - } - } -} - -} diff --git a/src/Processors/Merges/Algorithms/Graphite.h b/src/Processors/Merges/Algorithms/Graphite.h index dc39cb46386..ecb1aeb9804 100644 --- a/src/Processors/Merges/Algorithms/Graphite.h +++ b/src/Processors/Merges/Algorithms/Graphite.h @@ -1,8 +1,13 @@ #pragma once - -#include #include -#include + +namespace DB +{ + +class IAggregateFunction; +using AggregateFunctionPtr = std::shared_ptr; + +} /** Intended for implementation of "rollup" - aggregation (rounding) of older data * for a table with Graphite data (Graphite is the system for time series monitoring). @@ -92,32 +97,16 @@ namespace DB::Graphite { -// sync with rule_types_str -enum RuleType -{ - RuleTypeAll = 0, // default, with regex, compatible with old scheme - RuleTypePlain = 1, // plain metrics, with regex, compatible with old scheme - RuleTypeTagged = 2, // tagged metrics, with regex, compatible with old scheme - RuleTypeTagList = 3 // tagged metrics, with regex (converted to RuleTypeTagged from string like 'retention=10min ; env=(staging|prod)') -}; - -const String & ruleTypeStr(RuleType rule_type); - struct Retention { UInt32 age; UInt32 precision; }; -bool operator==(const Retention & a, const Retention & b); - using Retentions = std::vector; -std::ostream &operator<<(std::ostream & stream, const Retentions & a); - struct Pattern { - RuleType rule_type = RuleTypeAll; std::shared_ptr regexp; std::string regexp_str; AggregateFunctionPtr function; @@ -125,9 +114,6 @@ struct Pattern enum { TypeUndef, TypeRetention, TypeAggregation, TypeAll } type = TypeAll; /// The type of defined pattern, filled automatically }; -bool operator==(const Pattern & a, const Pattern & b); -std::ostream &operator<<(std::ostream & stream, const Pattern & a); - using Patterns = std::vector; using RetentionPattern = Pattern; using AggregationPattern = Pattern; @@ -139,16 +125,9 @@ struct Params String time_column_name; String value_column_name; String version_column_name; - bool patterns_typed; Graphite::Patterns patterns; - Graphite::Patterns patterns_plain; - Graphite::Patterns patterns_tagged; }; using RollupRule = std::pair; -Graphite::RollupRule selectPatternForPath(const Graphite::Params & params, const StringRef path); - -void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params); - } diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp index c4f60571dd9..328c34823a0 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.cpp @@ -1,4 +1,3 @@ -#include #include #include #include @@ -53,6 +52,62 @@ GraphiteRollupSortedAlgorithm::GraphiteRollupSortedAlgorithm( columns_definition = defineColumns(header, params); } +Graphite::RollupRule GraphiteRollupSortedAlgorithm::selectPatternForPath(StringRef path) const +{ + const Graphite::Pattern * first_match = &undef_pattern; + + for (const auto & pattern : params.patterns) + { + if (!pattern.regexp) + { + /// Default pattern + if (first_match->type == first_match->TypeUndef && pattern.type == pattern.TypeAll) + { + /// There is only default pattern for both retention and aggregation + return std::pair(&pattern, &pattern); + } + if (pattern.type != first_match->type) + { + if (first_match->type == first_match->TypeRetention) + { + return std::pair(first_match, &pattern); + } + if (first_match->type == first_match->TypeAggregation) + { + return std::pair(&pattern, first_match); + } + } + } + else if (pattern.regexp->match(path.data, path.size)) + { + /// General pattern with matched path + if (pattern.type == pattern.TypeAll) + { + /// Only for not default patterns with both function and retention parameters + return std::pair(&pattern, &pattern); + } + if (first_match->type == first_match->TypeUndef) + { + first_match = &pattern; + continue; + } + if (pattern.type != first_match->type) + { + if (first_match->type == first_match->TypeRetention) + { + return std::pair(first_match, &pattern); + } + if (first_match->type == first_match->TypeAggregation) + { + return std::pair(&pattern, first_match); + } + } + } + } + + return {nullptr, nullptr}; +} + UInt32 GraphiteRollupSortedAlgorithm::selectPrecision(const Graphite::Retentions & retentions, time_t time) const { static_assert(is_signed_v, "time_t must be signed type"); @@ -133,7 +188,7 @@ IMergingAlgorithm::Status GraphiteRollupSortedAlgorithm::merge() Graphite::RollupRule next_rule = merged_data.currentRule(); if (new_path) - next_rule = selectPatternForPath(this->params, next_path); + next_rule = selectPatternForPath(next_path); const Graphite::RetentionPattern * retention_pattern = std::get<0>(next_rule); time_t next_time_rounded; diff --git a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h index 4968cbfc470..0155b73b238 100644 --- a/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h +++ b/src/Processors/Merges/Algorithms/GraphiteRollupSortedAlgorithm.h @@ -102,6 +102,16 @@ private: time_t current_time = 0; time_t current_time_rounded = 0; + const Graphite::Pattern undef_pattern = + { /// temporary empty pattern for selectPatternForPath + .regexp = nullptr, + .regexp_str = "", + .function = nullptr, + .retentions = DB::Graphite::Retentions(), + .type = undef_pattern.TypeUndef, + }; + + Graphite::RollupRule selectPatternForPath(StringRef path) const; UInt32 selectPrecision(const Graphite::Retentions & retentions, time_t time) const; /// Insert the values into the resulting columns, which will not be changed in the future. diff --git a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp b/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp deleted file mode 100644 index 1d739bf566a..00000000000 --- a/src/Processors/Merges/Algorithms/tests/gtest_graphite.cpp +++ /dev/null @@ -1,597 +0,0 @@ -#include -#include -#include -#include - -#include - -#include -#include - -#include -#include -#include -#include - -using namespace DB; - -static int regAggregateFunctions = 0; - -void tryRegisterAggregateFunctions() -{ - if (!regAggregateFunctions) - { - registerAggregateFunctions(); - regAggregateFunctions = 1; - } -} - -static ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path) -{ - ConfigProcessor config_processor(config_path, true, true); - ConfigProcessor::LoadedConfig config = config_processor.loadConfig(false); - return config; -} - -static ConfigProcessor::LoadedConfig loadConfigurationFromString(std::string & s) -{ - char tmp_file[19]; - strcpy(tmp_file, "/tmp/rollup-XXXXXX"); - int fd = mkstemp(tmp_file); - if (fd == -1) - { - throw std::runtime_error(strerror(errno)); - } - try { - if (write(fd, s.c_str(), s.size()) < s.size()) - { - throw std::runtime_error("unable write to temp file"); - } - if (write(fd, "\n", 1) != 1) - { - throw std::runtime_error("unable write to temp file"); - } - close(fd); - auto config_path = std::string(tmp_file) + ".xml"; - if (std::rename(tmp_file, config_path.c_str())) - { - int err = errno; - remove(tmp_file); - throw std::runtime_error(strerror(err)); - } - ConfigProcessor::LoadedConfig config = loadConfiguration(config_path); - remove(tmp_file); - return config; - } - catch (...) - { - remove(tmp_file); - throw; - } -} - -static Graphite::Params setGraphitePatterns(ContextMutablePtr context, ConfigProcessor::LoadedConfig & config) -{ - context->setConfig(config.configuration); - - Graphite::Params params; - setGraphitePatternsFromConfig(context, "graphite_rollup", params); - - return params; -} - -struct PatternForCheck -{ - Graphite::RuleType rule_type; - std::string regexp_str; - String function; - Graphite::Retentions retentions; -}; - - -bool checkRule(const Graphite::Pattern & pattern, const struct PatternForCheck & pattern_check, - const std::string & typ, const std::string & path, std::string & message) -{ - bool rule_type_eq = (pattern.rule_type == pattern_check.rule_type); - bool regexp_eq = (pattern.regexp_str == pattern_check.regexp_str); - bool function_eq = (pattern.function == nullptr && pattern_check.function.empty()) - || (pattern.function != nullptr && pattern.function->getName() == pattern_check.function); - bool retentions_eq = (pattern.retentions == pattern_check.retentions); - - if (rule_type_eq && regexp_eq && function_eq && retentions_eq) - return true; - - message = typ + " rollup rule mismatch for '" + path + "'," + - (rule_type_eq ? "" : "rule_type ") + - (regexp_eq ? "" : "regexp ") + - (function_eq ? "" : "function ") + - (retentions_eq ? "" : "retentions "); - return false; -} - -std::ostream & operator<<(std::ostream & stream, const PatternForCheck & a) -{ - stream << "{ rule_type = " << ruleTypeStr(a.rule_type); - if (!a.regexp_str.empty()) - stream << ", regexp = '" << a.regexp_str << "'"; - if (!a.function.empty()) - stream << ", function = " << a.function; - if (!a.retentions.empty()) - { - stream << ",\n retentions = {\n"; - for (size_t i = 0; i < a.retentions.size(); i++) - { - stream << " { " << a.retentions[i].age << ", " << a.retentions[i].precision << " }"; - if (i < a.retentions.size() - 1) - stream << ","; - stream << "\n"; - } - stream << " }\n"; - } - else - stream << " "; - - stream << "}"; - return stream; -} - -struct PatternsForPath -{ - std::string path; - PatternForCheck retention_want; - PatternForCheck aggregation_want; -}; - -TEST(GraphiteTest, testSelectPattern) -{ - tryRegisterAggregateFunctions(); - - using namespace std::literals; - - std::string - xml(R"END( - - - \.sum$ - sum - - - ^((.*)|.)sum\? - sum - - - \.max$ - max - - - ^((.*)|.)max\? - max - - - \.min$ - min - - - ^((.*)|.)min\? - min - - - \.(count|sum|sum_sq)$ - sum - - - ^((.*)|.)(count|sum|sum_sq)\? - sum - - - ^retention\. - - 0 - 60 - - - 86400 - 3600 - - - - avg - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - -)END"); - - // Retentions must be ordered by 'age' descending. - std::vector tests - { - { - "test.sum", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(\.sum$)END", "sum", { } } - }, - { - "val.sum?env=test&tag=Fake3", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(^((.*)|.)sum\?)END", "sum", { } } - }, - { - "test.max", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(\.max$)END", "max", { } }, - }, - { - "val.max?env=test&tag=Fake4", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(^((.*)|.)max\?)END", "max", { } }, - }, - { - "test.min", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(\.min$)END", "min", { } }, - }, - { - "val.min?env=test&tag=Fake5", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(^((.*)|.)min\?)END", "min", { } }, - }, - { - "retention.count", - { Graphite::RuleTypeAll, R"END(^retention\.)END", "", { { 86400, 3600 }, { 0, 60 } } }, // ^retention - { Graphite::RuleTypeAll, R"END(\.(count|sum|sum_sq)$)END", "sum", { } }, - }, - { - "val.retention.count?env=test&tag=Fake5", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "val.count?env=test&tag=Fake5", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "test.p95", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - }, - { - "val.p95?env=test&tag=FakeNo", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - }, - { - "default", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - }, - { - "val.default?env=test&tag=FakeNo", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - } - }; - - auto config = loadConfigurationFromString(xml); - ContextMutablePtr context = getContext().context; - Graphite::Params params = setGraphitePatterns(context, config); - - for (const auto & t : tests) - { - auto rule = DB::Graphite::selectPatternForPath(params, t.path); - std:: string message; - if (!checkRule(*rule.first, t.retention_want, "retention", t.path, message)) - ADD_FAILURE() << message << ", got\n" << *rule.first << "\n, want\n" << t.retention_want << "\n"; - if (!checkRule(*rule.second, t.aggregation_want, "aggregation", t.path, message)) - ADD_FAILURE() << message << ", got\n" << *rule.second << "\n, want\n" << t.aggregation_want << "\n"; - } -} - - -namespace DB::Graphite -{ - std::string buildTaggedRegex(std::string regexp_str); -} - -struct RegexCheck -{ - std::string regex; - std::string regex_want; - std::string match; - std::string nomatch; -}; - -TEST(GraphiteTest, testBuildTaggedRegex) -{ - std::vector tests - { - { - "cpu\\.loadavg;project=DB.*;env=st.*", - R"END(^cpu\.loadavg\?(.*&)?env=st.*&(.*&)?project=DB.*(&.*)?$)END", - R"END(cpu.loadavg?env=staging&project=DBAAS)END", - R"END(cpu.loadavg?env=staging&project=D)END" - }, - { - R"END(project=DB.*;env=staging;)END", - R"END([\?&]env=staging&(.*&)?project=DB.*(&.*)?$)END", - R"END(cpu.loadavg?env=staging&project=DBPG)END", - R"END(cpu.loadavg?env=stagingN&project=DBAAS)END" - }, - { - "env=staging;", - R"END([\?&]env=staging(&.*)?$)END", - R"END(cpu.loadavg?env=staging&project=DPG)END", - R"END(cpu.loadavg?env=stagingN)END" - }, - { - " env = staging ;", // spaces are allowed, - R"END([\?&] env = staging (&.*)?$)END", - R"END(cpu.loadavg? env = staging &project=DPG)END", - R"END(cpu.loadavg?env=stagingN)END" - }, - { - "name;", - R"END(^name\?)END", - R"END(name?env=staging&project=DPG)END", - R"END(nameN?env=stagingN)END", - }, - { - "name", - R"END(^name\?)END", - R"END(name?env=staging&project=DPG)END", - R"END(nameN?env=stagingN)END", - } - }; - for (const auto & t : tests) - { - auto s = DB::Graphite::buildTaggedRegex(t.regex); - EXPECT_EQ(t.regex_want, s) << "result for '" << t.regex_want << "' mismatch"; - auto regexp = OptimizedRegularExpression(s); - EXPECT_TRUE(regexp.match(t.match.data(), t.match.size())) << t.match << " match for '" << s << "' failed"; - EXPECT_FALSE(regexp.match(t.nomatch.data(), t.nomatch.size())) << t.nomatch << " ! match for '" << s << "' failed"; - } -} - -TEST(GraphiteTest, testSelectPatternTyped) -{ - tryRegisterAggregateFunctions(); - - using namespace std::literals; - - std::string - xml(R"END( - - - plain - \.sum$ - sum - - - tagged - ^((.*)|.)sum\? - sum - - - plain - \.max$ - max - - - tagged - ^((.*)|.)max\? - max - - - plain - \.min$ - min - - - tagged - ^((.*)|.)min\? - min - - - plain - \.(count|sum|sum_sq)$ - sum - - - tagged - ^((.*)|.)(count|sum|sum_sq)\? - sum - - - plain - ^retention\. - - 0 - 60 - - - 86400 - 3600 - - - - tagged - - - 0 - 60 - - - 86400 - 3600 - - - - tag_list - retention=10min;env=staging - - 0 - 600 - - - 86400 - 3600 - - - - tag_list - retention=10min;env=[A-Za-z-]+rod[A-Za-z-]+ - - 0 - 600 - - - 86400 - 3600 - - - - tag_list - cpu\.loadavg - - 0 - 600 - - - 86400 - 3600 - - - - avg - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - -)END"); - - // Retentions must be ordered by 'age' descending. - std::vector tests - { - { - "test.sum", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypePlain, R"END(\.sum$)END", "sum", { } } - }, - { - "val.sum?env=test&tag=Fake3", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeTagged, R"END(^((.*)|.)sum\?)END", "sum", { } } - }, - { - "test.max", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypePlain, R"END(\.max$)END", "max", { } }, - }, - { - "val.max?env=test&tag=Fake4", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeTagged, R"END(^((.*)|.)max\?)END", "max", { } }, - }, - { - "test.min", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypePlain, R"END(\.min$)END", "min", { } }, - }, - { - "val.min?env=test&tag=Fake5", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeTagged, R"END(^((.*)|.)min\?)END", "min", { } }, - }, - { - "retention.count", - { Graphite::RuleTypePlain, R"END(^retention\.)END", "", { { 86400, 3600 }, { 0, 60 } } }, // ^retention - { Graphite::RuleTypePlain, R"END(\.(count|sum|sum_sq)$)END", "sum", { } }, - }, - { - "val.count?env=test&retention=hour&tag=Fake5", - { Graphite::RuleTypeTagged, R"END([\?&]retention=hour(&.*)?$)END", "", { { 86400, 3600 }, { 0, 60 } } }, // tagged retention=hour - { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "val.count?env=test&retention=hour", - { Graphite::RuleTypeTagged, R"END([\?&]retention=hour(&.*)?$)END", "", { { 86400, 3600 }, { 0, 60 } } }, // tagged retention=hour - { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "val.count?env=staging&retention=10min", - { Graphite::RuleTypeTagged, R"END([\?&]env=staging&(.*&)?retention=10min(&.*)?$)END", "", { { 86400, 3600 }, { 0, 600 } } }, // retention=10min ; env=staging - { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "val.count?env=production&retention=10min", - { Graphite::RuleTypeTagged, R"END([\?&]env=[A-Za-z-]+rod[A-Za-z-]+&(.*&)?retention=10min(&.*)?$)END", "", { { 86400, 3600 }, { 0, 600 } } }, // retention=10min ; env=[A-Za-z-]+rod[A-Za-z-]+ - { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "val.count?env=test&tag=Fake5", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeTagged, R"END(^((.*)|.)(count|sum|sum_sq)\?)END", "sum", { } }, - }, - { - "cpu.loadavg?env=test&tag=FakeNo", - { Graphite::RuleTypeTagged, R"END(^cpu\.loadavg\?)END", "", { { 86400, 3600 }, { 0, 600 } } }, // name=cpu\.loadavg - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, - }, - { - "test.p95", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - }, - { - "val.p95?env=test&tag=FakeNo", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - }, - { - "default", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - }, - { - "val.default?env=test&tag=FakeNo", - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - { Graphite::RuleTypeAll, "", "avg", { { 86400, 3600 }, { 3600, 300 }, { 0, 60 } } }, //default - } - }; - - auto config = loadConfigurationFromString(xml); - ContextMutablePtr context = getContext().context; - Graphite::Params params = setGraphitePatterns(context, config); - - for (const auto & t : tests) - { - auto rule = DB::Graphite::selectPatternForPath(params, t.path); - std:: string message; - if (!checkRule(*rule.first, t.retention_want, "retention", t.path, message)) - ADD_FAILURE() << message << ", got\n" << *rule.first << "\n, want\n" << t.retention_want << "\n"; - if (!checkRule(*rule.second, t.aggregation_want, "aggregation", t.path, message)) - ADD_FAILURE() << message << ", got\n" << *rule.second << "\n, want\n" << t.aggregation_want << "\n"; - } -} diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index ac6f4d8b7a4..cb52c8b86c0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -22,13 +22,17 @@ #include #include +#include namespace DB { namespace ErrorCodes { + extern const int NOT_IMPLEMENTED; extern const int BAD_ARGUMENTS; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int NO_ELEMENTS_IN_CONFIG; extern const int UNKNOWN_STORAGE; extern const int NO_REPLICA_NAME_GIVEN; } @@ -58,6 +62,171 @@ static Names extractColumnNames(const ASTPtr & node) } } +/** Is used to order Graphite::Retentions by age and precision descending. + * Throws exception if not both age and precision are less or greater then another. + */ +static bool compareRetentions(const Graphite::Retention & a, const Graphite::Retention & b) +{ + if (a.age > b.age && a.precision > b.precision) + { + return true; + } + else if (a.age < b.age && a.precision < b.precision) + { + return false; + } + String error_msg = "age and precision should only grow up: " + + std::to_string(a.age) + ":" + std::to_string(a.precision) + " vs " + + std::to_string(b.age) + ":" + std::to_string(b.precision); + throw Exception( + error_msg, + ErrorCodes::BAD_ARGUMENTS); +} + +/** Read the settings for Graphite rollup from config. + * Example + * + * + * Path + * + * click_cost + * any + * + * 0 + * 3600 + * + * + * 86400 + * 60 + * + * + * + * max + * + * 0 + * 60 + * + * + * 3600 + * 300 + * + * + * 86400 + * 3600 + * + * + * + */ +static void appendGraphitePattern( + const Poco::Util::AbstractConfiguration & config, + const String & config_element, + Graphite::Patterns & out_patterns, + ContextPtr context) +{ + Graphite::Pattern pattern; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_element, keys); + + for (const auto & key : keys) + { + if (key == "regexp") + { + pattern.regexp_str = config.getString(config_element + ".regexp"); + pattern.regexp = std::make_shared(pattern.regexp_str); + } + else if (key == "function") + { + String aggregate_function_name_with_params = config.getString(config_element + ".function"); + String aggregate_function_name; + Array params_row; + getAggregateFunctionNameAndParametersArray( + aggregate_function_name_with_params, aggregate_function_name, params_row, "GraphiteMergeTree storage initialization", context); + + /// TODO Not only Float64 + AggregateFunctionProperties properties; + pattern.function = AggregateFunctionFactory::instance().get( + aggregate_function_name, {std::make_shared()}, params_row, properties); + } + else if (startsWith(key, "retention")) + { + pattern.retentions.emplace_back(Graphite::Retention{ + .age = config.getUInt(config_element + "." + key + ".age"), + .precision = config.getUInt(config_element + "." + key + ".precision")}); + } + else + throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + if (!pattern.function && pattern.retentions.empty()) + throw Exception( + "At least one of an aggregate function or retention rules is mandatory for rollup patterns in GraphiteMergeTree", + ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + if (!pattern.function) + { + pattern.type = pattern.TypeRetention; + } + else if (pattern.retentions.empty()) + { + pattern.type = pattern.TypeAggregation; + } + else + { + pattern.type = pattern.TypeAll; + } + + if (pattern.type & pattern.TypeAggregation) /// TypeAggregation or TypeAll + if (pattern.function->allocatesMemoryInArena()) + throw Exception( + "Aggregate function " + pattern.function->getName() + " isn't supported in GraphiteMergeTree", ErrorCodes::NOT_IMPLEMENTED); + + /// retention should be in descending order of age. + if (pattern.type & pattern.TypeRetention) /// TypeRetention or TypeAll + std::sort(pattern.retentions.begin(), pattern.retentions.end(), compareRetentions); + + out_patterns.emplace_back(pattern); +} + +static void setGraphitePatternsFromConfig(ContextPtr context, const String & config_element, Graphite::Params & params) +{ + const auto & config = context->getConfigRef(); + + if (!config.has(config_element)) + throw Exception("No '" + config_element + "' element in configuration file", ErrorCodes::NO_ELEMENTS_IN_CONFIG); + + params.config_name = config_element; + params.path_column_name = config.getString(config_element + ".path_column_name", "Path"); + params.time_column_name = config.getString(config_element + ".time_column_name", "Time"); + params.value_column_name = config.getString(config_element + ".value_column_name", "Value"); + params.version_column_name = config.getString(config_element + ".version_column_name", "Timestamp"); + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_element, keys); + + for (const auto & key : keys) + { + if (startsWith(key, "pattern")) + { + appendGraphitePattern(config, config_element + "." + key, params.patterns, context); + } + else if (key == "default") + { + /// See below. + } + else if (key == "path_column_name" || key == "time_column_name" || key == "value_column_name" || key == "version_column_name") + { + /// See above. + } + else + throw Exception("Unknown element in config: " + key, ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG); + } + + if (config.has(config_element + ".default")) + appendGraphitePattern(config, config_element + "." + ".default", params.patterns, context); +} + + static String getMergeTreeVerboseHelp(bool) { using namespace std::string_literals; @@ -373,6 +542,12 @@ static StoragePtr create(const StorageFactory::Arguments & args) /// to make possible copying metadata files between replicas. Macros::MacroExpansionInfo info; info.table_id = args.table_id; + if (is_replicated_database) + { + auto database = DatabaseCatalog::instance().getDatabase(args.table_id.database_name); + info.shard = getReplicatedDatabaseShardName(database); + info.replica = getReplicatedDatabaseReplicaName(database); + } if (!allow_uuid_macro) info.table_id.uuid = UUIDHelpers::Nil; zookeeper_path = args.getContext()->getMacros()->expand(zookeeper_path, info); diff --git a/src/Storages/System/StorageSystemGraphite.cpp b/src/Storages/System/StorageSystemGraphite.cpp index 8711162385f..dd592600d18 100644 --- a/src/Storages/System/StorageSystemGraphite.cpp +++ b/src/Storages/System/StorageSystemGraphite.cpp @@ -10,7 +10,6 @@ NamesAndTypesList StorageSystemGraphite::getNamesAndTypes() { return { {"config_name", std::make_shared()}, - {"rule_type", std::make_shared()}, {"regexp", std::make_shared()}, {"function", std::make_shared()}, {"age", std::make_shared()}, @@ -86,7 +85,6 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co bool is_default = pattern.regexp == nullptr; String regexp; String function; - const String & rule_type = ruleTypeStr(pattern.rule_type); if (is_default) { @@ -109,7 +107,6 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co { size_t i = 0; res_columns[i++]->insert(config.first); - res_columns[i++]->insert(rule_type); res_columns[i++]->insert(regexp); res_columns[i++]->insert(function); res_columns[i++]->insert(retention.age); @@ -124,7 +121,6 @@ void StorageSystemGraphite::fillData(MutableColumns & res_columns, ContextPtr co { size_t i = 0; res_columns[i++]->insert(config.first); - res_columns[i++]->insert(rule_type); res_columns[i++]->insert(regexp); res_columns[i++]->insert(function); res_columns[i++]->insertDefault(); diff --git a/tests/integration/helpers/test_tools.py b/tests/integration/helpers/test_tools.py index ec3841f79d7..3577553be34 100644 --- a/tests/integration/helpers/test_tools.py +++ b/tests/integration/helpers/test_tools.py @@ -100,19 +100,3 @@ def exec_query_with_retry(instance, query, retry_count=40, sleep_time=0.5, silen time.sleep(sleep_time) else: raise exception - -def csv_compare(result, expected): - csv_result = TSV(result) - csv_expected = TSV(expected) - mismatch = [] - max_len = len(csv_result) if len(csv_result) > len(csv_expected) else len(csv_expected) - for i in range(max_len): - if i >= len(csv_result): - mismatch.append("-[%d]=%s" % (i, csv_expected.lines[i])) - elif i >= len(csv_expected): - mismatch.append("+[%d]=%s" % (i, csv_result.lines[i])) - elif csv_expected.lines[i] != csv_result.lines[i]: - mismatch.append("-[%d]=%s" % (i, csv_expected.lines[i])) - mismatch.append("+[%d]=%s" % (i, csv_result.lines[i])) - - return "\n".join(mismatch) diff --git a/tests/integration/test_graphite_merge_tree/test.py b/tests/integration/test_graphite_merge_tree/test.py index 9e48f12f007..7628211551d 100644 --- a/tests/integration/test_graphite_merge_tree/test.py +++ b/tests/integration/test_graphite_merge_tree/test.py @@ -6,7 +6,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -from helpers.test_tools import csv_compare cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', @@ -235,19 +234,18 @@ SELECT * FROM test.graphite; def test_system_graphite_retentions(graphite_table): expected = ''' -graphite_rollup all \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] -graphite_rollup all \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] -graphite_rollup all ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite'] -graphite_rollup all ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite'] -graphite_rollup all ^five_min\\\\. 0 300 3 0 ['test'] ['graphite'] -graphite_rollup all ^one_min avg 31536000 600 4 0 ['test'] ['graphite'] -graphite_rollup all ^one_min avg 7776000 300 4 0 ['test'] ['graphite'] -graphite_rollup all ^one_min avg 0 60 4 0 ['test'] ['graphite'] +graphite_rollup \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] +graphite_rollup \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] +graphite_rollup ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite'] +graphite_rollup ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite'] +graphite_rollup ^five_min\\\\. 0 300 3 0 ['test'] ['graphite'] +graphite_rollup ^one_min avg 31536000 600 4 0 ['test'] ['graphite'] +graphite_rollup ^one_min avg 7776000 300 4 0 ['test'] ['graphite'] +graphite_rollup ^one_min avg 0 60 4 0 ['test'] ['graphite'] ''' result = q('SELECT * from system.graphite_retentions') - mismatch = csv_compare(result, expected) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n" + assert TSV(result) == TSV(expected) q(''' DROP TABLE IF EXISTS test.graphite2; diff --git a/tests/integration/test_graphite_merge_tree_typed/__init__.py b/tests/integration/test_graphite_merge_tree_typed/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml b/tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml deleted file mode 100644 index c716540a61c..00000000000 --- a/tests/integration/test_graphite_merge_tree_typed/configs/graphite_rollup.xml +++ /dev/null @@ -1,120 +0,0 @@ - - - - metric - timestamp - value - updated - - plain - \.count$ - sum - - - plain - \.max$ - max - - - plain - ^five_min\. - - 0 - 300 - - - 5184000 - 3600 - - - 31536000 - 14400 - - - - plain - ^one_min - avg - - 0 - 60 - - - 7776000 - 300 - - - 31536000 - 600 - - - - tagged - - avg - - 0 - 60 - - - 7776000 - 300 - - - 31536000 - 600 - - - - tag_list - retention=five_min - avg - - 0 - 300 - - - 5184000 - 3600 - - - 31536000 - 14400 - - - - tagged - ^for_taggged - avg - - 0 - 60 - - - 7776000 - 300 - - - 31536000 - 600 - - - - all - ^ten_min\. - sum - - 0 - 600 - - - 5184000 - 7200 - - - 31536000 - 28800 - - - - diff --git a/tests/integration/test_graphite_merge_tree_typed/configs/users.xml b/tests/integration/test_graphite_merge_tree_typed/configs/users.xml deleted file mode 100644 index 66d0cd7e445..00000000000 --- a/tests/integration/test_graphite_merge_tree_typed/configs/users.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - - 0 - - - diff --git a/tests/integration/test_graphite_merge_tree_typed/test.py b/tests/integration/test_graphite_merge_tree_typed/test.py deleted file mode 100644 index e26fd0d2e77..00000000000 --- a/tests/integration/test_graphite_merge_tree_typed/test.py +++ /dev/null @@ -1,580 +0,0 @@ -import datetime -import os.path as p -import time - -import sys -import pytest -from helpers.client import QueryRuntimeException -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV -from helpers.test_tools import csv_compare - -cluster = ClickHouseCluster(__file__) -instance = cluster.add_instance('instance', - main_configs=['configs/graphite_rollup.xml'], - user_configs=["configs/users.xml"]) -q = instance.query - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - q('CREATE DATABASE test') - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture -def graphite_table(started_cluster): - q(''' -DROP TABLE IF EXISTS test.graphite; -CREATE TABLE test.graphite - (metric String, value Float64, timestamp UInt32, date Date, updated UInt32) - ENGINE = GraphiteMergeTree('graphite_rollup') - PARTITION BY toYYYYMM(date) - ORDER BY (metric, timestamp) - SETTINGS index_granularity=8192; -''') - - yield - - q('DROP TABLE test.graphite') - - -def test_rollup_versions_plain(graphite_table): - timestamp = int(time.time()) - rounded_timestamp = timestamp - timestamp % 60 - date = datetime.date.today().isoformat() - - # Insert rows with timestamps relative to the current time so that the - # first retention clause is active. - # Two parts are created. - q(''' -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('one_min.x1', 100, {timestamp}, '{date}', 1); -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('one_min.x1', 200, {timestamp}, '{date}', 2); -'''.format(timestamp=timestamp, date=date)) - - expected1 = '''\ -one_min.x1 100 {timestamp} {date} 1 -one_min.x1 200 {timestamp} {date} 2 -'''.format(timestamp=timestamp, date=date) - - assert TSV( - q('SELECT * FROM test.graphite ORDER BY updated') - ) == TSV(expected1) - - q('OPTIMIZE TABLE test.graphite') - - # After rollup only the row with max version is retained. - expected2 = '''\ -one_min.x1 200 {timestamp} {date} 2 -'''.format(timestamp=rounded_timestamp, date=date) - - assert TSV(q('SELECT * FROM test.graphite')) == TSV(expected2) - - -def test_rollup_versions_tagged(graphite_table): - timestamp = int(time.time()) - rounded_timestamp = timestamp - timestamp % 60 - date = datetime.date.today().isoformat() - - # Insert rows with timestamps relative to the current time so that the - # first retention clause is active. - # Two parts are created. - q(''' -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('x1?retention=one_min', 100, {timestamp}, '{date}', 1); -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('x1?retention=one_min', 200, {timestamp}, '{date}', 2); -'''.format(timestamp=timestamp, date=date)) - - expected1 = '''\ -x1?retention=one_min 100 {timestamp} {date} 1 -x1?retention=one_min 200 {timestamp} {date} 2 -'''.format(timestamp=timestamp, date=date) - - result = q('SELECT * FROM test.graphite ORDER BY metric, updated') - mismatch = csv_compare(result, expected1) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected1}\ndiff\n{mismatch}\n" - - q('OPTIMIZE TABLE test.graphite') - - # After rollup only the row with max version is retained. - expected2 = '''\ -x1?retention=one_min 200 {timestamp} {date} 2 -'''.format(timestamp=rounded_timestamp, date=date) - - result = q('SELECT * FROM test.graphite ORDER BY metric, updated') - mismatch = csv_compare(result, expected2) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected2}\ndiff\n{mismatch}\n" - - -def test_rollup_versions_all(graphite_table): - timestamp = int(time.time()) - rounded_timestamp = timestamp - timestamp % 600 - date = datetime.date.today().isoformat() - - # Insert rows with timestamps relative to the current time so that the - # first retention clause is active. - # Two parts are created. - q(''' -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('ten_min.x1', 100, {timestamp}, '{date}', 1); -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('ten_min.x1', 200, {timestamp}, '{date}', 2); -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('ten_min.x1?env=staging', 100, {timestamp}, '{date}', 1); -INSERT INTO test.graphite (metric, value, timestamp, date, updated) - VALUES ('ten_min.x1?env=staging', 200, {timestamp}, '{date}', 2); -'''.format(timestamp=timestamp, date=date)) - - expected1 = '''\ -ten_min.x1 100 {timestamp} {date} 1 -ten_min.x1 200 {timestamp} {date} 2 -ten_min.x1?env=staging 100 {timestamp} {date} 1 -ten_min.x1?env=staging 200 {timestamp} {date} 2 -'''.format(timestamp=timestamp, date=date) - - result = q('SELECT * FROM test.graphite ORDER BY metric, updated') - mismatch = csv_compare(result, expected1) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected1}\ndiff\n{mismatch}\n" - - q('OPTIMIZE TABLE test.graphite') - - # After rollup only the row with max version is retained. - expected2 = '''\ -ten_min.x1 200 {timestamp} {date} 2 -ten_min.x1?env=staging 200 {timestamp} {date} 2 -'''.format(timestamp=rounded_timestamp, date=date) - - result = q('SELECT * FROM test.graphite ORDER BY metric, updated') - mismatch = csv_compare(result, expected2) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected2}\ndiff\n{mismatch}\n" - - -def test_rollup_aggregation_plain(graphite_table): - # This query essentially emulates what rollup does. - result1 = q(''' -SELECT avg(v), max(upd) -FROM (SELECT timestamp, - argMax(value, (updated, number)) AS v, - max(updated) AS upd - FROM (SELECT 'one_min.x5' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(intDiv(number, 2)) AS updated, - number - FROM system.numbers LIMIT 1000000) - WHERE intDiv(timestamp, 600) * 600 = 1111444200 - GROUP BY timestamp) -''') - - expected1 = '''\ -999634.9918367347 499999 -''' - assert TSV(result1) == TSV(expected1) - - # Timestamp 1111111111 is in sufficiently distant past - # so that the last retention clause is active. - result2 = q(''' -INSERT INTO test.graphite - SELECT 'one_min.x' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, - toDate('2017-02-02') AS date, toUInt32(intDiv(number, 2)) AS updated - FROM (SELECT * FROM system.numbers LIMIT 1000000) - WHERE intDiv(timestamp, 600) * 600 = 1111444200; - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; -''') - - expected2 = '''\ -one_min.x 999634.9918367347 1111444200 2017-02-02 499999 -''' - - assert TSV(result2) == TSV(expected2) - - -def test_rollup_aggregation_tagged(graphite_table): - # This query essentially emulates what rollup does. - result1 = q(''' -SELECT avg(v), max(upd) -FROM (SELECT timestamp, - argMax(value, (updated, number)) AS v, - max(updated) AS upd - FROM (SELECT 'x?retention=one_min' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(intDiv(number, 2)) AS updated, - number - FROM system.numbers LIMIT 1000000) - WHERE intDiv(timestamp, 600) * 600 = 1111444200 - GROUP BY timestamp) -''') - - expected1 = '''\ -999634.9918367347 499999 -''' - assert TSV(result1) == TSV(expected1) - - # Timestamp 1111111111 is in sufficiently distant past - # so that the last retention clause is active. - result2 = q(''' -INSERT INTO test.graphite - SELECT 'x?retention=one_min' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + intDiv(number, 3)) AS timestamp, - toDate('2017-02-02') AS date, toUInt32(intDiv(number, 2)) AS updated - FROM (SELECT * FROM system.numbers LIMIT 1000000) - WHERE intDiv(timestamp, 600) * 600 = 1111444200; - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; -''') - - expected2 = '''\ -x?retention=one_min 999634.9918367347 1111444200 2017-02-02 499999 -''' - - assert TSV(result2) == TSV(expected2) - - -def test_rollup_aggregation_2_plain(graphite_table): - result = q(''' -INSERT INTO test.graphite - SELECT 'one_min.x' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 - intDiv(number, 3)) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(100 - number) AS updated - FROM (SELECT * FROM system.numbers LIMIT 50); - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; -''') - - expected = '''\ -one_min.x 24 1111110600 2017-02-02 100 -''' - - assert TSV(result) == TSV(expected) - - -def test_rollup_aggregation_2_tagged(graphite_table): - result = q(''' -INSERT INTO test.graphite - SELECT 'x?retention=one_min' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 - intDiv(number, 3)) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(100 - number) AS updated - FROM (SELECT * FROM system.numbers LIMIT 50); - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; -''') - - expected = '''\ -x?retention=one_min 24 1111110600 2017-02-02 100 -''' - - assert TSV(result) == TSV(expected) - - -def test_multiple_paths_and_versions_plain(graphite_table): - result = q(''' -INSERT INTO test.graphite - SELECT 'one_min.x' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + intDiv(number, 3) * 600) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(100 - number) AS updated - FROM (SELECT * FROM system.numbers LIMIT 50); - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; - - -INSERT INTO test.graphite - SELECT 'one_min.y' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + number * 600) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(100 - number) AS updated - FROM (SELECT * FROM system.numbers LIMIT 50); - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; -''') - - with open(p.join(p.dirname(__file__), - 'test_multiple_paths_and_versions.reference.plain') - ) as reference: - assert TSV(result) == TSV(reference) - - -def test_multiple_paths_and_versions_tagged(graphite_table): - result = q(''' -INSERT INTO test.graphite - SELECT 'x?retention=one_min' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + intDiv(number, 3) * 600) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(100 - number) AS updated - FROM (SELECT * FROM system.numbers LIMIT 50); - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; - - -INSERT INTO test.graphite - SELECT 'y?retention=one_min' AS metric, - toFloat64(number) AS value, - toUInt32(1111111111 + number * 600) AS timestamp, - toDate('2017-02-02') AS date, - toUInt32(100 - number) AS updated - FROM (SELECT * FROM system.numbers LIMIT 50); - -OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL; - -SELECT * FROM test.graphite; -''') - - with open(p.join(p.dirname(__file__), - 'test_multiple_paths_and_versions.reference.tagged') - ) as reference: - assert TSV(result) == TSV(reference) - - -def test_multiple_output_blocks(graphite_table): - MERGED_BLOCK_SIZE = 8192 - - to_insert = '' - expected = '' - for i in range(2 * MERGED_BLOCK_SIZE + 1): - rolled_up_time = 1000000200 + 600 * i - - for j in range(3): - cur_time = rolled_up_time + 100 * j - to_insert += 'one_min.x1 {} {} 2001-09-09 1\n'.format( - 10 * j, cur_time - ) - to_insert += 'one_min.x1 {} {} 2001-09-09 2\n'.format( - 10 * (j + 1), cur_time - ) - - expected += 'one_min.x1 20 {} 2001-09-09 2\n'.format(rolled_up_time) - - q('INSERT INTO test.graphite FORMAT TSV', to_insert) - - result = q(''' -OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL; - -SELECT * FROM test.graphite; -''') - - assert TSV(result) == TSV(expected) - - -def test_paths_not_matching_any_pattern(graphite_table): - to_insert = '''\ -one_min.x1 100 1000000000 2001-09-09 1 -zzzzzzzz 100 1000000001 2001-09-09 1 -zzzzzzzz 200 1000000001 2001-09-09 2 -''' - - q('INSERT INTO test.graphite FORMAT TSV', to_insert) - - expected = '''\ -one_min.x1 100 999999600 2001-09-09 1 -zzzzzzzz 200 1000000001 2001-09-09 2 -''' - - result = q(''' -OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL; - -SELECT * FROM test.graphite; -''') - - assert TSV(result) == TSV(expected) - - -def test_rules_isolation(graphite_table): - to_insert = '''\ -one_min.x1 100 1000000000 2001-09-09 1 -for_taggged 100 1000000001 2001-09-09 1 -for_taggged 200 1000000001 2001-09-09 2 -one_min?env=staging 100 1000000001 2001-09-09 1 -one_min?env=staging 200 1000000001 2001-09-09 2 -''' - - q('INSERT INTO test.graphite FORMAT TSV', to_insert) - - expected = '''\ -for_taggged 200 1000000001 2001-09-09 2 -one_min.x1 100 999999600 2001-09-09 1 -one_min?env=staging 200 1000000001 2001-09-09 2 -''' - - result = q(''' -OPTIMIZE TABLE test.graphite PARTITION 200109 FINAL; - -SELECT * FROM test.graphite; -''') - - result = q('SELECT * FROM test.graphite ORDER BY metric, updated') - mismatch = csv_compare(result, expected) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n" - - -def test_system_graphite_retentions(graphite_table): - expected = ''' -graphite_rollup plain \\\\.count$ sum 0 0 1 0 ['test'] ['graphite'] -graphite_rollup plain \\\\.max$ max 0 0 2 0 ['test'] ['graphite'] -graphite_rollup plain ^five_min\\\\. 31536000 14400 3 0 ['test'] ['graphite'] -graphite_rollup plain ^five_min\\\\. 5184000 3600 3 0 ['test'] ['graphite'] -graphite_rollup plain ^five_min\\\\. 0 300 3 0 ['test'] ['graphite'] -graphite_rollup plain ^one_min avg 31536000 600 4 0 ['test'] ['graphite'] -graphite_rollup plain ^one_min avg 7776000 300 4 0 ['test'] ['graphite'] -graphite_rollup plain ^one_min avg 0 60 4 0 ['test'] ['graphite'] -graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 31536000 600 5 0 ['test'] ['graphite'] -graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 7776000 300 5 0 ['test'] ['graphite'] -graphite_rollup tagged [\\\\?&]retention=one_min(&.*)?$ avg 0 60 5 0 ['test'] ['graphite'] -graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 31536000 14400 6 0 ['test'] ['graphite'] -graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 5184000 3600 6 0 ['test'] ['graphite'] -graphite_rollup tagged [\\\\?&]retention=five_min(&.*)?$ avg 0 300 6 0 ['test'] ['graphite'] -graphite_rollup tagged ^for_taggged avg 31536000 600 7 0 ['test'] ['graphite'] -graphite_rollup tagged ^for_taggged avg 7776000 300 7 0 ['test'] ['graphite'] -graphite_rollup tagged ^for_taggged avg 0 60 7 0 ['test'] ['graphite'] -graphite_rollup all ^ten_min\\\\. sum 31536000 28800 8 0 ['test'] ['graphite'] -graphite_rollup all ^ten_min\\\\. sum 5184000 7200 8 0 ['test'] ['graphite'] -graphite_rollup all ^ten_min\\\\. sum 0 600 8 0 ['test'] ['graphite'] - ''' - result = q('SELECT * from system.graphite_retentions') - - mismatch = csv_compare(result, expected) - assert len(mismatch) == 0, f"got\n{result}\nwant\n{expected}\ndiff\n{mismatch}\n" - - q(''' -DROP TABLE IF EXISTS test.graphite2; -CREATE TABLE test.graphite2 - (metric String, value Float64, timestamp UInt32, date Date, updated UInt32) - ENGINE = GraphiteMergeTree('graphite_rollup') - PARTITION BY toYYYYMM(date) - ORDER BY (metric, timestamp) - SETTINGS index_granularity=8192; - ''') - expected = ''' -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] -graphite_rollup ['test','test'] ['graphite','graphite2'] - ''' - result = q(''' - SELECT - config_name, - Tables.database, - Tables.table - FROM system.graphite_retentions - ''') - assert csv_compare(result, expected), f"got\n{result}\nwant\n{expected}" - - -def test_path_dangling_pointer(graphite_table): - q(''' -DROP TABLE IF EXISTS test.graphite2; -CREATE TABLE test.graphite2 - (metric String, value Float64, timestamp UInt32, date Date, updated UInt32) - ENGINE = GraphiteMergeTree('graphite_rollup') - PARTITION BY toYYYYMM(date) - ORDER BY (metric, timestamp) - SETTINGS index_granularity=1; - ''') - - path = 'abcd' * 4000000 # 16MB - q('INSERT INTO test.graphite2 FORMAT TSV', - "{}\t0.0\t0\t2018-01-01\t100\n".format(path)) - q('INSERT INTO test.graphite2 FORMAT TSV', - "{}\t0.0\t0\t2018-01-01\t101\n".format(path)) - for version in range(10): - q('INSERT INTO test.graphite2 FORMAT TSV', - "{}\t0.0\t0\t2018-01-01\t{}\n".format(path, version)) - - while True: - q('OPTIMIZE TABLE test.graphite2 PARTITION 201801 FINAL') - parts = int(q("SELECT count() FROM system.parts " - "WHERE active AND database='test' " - "AND table='graphite2'")) - if parts == 1: - break - print(('Parts', parts)) - - assert TSV( - q("SELECT value, timestamp, date, updated FROM test.graphite2") - ) == TSV("0\t0\t2018-01-01\t101\n") - - q('DROP TABLE test.graphite2') - - -def test_combined_rules(graphite_table): - # 1487970000 ~ Sat 25 Feb 00:00:00 MSK 2017 - to_insert = 'INSERT INTO test.graphite VALUES ' - expected_unmerged = '' - for i in range(384): - to_insert += "('five_min.count', {v}, {t}, toDate({t}), 1), ".format( - v=1, t=1487970000 + (i * 300) - ) - to_insert += "('five_min.max', {v}, {t}, toDate({t}), 1), ".format( - v=i, t=1487970000 + (i * 300) - ) - expected_unmerged += ("five_min.count\t{v1}\t{t}\n" - "five_min.max\t{v2}\t{t}\n").format( - v1=1, v2=i, - t=1487970000 + (i * 300) - ) - - q(to_insert) - assert TSV(q('SELECT metric, value, timestamp FROM test.graphite' - ' ORDER BY (timestamp, metric)')) == TSV(expected_unmerged) - - q('OPTIMIZE TABLE test.graphite PARTITION 201702 FINAL') - expected_merged = ''' - five_min.count 48 1487970000 2017-02-25 1 - five_min.count 48 1487984400 2017-02-25 1 - five_min.count 48 1487998800 2017-02-25 1 - five_min.count 48 1488013200 2017-02-25 1 - five_min.count 48 1488027600 2017-02-25 1 - five_min.count 48 1488042000 2017-02-25 1 - five_min.count 48 1488056400 2017-02-26 1 - five_min.count 48 1488070800 2017-02-26 1 - five_min.max 47 1487970000 2017-02-25 1 - five_min.max 95 1487984400 2017-02-25 1 - five_min.max 143 1487998800 2017-02-25 1 - five_min.max 191 1488013200 2017-02-25 1 - five_min.max 239 1488027600 2017-02-25 1 - five_min.max 287 1488042000 2017-02-25 1 - five_min.max 335 1488056400 2017-02-26 1 - five_min.max 383 1488070800 2017-02-26 1 - ''' - assert TSV(q('SELECT * FROM test.graphite' - ' ORDER BY (metric, timestamp)')) == TSV(expected_merged) diff --git a/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain b/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain deleted file mode 100644 index 0f10d11ed05..00000000000 --- a/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.plain +++ /dev/null @@ -1,84 +0,0 @@ -one_min.x 0 1111110600 2017-02-02 100 -one_min.x 3 1111111200 2017-02-02 97 -one_min.x 6 1111111800 2017-02-02 94 -one_min.x 9 1111112400 2017-02-02 91 -one_min.x 12 1111113000 2017-02-02 88 -one_min.x 15 1111113600 2017-02-02 85 -one_min.x 18 1111114200 2017-02-02 82 -one_min.x 21 1111114800 2017-02-02 79 -one_min.x 24 1111115400 2017-02-02 76 -one_min.x 27 1111116000 2017-02-02 73 -one_min.x 30 1111116600 2017-02-02 70 -one_min.x 33 1111117200 2017-02-02 67 -one_min.x 36 1111117800 2017-02-02 64 -one_min.x 39 1111118400 2017-02-02 61 -one_min.x 42 1111119000 2017-02-02 58 -one_min.x 45 1111119600 2017-02-02 55 -one_min.x 48 1111120200 2017-02-02 52 -one_min.x 0 1111110600 2017-02-02 100 -one_min.x 3 1111111200 2017-02-02 97 -one_min.x 6 1111111800 2017-02-02 94 -one_min.x 9 1111112400 2017-02-02 91 -one_min.x 12 1111113000 2017-02-02 88 -one_min.x 15 1111113600 2017-02-02 85 -one_min.x 18 1111114200 2017-02-02 82 -one_min.x 21 1111114800 2017-02-02 79 -one_min.x 24 1111115400 2017-02-02 76 -one_min.x 27 1111116000 2017-02-02 73 -one_min.x 30 1111116600 2017-02-02 70 -one_min.x 33 1111117200 2017-02-02 67 -one_min.x 36 1111117800 2017-02-02 64 -one_min.x 39 1111118400 2017-02-02 61 -one_min.x 42 1111119000 2017-02-02 58 -one_min.x 45 1111119600 2017-02-02 55 -one_min.x 48 1111120200 2017-02-02 52 -one_min.y 0 1111110600 2017-02-02 100 -one_min.y 1 1111111200 2017-02-02 99 -one_min.y 2 1111111800 2017-02-02 98 -one_min.y 3 1111112400 2017-02-02 97 -one_min.y 4 1111113000 2017-02-02 96 -one_min.y 5 1111113600 2017-02-02 95 -one_min.y 6 1111114200 2017-02-02 94 -one_min.y 7 1111114800 2017-02-02 93 -one_min.y 8 1111115400 2017-02-02 92 -one_min.y 9 1111116000 2017-02-02 91 -one_min.y 10 1111116600 2017-02-02 90 -one_min.y 11 1111117200 2017-02-02 89 -one_min.y 12 1111117800 2017-02-02 88 -one_min.y 13 1111118400 2017-02-02 87 -one_min.y 14 1111119000 2017-02-02 86 -one_min.y 15 1111119600 2017-02-02 85 -one_min.y 16 1111120200 2017-02-02 84 -one_min.y 17 1111120800 2017-02-02 83 -one_min.y 18 1111121400 2017-02-02 82 -one_min.y 19 1111122000 2017-02-02 81 -one_min.y 20 1111122600 2017-02-02 80 -one_min.y 21 1111123200 2017-02-02 79 -one_min.y 22 1111123800 2017-02-02 78 -one_min.y 23 1111124400 2017-02-02 77 -one_min.y 24 1111125000 2017-02-02 76 -one_min.y 25 1111125600 2017-02-02 75 -one_min.y 26 1111126200 2017-02-02 74 -one_min.y 27 1111126800 2017-02-02 73 -one_min.y 28 1111127400 2017-02-02 72 -one_min.y 29 1111128000 2017-02-02 71 -one_min.y 30 1111128600 2017-02-02 70 -one_min.y 31 1111129200 2017-02-02 69 -one_min.y 32 1111129800 2017-02-02 68 -one_min.y 33 1111130400 2017-02-02 67 -one_min.y 34 1111131000 2017-02-02 66 -one_min.y 35 1111131600 2017-02-02 65 -one_min.y 36 1111132200 2017-02-02 64 -one_min.y 37 1111132800 2017-02-02 63 -one_min.y 38 1111133400 2017-02-02 62 -one_min.y 39 1111134000 2017-02-02 61 -one_min.y 40 1111134600 2017-02-02 60 -one_min.y 41 1111135200 2017-02-02 59 -one_min.y 42 1111135800 2017-02-02 58 -one_min.y 43 1111136400 2017-02-02 57 -one_min.y 44 1111137000 2017-02-02 56 -one_min.y 45 1111137600 2017-02-02 55 -one_min.y 46 1111138200 2017-02-02 54 -one_min.y 47 1111138800 2017-02-02 53 -one_min.y 48 1111139400 2017-02-02 52 -one_min.y 49 1111140000 2017-02-02 51 diff --git a/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged b/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged deleted file mode 100644 index e2c63ab3b22..00000000000 --- a/tests/integration/test_graphite_merge_tree_typed/test_multiple_paths_and_versions.reference.tagged +++ /dev/null @@ -1,84 +0,0 @@ -x?retention=one_min 0 1111110600 2017-02-02 100 -x?retention=one_min 3 1111111200 2017-02-02 97 -x?retention=one_min 6 1111111800 2017-02-02 94 -x?retention=one_min 9 1111112400 2017-02-02 91 -x?retention=one_min 12 1111113000 2017-02-02 88 -x?retention=one_min 15 1111113600 2017-02-02 85 -x?retention=one_min 18 1111114200 2017-02-02 82 -x?retention=one_min 21 1111114800 2017-02-02 79 -x?retention=one_min 24 1111115400 2017-02-02 76 -x?retention=one_min 27 1111116000 2017-02-02 73 -x?retention=one_min 30 1111116600 2017-02-02 70 -x?retention=one_min 33 1111117200 2017-02-02 67 -x?retention=one_min 36 1111117800 2017-02-02 64 -x?retention=one_min 39 1111118400 2017-02-02 61 -x?retention=one_min 42 1111119000 2017-02-02 58 -x?retention=one_min 45 1111119600 2017-02-02 55 -x?retention=one_min 48 1111120200 2017-02-02 52 -x?retention=one_min 0 1111110600 2017-02-02 100 -x?retention=one_min 3 1111111200 2017-02-02 97 -x?retention=one_min 6 1111111800 2017-02-02 94 -x?retention=one_min 9 1111112400 2017-02-02 91 -x?retention=one_min 12 1111113000 2017-02-02 88 -x?retention=one_min 15 1111113600 2017-02-02 85 -x?retention=one_min 18 1111114200 2017-02-02 82 -x?retention=one_min 21 1111114800 2017-02-02 79 -x?retention=one_min 24 1111115400 2017-02-02 76 -x?retention=one_min 27 1111116000 2017-02-02 73 -x?retention=one_min 30 1111116600 2017-02-02 70 -x?retention=one_min 33 1111117200 2017-02-02 67 -x?retention=one_min 36 1111117800 2017-02-02 64 -x?retention=one_min 39 1111118400 2017-02-02 61 -x?retention=one_min 42 1111119000 2017-02-02 58 -x?retention=one_min 45 1111119600 2017-02-02 55 -x?retention=one_min 48 1111120200 2017-02-02 52 -y?retention=one_min 0 1111110600 2017-02-02 100 -y?retention=one_min 1 1111111200 2017-02-02 99 -y?retention=one_min 2 1111111800 2017-02-02 98 -y?retention=one_min 3 1111112400 2017-02-02 97 -y?retention=one_min 4 1111113000 2017-02-02 96 -y?retention=one_min 5 1111113600 2017-02-02 95 -y?retention=one_min 6 1111114200 2017-02-02 94 -y?retention=one_min 7 1111114800 2017-02-02 93 -y?retention=one_min 8 1111115400 2017-02-02 92 -y?retention=one_min 9 1111116000 2017-02-02 91 -y?retention=one_min 10 1111116600 2017-02-02 90 -y?retention=one_min 11 1111117200 2017-02-02 89 -y?retention=one_min 12 1111117800 2017-02-02 88 -y?retention=one_min 13 1111118400 2017-02-02 87 -y?retention=one_min 14 1111119000 2017-02-02 86 -y?retention=one_min 15 1111119600 2017-02-02 85 -y?retention=one_min 16 1111120200 2017-02-02 84 -y?retention=one_min 17 1111120800 2017-02-02 83 -y?retention=one_min 18 1111121400 2017-02-02 82 -y?retention=one_min 19 1111122000 2017-02-02 81 -y?retention=one_min 20 1111122600 2017-02-02 80 -y?retention=one_min 21 1111123200 2017-02-02 79 -y?retention=one_min 22 1111123800 2017-02-02 78 -y?retention=one_min 23 1111124400 2017-02-02 77 -y?retention=one_min 24 1111125000 2017-02-02 76 -y?retention=one_min 25 1111125600 2017-02-02 75 -y?retention=one_min 26 1111126200 2017-02-02 74 -y?retention=one_min 27 1111126800 2017-02-02 73 -y?retention=one_min 28 1111127400 2017-02-02 72 -y?retention=one_min 29 1111128000 2017-02-02 71 -y?retention=one_min 30 1111128600 2017-02-02 70 -y?retention=one_min 31 1111129200 2017-02-02 69 -y?retention=one_min 32 1111129800 2017-02-02 68 -y?retention=one_min 33 1111130400 2017-02-02 67 -y?retention=one_min 34 1111131000 2017-02-02 66 -y?retention=one_min 35 1111131600 2017-02-02 65 -y?retention=one_min 36 1111132200 2017-02-02 64 -y?retention=one_min 37 1111132800 2017-02-02 63 -y?retention=one_min 38 1111133400 2017-02-02 62 -y?retention=one_min 39 1111134000 2017-02-02 61 -y?retention=one_min 40 1111134600 2017-02-02 60 -y?retention=one_min 41 1111135200 2017-02-02 59 -y?retention=one_min 42 1111135800 2017-02-02 58 -y?retention=one_min 43 1111136400 2017-02-02 57 -y?retention=one_min 44 1111137000 2017-02-02 56 -y?retention=one_min 45 1111137600 2017-02-02 55 -y?retention=one_min 46 1111138200 2017-02-02 54 -y?retention=one_min 47 1111138800 2017-02-02 53 -y?retention=one_min 48 1111139400 2017-02-02 52 -y?retention=one_min 49 1111140000 2017-02-02 51 diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index a2e56fa0f1d..2b391cd292e 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -21,7 +21,7 @@ CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `de CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' -CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `rule_type` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' CREATE TABLE system.merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index a930e7db3fc..8309b6bcb53 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -32,7 +32,6 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (wal-dump) add_subdirectory (check-mysql-binlog) add_subdirectory (keeper-bench) - add_subdirectory (graphite-rollup) if (USE_NURAFT) add_subdirectory (keeper-data-dumper) diff --git a/utils/graphite-rollup/CMakeLists.txt b/utils/graphite-rollup/CMakeLists.txt deleted file mode 100644 index 3cc0d3e756f..00000000000 --- a/utils/graphite-rollup/CMakeLists.txt +++ /dev/null @@ -1,23 +0,0 @@ -add_executable(graphite-rollup-bench graphite-rollup-bench.cpp) -target_link_libraries( - graphite-rollup-bench - PRIVATE - clickhouse_storages_system - clickhouse_aggregate_functions - clickhouse_common_config - dbms -) -target_include_directories( - graphite-rollup-bench - SYSTEM PRIVATE - ${ClickHouse_SOURCE_DIR}/src ${CMAKE_BINARY_DIR}/src - ${ClickHouse_SOURCE_DIR}/base ${ClickHouse_SOURCE_DIR}/base/pcg-random - ${CMAKE_BINARY_DIR}/src/Core/include - ${POCO_INCLUDE_DIR} - ${ClickHouse_SOURCE_DIR}/contrib/double-conversion ${ClickHouse_SOURCE_DIR}/contrib/dragonbox/include - ${ClickHouse_SOURCE_DIR}/contrib/fmtlib/include - ${ClickHouse_SOURCE_DIR}/contrib/cityhash102/include - ${RE2_INCLUDE_DIR} ${CMAKE_BINARY_DIR}/contrib/re2_st -) - -target_compile_definitions(graphite-rollup-bench PRIVATE RULES_DIR="${CMAKE_CURRENT_SOURCE_DIR}") diff --git a/utils/graphite-rollup/graphite-rollup-bench.cpp b/utils/graphite-rollup/graphite-rollup-bench.cpp deleted file mode 100644 index dabe0353b0f..00000000000 --- a/utils/graphite-rollup/graphite-rollup-bench.cpp +++ /dev/null @@ -1,147 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -#include - -#include -#include -#include -#include -#include -#include -#include - -using namespace DB; - -static SharedContextHolder shared_context = Context::createShared(); - -std::vector loadMetrics(const std::string & metrics_file) -{ - std::vector metrics; - - FILE * stream; - char * line = nullptr; - size_t len = 0; - ssize_t nread; - - stream = fopen(metrics_file.c_str(), "r"); - if (stream == nullptr) - { - throw std::runtime_error(strerror(errno)); - } - - while ((nread = getline(&line, &len, stream)) != -1) - { - size_t l = strlen(line); - if (l > 0) - { - if (line[l - 1] == '\n') - { - line[l - 1] = '\0'; - l--; - } - if (l > 0) - { - metrics.push_back(StringRef(strdup(line), l)); - } - } - } - free(line); - if (ferror(stream)) - { - fclose(stream); - throw std::runtime_error(strerror(errno)); - } - - fclose(stream); - - return metrics; -} - -ConfigProcessor::LoadedConfig loadConfiguration(const std::string & config_path) -{ - ConfigProcessor config_processor(config_path, true, true); - ConfigProcessor::LoadedConfig config = config_processor.loadConfig(false); - return config; -} - -void bench(const std::string & config_path, const std::string & metrics_file, size_t n, bool verbose) -{ - auto config = loadConfiguration(config_path); - - auto context = Context::createGlobal(shared_context.get()); - context->setConfig(config.configuration.get()); - - Graphite::Params params; - setGraphitePatternsFromConfig(context, "graphite_rollup", params); - - std::vector metrics = loadMetrics(metrics_file); - - std::vector durations(metrics.size()); - size_t j, i; - for (j = 0; j < n; j++) - { - for (i = 0; i < metrics.size(); i++) - { - auto start = std::chrono::high_resolution_clock::now(); - - auto rule = DB::Graphite::selectPatternForPath(params, metrics[i]); - (void)rule; - - auto end = std::chrono::high_resolution_clock::now(); - double duration = (duration_cast>(end - start)).count() * 1E9; - durations[i] += duration; - - if (j == 0 && verbose) - { - std::cout << metrics[i].data << ": rule with regexp '" << rule.second->regexp_str << "' found\n"; - } - } - } - - for (i = 0; i < metrics.size(); i++) - { - std::cout << metrics[i].data << " " << durations[i] / n << " ns\n"; - free(const_cast(static_cast(metrics[i].data))); - } -} - -int main(int argc, char ** argv) -{ - registerAggregateFunctions(); - - std::string config_file, metrics_file; - - using namespace std::literals; - - std::string config_default = RULES_DIR + "/rollup.xml"s; - std::string metrics_default = RULES_DIR + "/metrics.txt"s; - - namespace po = boost::program_options; - po::variables_map vm; - - po::options_description desc; - desc.add_options()("help,h", "produce help")( - "config,c", po::value()->default_value(config_default), "XML config with rollup rules")( - "metrics,m", po::value()->default_value(metrics_default), "metrcis files (one metric per line) for run benchmark")( - "verbose,V", po::bool_switch()->default_value(false), "verbose output (print found rule)"); - - po::parsed_options parsed = po::command_line_parser(argc, argv).options(desc).run(); - po::store(parsed, vm); - po::notify(vm); - - if (vm.count("help")) - { - std::cout << desc << '\n'; - exit(1); - } - - bench(vm["config"].as(), vm["metrics"].as(), 10000, vm["verbose"].as()); - - return 0; -} diff --git a/utils/graphite-rollup/metrics.txt b/utils/graphite-rollup/metrics.txt deleted file mode 100644 index 199c3791310..00000000000 --- a/utils/graphite-rollup/metrics.txt +++ /dev/null @@ -1,11 +0,0 @@ -test.sum -sum?env=test&tag=Fake3 -test.max -max?env=test&tag=Fake4 -test.min -min?env=test&tag=Fake5 -fake5?env=test&tag=Fake5 -test.p95 -p95?env=test&tag=FakeNo -default -default?env=test&tag=FakeNo diff --git a/utils/graphite-rollup/rollup-tag-list.xml b/utils/graphite-rollup/rollup-tag-list.xml deleted file mode 100644 index ef28f2089ad..00000000000 --- a/utils/graphite-rollup/rollup-tag-list.xml +++ /dev/null @@ -1,167 +0,0 @@ - - - - plain - \.sum$ - sum - - 0 - 60 - - - 86400 - 3600 - - - - tagged - ^((.*)|.)sum\? - sum - - 0 - 60 - - - 86400 - 3600 - - - - plain - \.max$ - max - - 0 - 60 - - - 86400 - 3600 - - - - tagged - ^((.*)|.)max\? - max - - 0 - 60 - - - 86400 - 3600 - - - - plain - \.min$ - min - - 0 - 60 - - - 86400 - 3600 - - - - tagged - ^((.*)|.)min\? - min - - 0 - 60 - - - 86400 - 3600 - - - - plain - \.fake1\..*\.Fake1\. - sum - - - tag_list - fake1;tag=Fake1 - sum - - - plain - \.fake2\..*\.Fake2\. - sum - - - tag_list - fake2;tag=Fake2 - sum - - - plain - \.fake3\..*\.Fake3\. - sum - - - tag_list - fake3;tag=Fake3 - sum - - - plain - \.fake4\..*\.Fake4\. - sum - - - tag_list - fake4;tag=Fake4 - sum - - - plain - \.fake5\..*\.Fake5\. - sum - - - tag_list - fake5;tag=Fake5 - sum - - - plain - \.fake6\..*\.Fake6\. - sum - - - tag_list - fake6;tag=Fake6 - sum - - - plain - \.fake7\..*\.Fake7\. - sum - - - tag_list - fake7;tag=Fake7 - sum - - - avg - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - diff --git a/utils/graphite-rollup/rollup-typed.xml b/utils/graphite-rollup/rollup-typed.xml deleted file mode 100644 index 0b27d43ece9..00000000000 --- a/utils/graphite-rollup/rollup-typed.xml +++ /dev/null @@ -1,167 +0,0 @@ - - - - plain - \.sum$ - sum - - 0 - 60 - - - 86400 - 3600 - - - - tagged - ^((.*)|.)sum\? - sum - - 0 - 60 - - - 86400 - 3600 - - - - plain - \.max$ - max - - 0 - 60 - - - 86400 - 3600 - - - - tagged - ^((.*)|.)max\? - max - - 0 - 60 - - - 86400 - 3600 - - - - plain - \.min$ - min - - 0 - 60 - - - 86400 - 3600 - - - - tagged - ^((.*)|.)min\? - min - - 0 - 60 - - - 86400 - 3600 - - - - plain - \.fake1\..*\.Fake1\. - sum - - - tagged - - sum - - - plain - \.fake2\..*\.Fake2\. - sum - - - tagged - - sum - - - plain - \.fake3\..*\.Fake3\. - sum - - - tagged - - sum - - - plain - \.fake4\..*\.Fake4\. - sum - - - tagged - - sum - - - plain - \.fake5\..*\.Fake5\. - sum - - - tagged - - sum - - - plain - \.fake6\..*\.Fake6\. - sum - - - tagged - - sum - - - plain - \.fake7\..*\.Fake7\. - sum - - - tagged - - sum - - - avg - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - diff --git a/utils/graphite-rollup/rollup.xml b/utils/graphite-rollup/rollup.xml deleted file mode 100644 index 641b0130509..00000000000 --- a/utils/graphite-rollup/rollup.xml +++ /dev/null @@ -1,147 +0,0 @@ - - - - \.sum$ - sum - - 0 - 60 - - - 86400 - 3600 - - - - ^((.*)|.)sum\? - sum - - 0 - 60 - - - 86400 - 3600 - - - - \.max$ - max - - 0 - 60 - - - 86400 - 3600 - - - - ^((.*)|.)max\? - max - - 0 - 60 - - - 86400 - 3600 - - - - \.min$ - min - - 0 - 60 - - - 86400 - 3600 - - - - ^((.*)|.)min\? - min - - 0 - 60 - - - 86400 - 3600 - - - - \.fake1\..*\.Fake1\. - sum - - - - sum - - - \.fake2\..*\.Fake2\. - sum - - - - sum - - - \.fake3\..*\.Fake3\. - sum - - - - sum - - - \.fake4\..*\.Fake4\. - sum - - - - sum - - - \.fake5\..*\.Fake5\. - sum - - - - sum - - - \.fake6\..*\.Fake6\. - sum - - - - sum - - - \.fake7\..*\.Fake7\. - sum - - - - sum - - - avg - - 0 - 60 - - - 3600 - 300 - - - 86400 - 3600 - - - - From e90f047ba0df2c901e826117bb1e8b74457ccf08 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Wed, 8 Dec 2021 11:06:10 -0400 Subject: [PATCH 584/609] Doc. merge_tree_clear_old_temporary_directories_interval_seconds merge_tree_clear_old_parts_interval_seconds were moved to merge_tree settings --- .../settings/merge-tree-settings.md | 21 +++++++++++++++++++ docs/en/operations/settings/settings.md | 20 ------------------ .../settings/merge-tree-settings.md | 20 ++++++++++++++++++ docs/ru/operations/settings/settings.md | 20 ------------------ 4 files changed, 41 insertions(+), 40 deletions(-) diff --git a/docs/en/operations/settings/merge-tree-settings.md b/docs/en/operations/settings/merge-tree-settings.md index 0fd1e54955c..af75d130ed3 100644 --- a/docs/en/operations/settings/merge-tree-settings.md +++ b/docs/en/operations/settings/merge-tree-settings.md @@ -356,3 +356,24 @@ Possible values: - 1 — Parts are detached. Default value: `0`. + +## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds} + +Sets the interval in seconds for ClickHouse to execute the cleanup of old temporary directories. + +Possible values: + +- Any positive integer. + +Default value: `60` seconds. + +## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds} + +Sets the interval in seconds for ClickHouse to execute the cleanup of old parts, WALs, and mutations. + +Possible values: + +- Any positive integer. + +Default value: `1` second. + diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index fa4cc41e8ff..30d129d9b29 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -885,26 +885,6 @@ Possible values: Default value: 2013265920. -## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds} - -Sets the interval in seconds for ClickHouse to execute the cleanup of old temporary directories. - -Possible values: - -- Any positive integer. - -Default value: `60` seconds. - -## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds} - -Sets the interval in seconds for ClickHouse to execute the cleanup of old parts, WALs, and mutations. - -Possible values: - -- Any positive integer. - -Default value: `1` second. - ## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} The minimum data volume required for using direct I/O access to the storage disk. diff --git a/docs/ru/operations/settings/merge-tree-settings.md b/docs/ru/operations/settings/merge-tree-settings.md index 31cc229c6aa..e30539498b3 100644 --- a/docs/ru/operations/settings/merge-tree-settings.md +++ b/docs/ru/operations/settings/merge-tree-settings.md @@ -355,3 +355,23 @@ Eсли суммарное число активных кусков во все - 1 — куски данных открепляются. Значение по умолчанию: `0`. + +## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds} + +Задает интервал в секундах для удаления старых временных каталогов на сервере ClickHouse. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `60` секунд. + +## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds} + +Задает интервал в секундах для удаления старых кусков данных, журналов предзаписи (WAL) и мутаций на сервере ClickHouse. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `1` секунда. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index f9717b0fb27..1b4da512c9f 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -807,26 +807,6 @@ ClickHouse может парсить только базовый формат `Y Значение по умолчанию: 2013265920. -## merge_tree_clear_old_temporary_directories_interval_seconds {#setting-merge-tree-clear-old-temporary-directories-interval-seconds} - -Задает интервал в секундах для удаления старых временных каталогов на сервере ClickHouse. - -Возможные значения: - -- Положительное целое число. - -Значение по умолчанию: `60` секунд. - -## merge_tree_clear_old_parts_interval_seconds {#setting-merge-tree-clear-old-parts-interval-seconds} - -Задает интервал в секундах для удаления старых кусков данных, журналов предзаписи (WAL) и мутаций на сервере ClickHouse . - -Возможные значения: - -- Положительное целое число. - -Значение по умолчанию: `1` секунда. - ## min_bytes_to_use_direct_io {#settings-min-bytes-to-use-direct-io} Минимальный объём данных, необходимый для прямого (небуферизованного) чтения/записи (direct I/O) на диск. From 417bd12629fc53faf28dd3862641462babb3593e Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 2 Dec 2021 17:19:09 +0300 Subject: [PATCH 585/609] Fix 'APPLY lambda' parsing --- src/Parsers/ExpressionElementParsers.cpp | 10 ++++++++-- .../02128_apply_lambda_parsing.reference | 10 ++++++++++ .../0_stateless/02128_apply_lambda_parsing.sql | 13 +++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02128_apply_lambda_parsing.reference create mode 100644 tests/queries/0_stateless/02128_apply_lambda_parsing.sql diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e9ec7b43a21..e598698daf8 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include @@ -35,7 +34,6 @@ #include #include -#include #include "ASTColumnsMatcher.h" #include @@ -48,6 +46,8 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int SYNTAX_ERROR; extern const int LOGICAL_ERROR; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int TYPE_MISMATCH; } @@ -1935,7 +1935,13 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e { if (const auto * func = lambda->as(); func && func->name == "lambda") { + if (func->arguments->children.size() != 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "lambda requires two arguments"); + const auto * lambda_args_tuple = func->arguments->children.at(0)->as(); + if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") + throw Exception(ErrorCodes::TYPE_MISMATCH, "First argument of lambda must be a tuple"); + const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children; if (lambda_arg_asts.size() != 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, "APPLY column transformer can only accept lambda with one argument"); diff --git a/tests/queries/0_stateless/02128_apply_lambda_parsing.reference b/tests/queries/0_stateless/02128_apply_lambda_parsing.reference new file mode 100644 index 00000000000..120eec989de --- /dev/null +++ b/tests/queries/0_stateless/02128_apply_lambda_parsing.reference @@ -0,0 +1,10 @@ +1 +1 +1 +1 +1 +1 +2 +3 +4 +5 diff --git a/tests/queries/0_stateless/02128_apply_lambda_parsing.sql b/tests/queries/0_stateless/02128_apply_lambda_parsing.sql new file mode 100644 index 00000000000..51cec494d78 --- /dev/null +++ b/tests/queries/0_stateless/02128_apply_lambda_parsing.sql @@ -0,0 +1,13 @@ +WITH * APPLY lambda(e); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT * APPLY lambda(); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT * APPLY lambda(1); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT * APPLY lambda(x); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT * APPLY lambda(range(1)); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT * APPLY lambda(range(x)); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } +SELECT * APPLY lambda(1, 2); -- { clientError TYPE_MISMATCH } +SELECT * APPLY lambda(x, y); -- { clientError TYPE_MISMATCH } +SELECT * APPLY lambda((x, y), 2); -- { clientError BAD_ARGUMENTS } +SELECT * APPLY lambda((x, y), x + y); -- { clientError BAD_ARGUMENTS } +SELECT * APPLY lambda(tuple(1), 1); -- { clientError BAD_ARGUMENTS } +SELECT * APPLY lambda(tuple(x), 1) FROM numbers(5); +SELECT * APPLY lambda(tuple(x), x + 1) FROM numbers(5); From 7ba60aa770770e9bfc49f3a7824149dade0ff9f4 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 6 Dec 2021 14:03:26 +0300 Subject: [PATCH 586/609] Change error codes to SYNTAX_ERROR --- src/Parsers/ExpressionElementParsers.cpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Parsers/ExpressionElementParsers.cpp b/src/Parsers/ExpressionElementParsers.cpp index e598698daf8..584c2a32afd 100644 --- a/src/Parsers/ExpressionElementParsers.cpp +++ b/src/Parsers/ExpressionElementParsers.cpp @@ -46,8 +46,6 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int SYNTAX_ERROR; extern const int LOGICAL_ERROR; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int TYPE_MISMATCH; } @@ -1936,20 +1934,20 @@ bool ParserColumnsTransformers::parseImpl(Pos & pos, ASTPtr & node, Expected & e if (const auto * func = lambda->as(); func && func->name == "lambda") { if (func->arguments->children.size() != 2) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "lambda requires two arguments"); + throw Exception(ErrorCodes::SYNTAX_ERROR, "lambda requires two arguments"); const auto * lambda_args_tuple = func->arguments->children.at(0)->as(); if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") - throw Exception(ErrorCodes::TYPE_MISMATCH, "First argument of lambda must be a tuple"); + throw Exception(ErrorCodes::SYNTAX_ERROR, "First argument of lambda must be a tuple"); const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children; if (lambda_arg_asts.size() != 1) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "APPLY column transformer can only accept lambda with one argument"); + throw Exception(ErrorCodes::SYNTAX_ERROR, "APPLY column transformer can only accept lambda with one argument"); if (auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[0]); opt_arg_name) lambda_arg = *opt_arg_name; else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "lambda argument declarations must be identifiers"); + throw Exception(ErrorCodes::SYNTAX_ERROR, "lambda argument declarations must be identifiers"); } else { From fc382afacb4a2f0bfe63e544484725d503489b60 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 6 Dec 2021 14:04:19 +0300 Subject: [PATCH 587/609] Update test --- .../02128_apply_lambda_parsing.sql | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02128_apply_lambda_parsing.sql b/tests/queries/0_stateless/02128_apply_lambda_parsing.sql index 51cec494d78..5fc809ca75d 100644 --- a/tests/queries/0_stateless/02128_apply_lambda_parsing.sql +++ b/tests/queries/0_stateless/02128_apply_lambda_parsing.sql @@ -1,13 +1,13 @@ -WITH * APPLY lambda(e); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT * APPLY lambda(); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT * APPLY lambda(1); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT * APPLY lambda(x); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT * APPLY lambda(range(1)); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT * APPLY lambda(range(x)); -- { clientError NUMBER_OF_ARGUMENTS_DOESNT_MATCH } -SELECT * APPLY lambda(1, 2); -- { clientError TYPE_MISMATCH } -SELECT * APPLY lambda(x, y); -- { clientError TYPE_MISMATCH } -SELECT * APPLY lambda((x, y), 2); -- { clientError BAD_ARGUMENTS } -SELECT * APPLY lambda((x, y), x + y); -- { clientError BAD_ARGUMENTS } -SELECT * APPLY lambda(tuple(1), 1); -- { clientError BAD_ARGUMENTS } +WITH * APPLY lambda(e); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(1); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(x); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(range(1)); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(range(x)); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(1, 2); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(x, y); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda((x, y), 2); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda((x, y), x + y); -- { clientError SYNTAX_ERROR } +SELECT * APPLY lambda(tuple(1), 1); -- { clientError SYNTAX_ERROR } SELECT * APPLY lambda(tuple(x), 1) FROM numbers(5); SELECT * APPLY lambda(tuple(x), x + 1) FROM numbers(5); From 81d45dbf661e4b3456803933d8f32f657f1fa714 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 7 Dec 2021 19:09:27 +0300 Subject: [PATCH 588/609] Fix possible crash in fuzzer --- programs/client/Client.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 383b9bb5e52..b6214d66628 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -705,6 +705,12 @@ bool Client::processWithFuzzing(const String & full_query) throw; } + if (!orig_ast) + { + // Can't continue after a parsing error + return true; + } + // `USE db` should not be executed // since this will break every query after `DROP db` if (orig_ast->as()) @@ -712,12 +718,6 @@ bool Client::processWithFuzzing(const String & full_query) return true; } - if (!orig_ast) - { - // Can't continue after a parsing error - return true; - } - // Don't repeat: // - INSERT -- Because the tables may grow too big. // - CREATE -- Because first we run the unmodified query, it will succeed, From 51bb37715f7134d0a15d40183f5b13e1ddc95a26 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 8 Dec 2021 21:28:50 +0300 Subject: [PATCH 589/609] Fix fuzzer --- 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 351b4a3c541..bafbd415e24 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -189,6 +189,7 @@ continue --receive_data_timeout_ms=10000 \ --stacktrace \ --query-fuzzer-runs=1000 \ + --testmode --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \ $NEW_TESTS_OPT \ > >(tail -n 100000 > fuzzer.log) \ From 5e26dcbfa716af3dbd77841818eaec0090db5775 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 9 Dec 2021 00:27:47 +0300 Subject: [PATCH 590/609] Auto version update to [21.12.1.9017] [54457] --- cmake/autogenerated_versions.txt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index f13110d7179..31fef1250da 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -6,7 +6,7 @@ SET(VERSION_REVISION 54457) SET(VERSION_MAJOR 21) SET(VERSION_MINOR 12) SET(VERSION_PATCH 1) -SET(VERSION_GITHASH 503a418dedf0011e9040c3a1b6913e0b5488be4c) -SET(VERSION_DESCRIBE v21.12.1.1-prestable) -SET(VERSION_STRING 21.12.1.1) +SET(VERSION_GITHASH 4cc45c1e15912ee300bca7cc8b8da2b888a70e2a) +SET(VERSION_DESCRIBE v21.12.1.9017-prestable) +SET(VERSION_STRING 21.12.1.9017) # end of autochange From f2ccddb7be65179154d9b80a962c060828934529 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 9 Dec 2021 00:33:01 +0300 Subject: [PATCH 591/609] Auto version update to [21.13.1.1] [54458] --- cmake/autogenerated_versions.txt | 8 +- debian/changelog | 4 +- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- .../StorageSystemContributors.generated.cpp | 941 +++++++++--------- 6 files changed, 499 insertions(+), 460 deletions(-) diff --git a/cmake/autogenerated_versions.txt b/cmake/autogenerated_versions.txt index 31fef1250da..8e7c061088a 100644 --- a/cmake/autogenerated_versions.txt +++ b/cmake/autogenerated_versions.txt @@ -2,11 +2,11 @@ # NOTE: has nothing common with DBMS_TCP_PROTOCOL_VERSION, # only DBMS_TCP_PROTOCOL_VERSION should be incremented on protocol changes. -SET(VERSION_REVISION 54457) +SET(VERSION_REVISION 54458) SET(VERSION_MAJOR 21) -SET(VERSION_MINOR 12) +SET(VERSION_MINOR 13) SET(VERSION_PATCH 1) SET(VERSION_GITHASH 4cc45c1e15912ee300bca7cc8b8da2b888a70e2a) -SET(VERSION_DESCRIBE v21.12.1.9017-prestable) -SET(VERSION_STRING 21.12.1.9017) +SET(VERSION_DESCRIBE v21.13.1.1-prestable) +SET(VERSION_STRING 21.13.1.1) # end of autochange diff --git a/debian/changelog b/debian/changelog index a2709485e44..3c1be00d664 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (21.12.1.1) unstable; urgency=low +clickhouse (21.13.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Tue, 02 Nov 2021 00:56:42 +0300 + -- clickhouse-release Thu, 09 Dec 2021 00:32:58 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index e84cb601c0f..9ce06939a85 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -5,7 +5,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=21.12.1.* +ARG version=21.13.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 96e7e73af33..bfdf65cd56c 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -5,7 +5,7 @@ ARG apt_archive="http://archive.ubuntu.com" RUN sed -i "s|http://archive.ubuntu.com|$apt_archive|g" /etc/apt/sources.list ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=21.12.1.* +ARG version=21.13.1.* ARG gosu_ver=1.10 # set non-empty deb_location_url url to create a docker image diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index 76967da9f9a..c24c013646f 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb https://repo.clickhouse.com/deb/stable/ main/" -ARG version=21.12.1.* +ARG version=21.13.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ diff --git a/src/Storages/System/StorageSystemContributors.generated.cpp b/src/Storages/System/StorageSystemContributors.generated.cpp index c33fa6cad44..87bd266af96 100644 --- a/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/src/Storages/System/StorageSystemContributors.generated.cpp @@ -1,48 +1,41 @@ // autogenerated by ./StorageSystemContributors.sh const char * auto_contributors[] { - "박현우", "0xflotus", "20018712", "243f6a88 85a308d3", "243f6a8885a308d313198a2e037", "3ldar-nasyrov", "821008736@qq.com", - "abdrakhmanov", - "abel-wang", - "abyss7", - "achimbab", - "achulkov2", - "adevyatova", - "ageraab", + "ANDREI STAROVEROV", "Ahmed Dardery", "Aimiyoo", - "akazz", "Akazz", - "akonyaev", - "akuzm", "Alain BERRIER", "Albert Kidrachev", "Alberto", - "Aleksandra (Ася)", "Aleksandr Karo", - "Aleksandrov Vladimir", "Aleksandr Shalimov", - "alekseik1", + "Aleksandra (Ася)", + "Aleksandrov Vladimir", "Aleksei Levushkin", "Aleksei Semiglazov", - "Aleksey", "Aleksey Akulovich", - "alesapin", + "Aleksey", + "Alex Bocharov", + "Alex Cao", + "Alex Karo", + "Alex Krash", + "Alex Ryndin", + "Alex Zatelepin", "Alex", "Alexander Avdonkin", "Alexander Bezpiatov", "Alexander Burmak", "Alexander Chashnikov", "Alexander Ermolaev", - "Alexander Gololobov", "Alexander GQ Gerasiov", + "Alexander Gololobov", "Alexander Kazakov", - "alexander kozhikhov", "Alexander Kozhikhov", "Alexander Krasheninnikov", "Alexander Kuranoff", @@ -59,63 +52,46 @@ const char * auto_contributors[] { "Alexander Sapin", "Alexander Tokmakov", "Alexander Tretiakov", - "Alexandra", - "Alexandra Latysheva", - "Alexandre Snarskii", "Alexandr Kondratev", "Alexandr Krasheninnikov", "Alexandr Orlov", - "Alex Bocharov", + "Alexandra Latysheva", + "Alexandra", + "Alexandre Snarskii", "Alexei Averchenko", - "Alexey", "Alexey Arno", "Alexey Boykov", "Alexey Dushechkin", "Alexey Elymanov", "Alexey Ilyukhov", - "alexey-milovidov", "Alexey Milovidov", "Alexey Tronov", "Alexey Vasiliev", "Alexey Zatelepin", - "Alex Karo", - "Alex Krash", - "alex.lvxin", - "Alex Ryndin", + "Alexey", "Alexsey Shestakov", - "alex-zaitsev", - "Alex Zatelepin", - "alfredlu", + "Ali Demirci", "Aliaksandr Pliutau", "Aliaksandr Shylau", - "Ali Demirci", "Alina Terekhova", - "amesaru", "Amesaru", "Amir Vaza", "Amos Bird", - "amoschen", - "amudong", + "Amr Alaa", "Amy Krishnevsky", + "AnaUvarova", "Anastasiya Rodigina", "Anastasiya Tsarkova", "Anatoly Pugachev", - "ana-uvarova", - "AnaUvarova", "Andr0901", - "andrc1901", "Andreas Hunkeler", "AndreevDm", "Andrei Bodrov", "Andrei Ch", "Andrei Chulkov", - "andrei-karpliuk", "Andrei Nekrashevich", - "ANDREI STAROVEROV", "Andrew Grigorev", "Andrew Onyshchuk", - "andrewsg", - "Andrey", "Andrey Chulkov", "Andrey Dudin", "Andrey Kadochnikov", @@ -124,20 +100,16 @@ const char * auto_contributors[] { "Andrey M", "Andrey Mironov", "Andrey Skobtsov", + "Andrey Torsunov", "Andrey Urusov", "Andrey Z", + "Andrey", "Andy Liang", "Andy Yang", "Anmol Arora", - "Anna", "Anna Shakhova", - "anneji", - "anneji-dev", - "annvsh", - "anrodigina", + "Anna", "Anthony N. Simon", - "antikvist", - "anton", "Anton Ivashkin", "Anton Kobzev", "Anton Kvasha", @@ -149,151 +121,94 @@ const char * auto_contributors[] { "Anton Tikhonov", "Anton Yuzhaninov", "Anton Zhabolenko", - "ap11", - "a.palagashvili", - "aprudaev", "Ariel Robaldo", "Arsen Hakobyan", "Arslan G", "ArtCorp", "Artem Andreenko", - "Artemeey", "Artem Gavrilov", "Artem Hnilov", - "Artemkin Pavel", "Artem Konovalov", "Artem Streltsov", "Artem Zuikov", + "Artemeey", + "Artemkin Pavel", "Arthur Petukhovsky", "Arthur Tokarchuk", "Arthur Wong", - "artpaul", - "Artur", "Artur Beglaryan", "Artur Filatenkov", + "Artur", "AsiaKorushkina", - "asiana21", - "atereh", "Atri Sharma", - "avasiliev", - "avogar", "Avogar", - "avsharapov", - "awesomeleo", "Azat Khuzhin", + "BSD_Conqueror", "Babacar Diassé", "Bakhtiyor Ruziev", "BanyRule", "Baudouin Giard", "BayoNet", - "bbkas", - "benamazing", - "benbiti", "Benjamin Naecker", "Bertrand Junqua", - "bgranvea", "Bharat Nallan", - "bharatnc", "Big Elephant", "Bill", "BiteTheDDDDt", "BlahGeek", - "blazerer", - "bluebirddm", - "bobrovskij artemij", - "Bogdan", "Bogdan Voronin", + "Bogdan", "BohuTANG", "Bolinov", "BoloniniD", - "booknouse", "Boris Granveaud", "Bowen Masco", - "bo zeng", "Braulio Valdivielso", "Brett Hoerner", - "BSD_Conqueror", - "bseng", "Bulat Gaifullin", "Carbyn", - "caspian", "Caspian", - "cekc", - "centos7", - "champtar", - "chang.chen", - "changvvb", "Chao Ma", "Chao Wang", - "chasingegg", - "chengy8934", - "chenjian", - "chenqi", - "chenxing-xc", - "chenxing.xc", "Chen Yufei", - "chertus", "Chienlung Cheung", - "chou.fan", "Christian", - "christophe.kalenzaga", "Ciprian Hacman", "Clement Rodriguez", - "Clément Rodriguez", "ClickHouse Admin", - "cn-ds", + "Clément Rodriguez", "Cody Baker", "Colum", - "comunodi", "Constantin S. Pan", - "coraxster", + "Constantine Peresypkin", "CurtizJ", - "damozhaeva", + "DIAOZHAFENG", "Daniel Bershatsky", "Daniel Dao", "Daniel Qin", "Danila Kutenin", - "dankondr", "Dao Minh Thuc", - "daoready", "Daria Mozhaeva", "Dario", - "Darío", "DarkWanderer", - "dasmfm", - "davydovska", - "decaseal", + "Darío", "Denis Burlaka", "Denis Glazachev", "Denis Krivak", "Denis Zhuravlev", "Denny Crane", - "dependabot[bot]", - "dependabot-preview[bot]", "Derek Perkins", - "detailyang", - "dfenelonov", - "dgrr", - "DIAOZHAFENG", - "dimarub2000", "Ding Xiang Fei", - "dinosaur", - "divanorama", - "dkxiaohei", - "dmi-feo", "Dmitriev Mikhail", - "dmitrii", "Dmitrii Kovalkov", "Dmitrii Raev", - "dmitriiut", - "Dmitriy", + "Dmitriy Dorofeev", "Dmitriy Lushnikov", - "Dmitry", + "Dmitriy", "Dmitry Belyavtsev", "Dmitry Bilunov", "Dmitry Galuza", "Dmitry Krylov", - "dmitry kuzmin", "Dmitry Luhtionov", "Dmitry Moskowski", "Dmitry Muzyka", @@ -302,148 +217,89 @@ const char * auto_contributors[] { "Dmitry Rubashkin", "Dmitry S..ky / skype: dvska-at-skype", "Dmitry Ukolov", + "Dmitry", "Doge", "Dongdong Yang", "DoomzD", "Dr. Strange Looker", "DuckSoft", - "d.v.semenov", - "eaxdev", - "eejoin", - "egatov", "Egor O'Sten", "Egor Savin", "Ekaterina", - "elBroom", "Eldar Zaitov", "Elena Baskakova", - "elenaspb2019", - "elevankoff", "Elghazal Ahmed", "Elizaveta Mironyuk", - "emakarov", - "emhlbmc", - "emironyuk", + "Elykov Alexandr", "Emmanuel Donin de Rosière", - "Eric", "Eric Daniel", + "Eric", "Erixonich", - "ermaotech", "Ernest Poletaev", "Eugene Klimov", "Eugene Konkov", "Evgenia Sudarikova", - "Evgeniia Sudarikova", "Evgenii Pravda", + "Evgeniia Sudarikova", "Evgeniy Gatov", "Evgeniy Udodov", "Evgeny Konkov", "Evgeny Markov", - "evtan", + "Evgeny", "Ewout", - "exprmntr", - "ezhaka", - "f1yegor", - "Fabiano Francesconi", + "FArthur-cmd", "Fabian Stäber", + "Fabiano Francesconi", "Fadi Hadzh", "Fan()", - "fancno", - "FArthur-cmd", - "fastio", - "favstovol", "FawnD2", "Federico Ceratto", "FeehanG", - "feihengye", - "felixoid", - "felixxdu", - "feng lv", - "fenglv", - "fessmage", "FgoDt", - "fibersel", "Filatenkov Artur", - "filimonov", - "filipe", "Filipe Caixeta", - "flow", "Flowyi", - "flynn", - "foxxmary", "Francisco Barón", - "frank", - "franklee", + "Frank Chen", "Frank Zhao", - "fredchenbj", "Fruit of Eden", - "Fullstop000", - "fuqi", - "Fuwang Hu", - "fuwhu", "Fu Zhe", - "fuzhe1989", - "fuzzERot", + "Fullstop000", + "Fuwang Hu", "Gagan Arneja", "Gao Qiang", - "g-arslan", "Gary Dotzler", + "George G", "George", "George3d6", - "George G", "Georgy Ginzburg", "Gervasio Varela", - "ggerogery", - "giordyb", "Gleb Kanterov", "Gleb Novikov", "Gleb-Tretyakov", - "glockbender", - "glushkovds", "Gregory", - "Grigory", "Grigory Buteyko", "Grigory Pervakov", + "Grigory", "Guillaume Tassery", - "guoleiyi", "Guo Wei (William)", - "gyuton", "Haavard Kvaalen", "Habibullah Oladepo", "Hamoon", - "hao.he", "Hasitha Kanchana", "Hasnat", - "hchen9", - "hcz", - "heng zhao", - "hermano", - "hexiaoting", - "hhell", "Hiroaki Nakamura", - "hotid", - "huangzhaowei", "HuFuwang", "Hui Wang", - "hustnn", - "huzhichengdd", - "ice1x", - "idfer", - "igomac", - "igor", - "Igor", "Igor Hatarist", - "igor.lapko", "Igor Mineev", "Igor Strykhar", - "Igr", + "Igor", "Igr Mineev", - "ikarishinjieva", + "Igr", "Ikko Ashimine", - "ikopylov", "Ildar Musin", "Ildus Kurbangaliev", - "Ilya", "Ilya Breev", "Ilya Golshtein", "Ilya Khomutov", @@ -454,78 +310,56 @@ const char * auto_contributors[] { "Ilya Shipitsin", "Ilya Skrypitsa", "Ilya Yatsishin", + "Ilya", "ImgBotApp", - "imgbot[bot]", - "ip", - "Islam Israfilov", "Islam Israfilov (Islam93)", - "it1804", - "Ivan", + "Islam Israfilov", "Ivan A. Torgashov", "Ivan Babrou", "Ivan Blinkov", "Ivan He", - "ivan-kush", "Ivan Kush", "Ivan Kushnarenko", "Ivan Lezhankin", "Ivan Milov", "Ivan Remen", "Ivan Starkov", - "ivanzhukov", "Ivan Zhukov", - "ivoleg", + "Ivan", "Jack Song", "JackyWoo", "Jacob Hayes", - "jakalletti", "Jakub Kuklis", "JaosnHsieh", - "jasine", - "Jason", "Jason Keirstead", - "jasperzhu", - "javartisan", - "javi", - "javi santana", + "Jason", "Javi Santana", "Javi santana bot", "Jean Baptiste Favre", "Jeffrey Dang", - "jennyma", - "jetgm", "Jiading Guo", "Jiang Tao", - "jianmei zhang", - "jkuklis", - "João Figueiredo", "Jochen Schalanda", - "John", "John Hummel", "John Skopis", + "John", "Jonatas Freitas", + "João Figueiredo", "Julian Zhou", - "jyz0309", "Kang Liu", "Karl Pietrzak", - "karnevil13", - "keenwolf", "Keiji Yoshida", "Ken Chen", "Ken MacInnis", "Kevin Chiang", "Kevin Michel", - "kevin wan", "Kiran", "Kirill Danshin", "Kirill Ershov", - "kirillikoff", "Kirill Malev", "Kirill Shvakov", - "kmeaw", "Koblikov Mihail", "KochetovNicolai", - "kolsys", "Konstantin Grabar", "Konstantin Ilchenko", "Konstantin Lebedev", @@ -534,73 +368,36 @@ const char * auto_contributors[] { "Konstantin Rudenskii", "Korenevskiy Denis", "Korviakov Andrey", - "koshachy", "Kostiantyn Storozhuk", "Kozlov Ivan", - "kreuzerkrieg", "Kruglov Pavel", - "ks1322", "Kseniia Sumarokova", - "kshvakov", - "kssenii", "Ky Li", - "l", - "l1tsolaiki", - "lalex", + "LB", "Latysheva Alexandra", - "laurieliyang", - "lehasm", - "Léo Ercolanelli", "Leonardo Cecchi", "Leopold Schabel", - "leozhang", "Lev Borodin", - "levie", - "levushkin aleksej", - "levysh", "Lewinma", - "lhuang0928", - "lhuang09287750", - "liang.huang", - "liangqian", - "libenwang", - "lichengxiang", - "linceyou", - "listar", - "litao91", - "liu-bov", "Liu Cong", "LiuCong", - "liuyangkuan", "LiuYangkuan", - "liuyimin", - "liyang", - "lomberts", - "long2ice", "Lopatin Konstantin", "Loud_Scream", - "lthaooo", - "ltybc-coder", - "luc1ph3r", "Lucid Dreams", "Luis Bosque", - "lulichao", "Lv Feng", + "Léo Ercolanelli", "M0r64n", - "madianjun", "MagiaGroz", - "maiha", - "Maksim", + "Maks Skorokhod", "Maksim Fedotov", "Maksim Kita", - "Maks Skorokhod", - "malkfilipp", + "Maksim", "Malte", - "manmitya", - "maqroll", "Marat IDRISOV", - "Marek Vavruša", "Marek Vavrusa", + "Marek Vavruša", "Marek Vavruša", "Mariano Benítez Mulet", "Mark Andreev", @@ -609,21 +406,18 @@ const char * auto_contributors[] { "Maroun Maroun", "Marquitos", "Marsel Arduanov", - "Martijn Bakker", "Marti Raudsepp", + "Martijn Bakker", "Marvin Taschenberger", "Masha", - "mastertheknife", "Matthew Peveler", "Matwey V. Kornilov", - "Mátyás Jani", - "Max", "Max Akhmedov", "Max Bruce", - "maxim", + "Max Vetrov", + "Max", + "MaxWk", "Maxim Akhmedov", - "MaximAL", - "maxim-babenko", "Maxim Babenko", "Maxim Fedotov", "Maxim Fridental", @@ -634,141 +428,100 @@ const char * auto_contributors[] { "Maxim Serebryakov", "Maxim Smirnov", "Maxim Ulanovskiy", - "maxkuzn", - "maxulan", - "Max Vetrov", - "MaxWk", + "MaximAL", "Mc.Spring", - "mehanizm", "MeiK", - "melin", - "memo", "Memo", - "meo", - "meoww-bot", - "mergify[bot]", "Metehan Çetinkaya", "Metikov Vadim", - "mf5137", - "mfridental", "Michael Furmur", "Michael Kolupaev", "Michael Monashev", "Michael Razuvaev", "Michael Smitasin", "Michal Lisowski", - "michon470", "MicrochipQ", "Miguel Fernández", - "miha-g", "Mihail Fandyushin", - "mikael", "Mikahil Nacharov", - "Mike", "Mike F", "Mike Kot", - "mikepop7", - "Mikhail", + "Mike", "Mikhail Andreev", "Mikhail Cheshkov", "Mikhail Fandyushin", "Mikhail Filimonov", - "Mikhail f. Shiryaev", "Mikhail Gaidamaka", "Mikhail Korotov", "Mikhail Malafeev", "Mikhail Nacharov", "Mikhail Salosin", "Mikhail Surin", + "Mikhail f. Shiryaev", + "Mikhail", "MikuSugar", "Milad Arabi", - "millb", "Misko Lee", - "mnkonkova", - "mo-avatar", "Mohamad Fadhil", "Mohammad Hossein Sekhavat", - "morty", - "moscas", "Mostafa Dahab", "MovElb", "Mr.General", "Murat Kabilov", - "muzzlerator", - "m-ves", - "mwish", "MyroTk", - "myrrc", - "nagorny", + "Mátyás Jani", + "NIKITA MIKHAILOV", "Narek Galstyan", - "nauta", - "nautaa", + "Natasha Murashkina", + "NeZeD [Mac Pro]", "Neeke Gao", - "neng.liu", "Neng Liu", "NengLiu", - "never lee", - "NeZeD [Mac Pro]", - "nicelulu", - "Nickita", "Nickita Taranov", + "Nickita", "Nickolay Yastrebov", - "nickzhwang", - "Nicolae Vartolomei", "Nico Mandery", "Nico Piderman", + "Nicolae Vartolomei", "Nik", "Nikhil Nadig", "Nikhil Raman", "Nikita Lapkov", "Nikita Mikhailov", - "NIKITA MIKHAILOV", "Nikita Mikhalev", - "nikitamikhaylov", "Nikita Mikhaylov", "Nikita Orlov", "Nikita Tikhomirov", "Nikita Vasilev", "Nikolai Kochetov", "Nikolai Sorokin", - "Nikolay", "Nikolay Degterinsky", "Nikolay Kirsh", "Nikolay Semyachkin", "Nikolay Shcheglov", "Nikolay Vasiliev", "Nikolay Volosatov", + "Nikolay", "Niu Zhaojie", - "nonexistence", - "ns-vasilev", - "nvartolomei", - "oandrew", - "objatie_groba", - "ocadaruma", "Odin Hultgren Van Der Horst", - "ogorbacheva", "Okada Haruki", "Oleg Ershov", "Oleg Favstov", "Oleg Komarov", - "olegkv", "Oleg Matrokhin", "Oleg Obleukhov", "Oleg Strokachuk", "Olga Khvostikova", - "olgarev", "Olga Revyakina", "OmarBazaraa", - "Onehr7", "OnePiece", - "orantius", + "Onehr7", "Orivej Desh", "Oskar Wojciski", "OuO", - "palasonicq", + "PHO", "Paramtamtam", "Patrick Zippenfenig", - "Pavel", "Pavel Cheremushkin", "Pavel Kartaviy", "Pavel Kartavyy", @@ -778,87 +531,59 @@ const char * auto_contributors[] { "Pavel Medvedev", "Pavel Patrin", "Pavel Yakunin", + "Pavel", "Pavlo Bashynskiy", "Pawel Rog", - "pawelsz-rb", - "pdv-ru", + "Peignon Melvyn", "Peng Jian", "Persiyanov Dmitriy Andreevich", "Pervakov Grigorii", "Pervakov Grigory", - "peshkurov", - "philip.han", "Philippe Ombredanne", - "PHO", - "pingyu", - "potya", "Potya", "Pradeep Chhetri", - "presto53", - "proller", - "pufit", - "pyos", "Pysaoke", - "qianlixiang", - "qianmoQ", - "quid", "Quid37", - "quoctan132", - "r1j1k", "Rafael David Tinoco", - "rainbowsysu", "Ramazan Polat", - "Raúl Marín", "Ravengg", + "Raúl Marín", "Realist007", - "redclusive", "RedClusive", "RegulusZ", "Reilee", "Reto Kromer", "Ri", - "ritaank", - "robert", + "Rich Raposa", "Robert Hodges", - "robot-clickhouse", - "robot-metrika-test", - "rodrigargar", "Rohit Agarwal", "Romain Neutron", - "roman", "Roman Bug", + "Roman Chyrva", "Roman Lipovsky", "Roman Nikolaev", "Roman Nozdrin", "Roman Peshkurov", "Roman Tsisyk", - "romanzhukov", "Roman Zhukov", - "root", - "roverxu", - "ruct", - "Ruslan", + "Roman", "Ruslan Savchenko", + "Ruslan", "Russ Frank", "Ruzal Ibragimov", - "ryzuo", + "S.M.A. Djawadi", "Sabyanin Maxim", "SaltTan", "Sami Kerola", "Samuel Chou", - "santaux", - "satanson", "Saulius Valatka", - "sdk2", "Sean Haynes", - "Sébastien", - "Sébastien Launay", - "serebrserg", + "Serg Kulakov", + "Serge Rider", "Sergei Bocharov", "Sergei Semin", "Sergei Shtykov", "Sergei Tsetlin (rekub)", - "Serge Rider", "Sergey Demurin", "Sergey Elantsev", "Sergey Fedorov", @@ -872,144 +597,101 @@ const char * auto_contributors[] { "Sergey Zaikin", "Sergi Almacellas Abellana", "Sergi Vladykin", - "Serg Kulakov", - "sev7e0", "SevaCode", - "sevirov", "Seyed Mehrshad Hosseini", - "sfod", - "shangshujie", - "shedx", "Sherry Wang", "Shoh Jahon", "Silviu Caragea", "Simeon Emanuilov", "Simon Liu", "Simon Podlipsky", - "Šimon Podlipský", - "simon-says", "Sina", "Sjoerd Mulder", "Slach", - "S.M.A. Djawadi", "Snow", "Sofia Antipushina", - "songenjie", - "spff", - "spongedc", - "spyros87", "Stanislav Pavlovichev", "Stas Kelvich", "Stas Pavlovichev", - "stavrolia", "Stefan Thies", - "Stepan", "Stepan Herold", - "stepenhu", + "Stepan", "Steve-金勇", "Stig Bakken", "Storozhuk Kostiantyn", "Stupnikov Andrey", - "su-houzhen", - "sundy", - "sundy-li", - "sundyli", "SuperBot", - "svladykin", + "SuperDJY", + "Sébastien Launay", + "Sébastien", "TAC", + "TCeason", "Tagir Kuskarov", - "tai", - "taichong", "Tai White", - "taiyang-li", "Taleh Zaliyev", "Tangaev", - "tao jiang", - "Tatiana", "Tatiana Kirillova", - "tavplubix", - "TCeason", - "Teja", + "Tatiana", "Teja Srivastasa", + "Teja", "Tema Novikov", - "templarzq", "Tentoshka", - "terrylin", "The-Alchemist", - "Thomas Berdy", "Thom O'Connor", - "tianzhou", + "Thomas Berdy", "Tiaonmmn", - "tiger.yan", "Tigran Khudaverdyan", - "tison", + "Timur Magomedov", "TiunovNN", "Tobias Adamson", "Tobias Lins", "Tom Bombadil", - "topvisor", + "Tom Risse", + "Tomáš Hromada", "Tsarkova Anastasia", "TszkitLo40", - "turbo jason", - "tyrionhuang", - "ubuntu", "Ubuntu", "Ubus", "UnamedRus", - "unegare", - "unknown", - "urgordeadbeef", "V", - "Vadim", - "VadimPE", + "VDimir", "Vadim Plakhtinskiy", "Vadim Skipin", "Vadim Volodin", + "Vadim", + "VadimPE", "Val", "Valera Ryaboshapko", - "Vasilyev Nikita", "Vasily Kozhukhovskiy", "Vasily Morozov", "Vasily Nemkov", "Vasily Okunev", "Vasily Vasilkov", - "vdimir", - "VDimir", - "velom", + "Vasilyev Nikita", "Veloman Yunkan", "Veniamin Gvozdikov", "Veselkov Konstantin", - "vesslanjin", - "vgocoder", "Viachaslau Boben", - "vic", - "vicdashkov", - "vicgao", - "Victor", "Victor Tarnavsky", + "Victor", "Viktor Taranenko", - "vinity", "Vitalii S", "Vitaliy Fedorchenko", "Vitaliy Karnienko", "Vitaliy Kozlovskiy", "Vitaliy Lyudvichenko", "Vitaliy Zakaznikov", - "Vitaly", "Vitaly Baranov", "Vitaly Orlov", "Vitaly Samigullin", "Vitaly Stoyan", - "vitstn", - "vivarum", + "Vitaly", "Vivien Maisonneuve", "Vlad Arkhipov", - "Vladimir", "Vladimir Bunchuk", "Vladimir C", "Vladimir Ch", "Vladimir Chebotarev", - "vladimir golovchenko", "Vladimir Golovchenko", "Vladimir Goncharov", "Vladimir Klimontovich", @@ -1017,81 +699,437 @@ const char * auto_contributors[] { "Vladimir Kopysov", "Vladimir Kozbin", "Vladimir Smirnov", + "Vladimir", "Vladislav Rassokhin", "Vladislav Smirnov", "Vojtech Splichal", "Volodymyr Kuznetsov", "Vsevolod Orlov", - "vxider", "Vxider", "Vyacheslav Alipov", - "vzakaznikov", - "wangchao", "Wang Fenjin", "WangZengrui", - "weeds085490", "Weiqing Xu", "William Shallum", "Winter Zhang", - "wzl", "Xianda Ke", "Xiang Zhou", - "xiedeyantu", - "xPoSx", - "Yağızcan Değirmenci", - "yang", + "Y Lu", "Yangkuan Liu", - "yangshuai", "Yatsishin Ilya", - "yeer", + "Yağızcan Değirmenci", "Yegor Andreenko", "Yegor Levankov", - "ygrek", - "yhgcn", - "Yiğit Konur", - "yiguolei", "Yingchun Lai", "Yingfan Chen", + "Yiğit Konur", + "Yohann Jardin", + "Yuntao Wu", + "Yuri Dyachenko", + "Yurii Vlasenko", + "Yuriy Baranov", + "Yuriy Chernyshov", + "Yuriy Korzhenevskiy", + "Yuriy", + "Yury Karpovich", + "Yury Stankevich", + "ZhiYong Wang", + "Zhichang Yu", + "Zhichun Wu", + "Zhipeng", + "Zijie Lu", + "Zoran Pandovski", + "a.palagashvili", + "abdrakhmanov", + "abel-wang", + "abyss7", + "achimbab", + "achulkov2", + "adevyatova", + "ageraab", + "akazz", + "akonyaev", + "akuzm", + "alekseik1", + "alesapin", + "alex-zaitsev", + "alex.lvxin", + "alexander kozhikhov", + "alexey-milovidov", + "alfredlu", + "amesaru", + "amoschen", + "amudong", + "ana-uvarova", + "andrc1901", + "andrei-karpliuk", + "andrewsg", + "anneji", + "anneji-dev", + "annvsh", + "anrodigina", + "antikvist", + "anton", + "ap11", + "aprudaev", + "artpaul", + "asiana21", + "atereh", + "avasiliev", + "avogar", + "avsharapov", + "awesomeleo", + "bbkas", + "benamazing", + "benbiti", + "bgranvea", + "bharatnc", + "blazerer", + "bluebirddm", + "bo zeng", + "bobrovskij artemij", + "booknouse", + "bseng", + "caspian", + "cekc", + "centos7", + "cfcz48", + "cgp", + "champtar", + "chang.chen", + "changvvb", + "chasingegg", + "chengy8934", + "chenjian", + "chenqi", + "chenxing-xc", + "chenxing.xc", + "chertus", + "chou.fan", + "christophe.kalenzaga", + "cms", + "cmsxbc", + "cn-ds", + "comunodi", + "congbaoyangrou", + "coraxster", + "d.v.semenov", + "damozhaeva", + "dankondr", + "daoready", + "dasmfm", + "davydovska", + "decaseal", + "dependabot-preview[bot]", + "dependabot[bot]", + "detailyang", + "dfenelonov", + "dgrr", + "dimarub2000", + "dinosaur", + "divanorama", + "dkxiaohei", + "dmi-feo", + "dmitrii", + "dmitriiut", + "dmitry kuzmin", + "dongyifeng", + "eaxdev", + "eejoin", + "egatov", + "elBroom", + "elenaspb2019", + "elevankoff", + "emakarov", + "emhlbmc", + "emironyuk", + "ermaotech", + "evtan", + "exprmntr", + "ezhaka", + "f1yegor", + "fancno", + "fastio", + "favstovol", + "feihengye", + "felixoid", + "felixxdu", + "feng lv", + "fenglv", + "fessmage", + "fibersel", + "filimonov", + "filipe", + "flow", + "flynn", + "foxxmary", + "frank chen", + "frank", + "franklee", + "fredchenbj", + "freedomDR", + "fuqi", + "fuwhu", + "fuzhe1989", + "fuzzERot", + "g-arslan", + "ggerogery", + "giordyb", + "glockbender", + "glushkovds", + "guoleiyi", + "gyuton", + "hao.he", + "hchen9", + "hcz", + "heng zhao", + "hermano", + "hexiaoting", + "hhell", + "hotid", + "huangzhaowei", + "hustnn", + "huzhichengdd", + "ice1x", + "idfer", + "igomac", + "igor", + "igor.lapko", + "ikarishinjieva", + "ikopylov", + "imgbot[bot]", + "ip", + "it1804", + "ivan-kush", + "ivanzhukov", + "ivoleg", + "jakalletti", + "jasine", + "jasperzhu", + "javartisan", + "javi santana", + "javi", + "jennyma", + "jetgm", + "jianmei zhang", + "jkuklis", + "jus1096", + "jyz0309", + "karnevil13", + "keenwolf", + "kevin wan", + "khamadiev", + "kirillikoff", + "kmeaw", + "kolsys", + "koshachy", + "kreuzerkrieg", + "ks1322", + "kshvakov", + "kssenii", + "l", + "l1tsolaiki", + "lalex", + "laurieliyang", + "lehasm", + "leosunli", + "leozhang", + "levie", + "levushkin aleksej", + "levysh", + "lhuang0928", + "lhuang09287750", + "liang.huang", + "liangqian", + "libenwang", + "lichengxiang", + "linceyou", + "listar", + "litao91", + "liu-bov", + "liuyangkuan", + "liuyimin", + "liyang", + "liyang830", + "lomberts", + "loneylee", + "long2ice", + "lthaooo", + "ltybc-coder", + "luc1ph3r", + "lulichao", + "m-ves", + "madianjun", + "maiha", + "malkfilipp", + "manmitya", + "maqroll", + "mastertheknife", + "maxim", + "maxim-babenko", + "maxkuzn", + "maxulan", + "mehanizm", + "melin", + "memo", + "meo", + "meoww-bot", + "mergify[bot]", + "mf5137", + "mfridental", + "michael1589", + "michon470", + "miha-g", + "mikael", + "mikepop7", + "millb", + "mnkonkova", + "mo-avatar", + "morty", + "moscas", + "msaf1980", + "muzzlerator", + "mwish", + "myrrc", + "nagorny", + "nauta", + "nautaa", + "neng.liu", + "never lee", + "nicelulu", + "nickzhwang", + "nikitamikhaylov", + "nonexistence", + "ns-vasilev", + "nvartolomei", + "oandrew", + "objatie_groba", + "ocadaruma", + "ogorbacheva", + "olegkv", + "olgarev", + "orantius", + "p0ny", + "palasonicq", + "pawelsz-rb", + "pdv-ru", + "peshkurov", + "philip.han", + "pingyu", + "potya", + "presto53", + "proller", + "pufit", + "pyos", + "qianlixiang", + "qianmoQ", + "qieqieplus", + "quid", + "quoctan132", + "r1j1k", + "rainbowsysu", + "redclusive", + "ritaank", + "robert", + "robot-clickhouse", + "robot-metrika-test", + "rodrigargar", + "roman", + "romanzhukov", + "root", + "roverxu", + "ruct", + "ryzuo", + "santaux", + "satanson", + "sdk2", + "serebrserg", + "sev7e0", + "sevirov", + "sfod", + "shangshujie", + "shedx", + "simon-says", + "songenjie", + "spff", + "spongedc", + "spume", + "spyros87", + "stavrolia", + "stepenhu", + "su-houzhen", + "sundy", + "sundy-li", + "sundyli", + "sunlisheng", + "svladykin", + "tai", + "taichong", + "taiyang-li", + "tao jiang", + "tavplubix", + "templarzq", + "terrylin", + "tianzhou", + "tiger.yan", + "tison", + "topvisor", + "turbo jason", + "tyrionhuang", + "ubuntu", + "unegare", + "unknown", + "urgordeadbeef", + "vdimir", + "velom", + "vesslanjin", + "vgocoder", + "vic", + "vicdashkov", + "vicgao", + "vinity", + "vitstn", + "vivarum", + "vladimir golovchenko", + "vxider", + "vzakaznikov", + "wangchao", + "weeds085490", + "wzl", + "xPoSx", + "xiedeyantu", + "yandd", + "yang", + "yangshuai", + "yeer", + "ygrek", + "yhgcn", + "yiguolei", "yingjinghan", "ylchou", - "Y Lu", - "Yohann Jardin", "yonesko", "yuchuansun", "yuefoo", "yulu86", "yuluxu", - "Yuntao Wu", - "Yuri Dyachenko", - "Yurii Vlasenko", - "Yuriy", - "Yuriy Baranov", - "Yuriy Chernyshov", - "Yuriy Korzhenevskiy", - "Yury Karpovich", - "Yury Stankevich", "ywill3", "zamulla", "zhang2014", + "zhanghuajie", "zhanglistar", "zhangshengyu", "zhangxiao018", "zhangxiao871", "zhen ni", - "Zhichang Yu", - "Zhichun Wu", - "Zhipeng", - "ZhiYong Wang", "zhongyuankai", "zhukai", - "Zijie Lu", "zlx19950903", - "Zoran Pandovski", "zvonand", "zvrr", "zvvr", "zxc111", "zzsmdfj", + "Šimon Podlipský", "Артем Стрельцов", "Владислав Тихонов", "Георгий Кондратьев", @@ -1121,4 +1159,5 @@ const char * auto_contributors[] { "靳阳", "黄朝晖", "黄璞", + "박현우", nullptr}; From 45527768b2dcf669ce061f3cb77a00c8e54c32c5 Mon Sep 17 00:00:00 2001 From: vxider Date: Thu, 9 Dec 2021 02:34:37 +0000 Subject: [PATCH 592/609] small update for window view doc --- docs/zh/sql-reference/functions/window-view-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/sql-reference/functions/window-view-functions.md b/docs/zh/sql-reference/functions/window-view-functions.md index 86425bc78e2..a8afac9a85d 100644 --- a/docs/zh/sql-reference/functions/window-view-functions.md +++ b/docs/zh/sql-reference/functions/window-view-functions.md @@ -53,7 +53,7 @@ hop(time_attr, hop_interval, window_interval [, timezone]) **参数** - `time_attr` - [DateTime](../../sql-reference/data-types/datetime.md)类型的时间数据。 -- `hop_interval` - Hop interval in [Interval](../../sql-reference/data-types/special-data-types/interval.md)类型的滑动间隔,需要大于0。 +- `hop_interval` - [Interval](../../sql-reference/data-types/special-data-types/interval.md)类型的滑动间隔,需要大于0。 - `window_interval` - [Interval](../../sql-reference/data-types/special-data-types/interval.md)类型的窗口大小,需要大于0。 - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) 类型的时区(可选参数)。 From b9861c45e1eb6ce5ab4e5f71354884d51ee1fbce Mon Sep 17 00:00:00 2001 From: JackyWoo Date: Thu, 9 Dec 2021 15:17:24 +0800 Subject: [PATCH 593/609] terminate build when linker path not found --- cmake/tools.cmake | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index eb3624f3b3b..69a37304f58 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -91,6 +91,9 @@ endif () if (LINKER_NAME) if (COMPILER_CLANG AND (CMAKE_CXX_COMPILER_VERSION VERSION_GREATER 12.0.0 OR CMAKE_CXX_COMPILER_VERSION VERSION_EQUAL 12.0.0)) find_program (LLD_PATH NAMES ${LINKER_NAME}) + if (NOT LLD_PATH) + message (FATAL_ERROR "Using linker ${LINKER_NAME} but can't find its path.") + endif () set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} --ld-path=${LLD_PATH}") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} --ld-path=${LLD_PATH}") else () From fad3158bf7a50503f88cfc83c5a46e372acfc2f3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 10:55:51 +0300 Subject: [PATCH 594/609] Fix images names in integration tests --- tests/ci/integration_test_check.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 4a60d825687..723e81d63cb 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -25,15 +25,15 @@ from tee_popen import TeePopen DOWNLOAD_RETRIES_COUNT = 5 IMAGES = [ - "yandex/clickhouse-integration-tests-runner", - "yandex/clickhouse-mysql-golang-client", - "yandex/clickhouse-mysql-java-client", - "yandex/clickhouse-mysql-js-client", - "yandex/clickhouse-mysql-php-client", - "yandex/clickhouse-postgresql-java-client", - "yandex/clickhouse-integration-test", - "yandex/clickhouse-kerberos-kdc", - "yandex/clickhouse-integration-helper", + "clickhouse/integration-tests-runner", + "clickhouse/mysql-golang-client", + "clickhouse/mysql-java-client", + "clickhouse/mysql-js-client", + "clickhouse/mysql-php-client", + "clickhouse/postgresql-java-client", + "clickhouse/integration-test", + "clickhouse/kerberos-kdc", + "clickhouse/integration-helper", ] def get_json_params_dict(check_name, pr_info, docker_images): From 7ea7aa8db82ed91d2d4c058cae8f1e49796b0be3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 12:17:03 +0300 Subject: [PATCH 595/609] Fix build check --- tests/ci/build_report_check.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index f807dcc7cc2..3d97a973017 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -121,6 +121,9 @@ if __name__ == "__main__": build_logs += build_logs_url logging.info("Totally got %s results", len(build_results)) + if len(build_results) == 0: + logging.info("No builds, failing check") + sys.exit(1) s3_helper = S3Helper('https://s3.amazonaws.com') From 1df2383d000db19dcd48f601f7eef54775296a61 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 9 Dec 2021 12:17:31 +0300 Subject: [PATCH 596/609] 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 bafbd415e24..11a0396caeb 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -189,7 +189,7 @@ continue --receive_data_timeout_ms=10000 \ --stacktrace \ --query-fuzzer-runs=1000 \ - --testmode + --testmode \ --queries-file $(ls -1 ch/tests/queries/0_stateless/*.sql | sort -R) \ $NEW_TESTS_OPT \ > >(tail -n 100000 > fuzzer.log) \ From dbf50910168191f4e00243647a7ba1e21ab89c6a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 9 Dec 2021 13:39:28 +0300 Subject: [PATCH 597/609] Parallel reading from replicas (#29279) --- docker/test/stateful/run.sh | 8 +- programs/server/config.xml | 32 +++ src/CMakeLists.txt | 2 + src/Client/Connection.cpp | 19 ++ src/Client/Connection.h | 5 + src/Client/HedgedConnections.cpp | 6 +- src/Client/HedgedConnections.h | 11 +- src/Client/IConnections.h | 17 +- src/Client/IServerConnection.h | 7 + src/Client/LocalConnection.cpp | 5 + src/Client/LocalConnection.h | 2 + src/Client/MultiplexedConnections.cpp | 40 ++- src/Client/MultiplexedConnections.h | 6 +- src/Common/PoolBase.h | 1 - src/Core/Protocol.h | 68 ++--- src/Core/ProtocolDefines.h | 6 +- src/Core/Settings.h | 2 + src/IO/WriteHelpers.h | 1 + src/Interpreters/ClientInfo.cpp | 16 ++ src/Interpreters/ClientInfo.h | 5 + src/Interpreters/Cluster.h | 2 + .../ClusterProxy/IStreamFactory.h | 2 + .../ClusterProxy/SelectStreamFactory.cpp | 2 + src/Interpreters/Context.cpp | 16 +- src/Interpreters/Context.h | 12 +- .../QueryPlan/ReadFromMergeTree.cpp | 44 +++- src/Processors/QueryPlan/ReadFromMergeTree.h | 5 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 89 ++++++- src/Processors/QueryPlan/ReadFromRemote.h | 20 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 75 ++++-- src/QueryPipeline/RemoteQueryExecutor.h | 36 ++- src/Server/TCPHandler.cpp | 60 ++++- src/Server/TCPHandler.h | 4 + src/Storages/IStorage.h | 1 + src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/IntersectionsIndexes.h | 237 +++++++++++++++++ src/Storages/MergeTree/MarkRange.cpp | 25 ++ src/Storages/MergeTree/MarkRange.h | 6 + .../MergeTreeBaseSelectProcessor.cpp | 249 +++++++++++++++++- .../MergeTree/MergeTreeBaseSelectProcessor.h | 84 +++++- .../MergeTree/MergeTreeBlockReadUtils.cpp | 5 +- .../MergeTree/MergeTreeBlockReadUtils.h | 4 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 15 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 6 +- .../MergeTreeInOrderSelectProcessor.cpp | 5 +- .../MergeTreeInOrderSelectProcessor.h | 5 +- .../MergeTree/MergeTreeRangeReader.cpp | 5 +- .../MergeTreeReverseSelectProcessor.cpp | 5 +- .../MergeTreeReverseSelectProcessor.h | 6 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 12 +- .../MergeTree/MergeTreeSelectProcessor.h | 9 +- .../MergeTree/MergeTreeSequentialSource.cpp | 2 + .../MergeTreeThreadSelectProcessor.cpp | 71 ++++- .../MergeTreeThreadSelectProcessor.h | 14 +- .../ParallelReplicasReadingCoordinator.cpp | 143 ++++++++++ .../ParallelReplicasReadingCoordinator.h | 20 ++ src/Storages/MergeTree/RequestResponse.cpp | 141 ++++++++++ src/Storages/MergeTree/RequestResponse.h | 57 ++++ .../MergeTree/tests/gtest_coordinator.cpp | 240 +++++++++++++++++ src/Storages/StorageDistributed.cpp | 7 +- src/Storages/StorageMergeTree.cpp | 9 +- src/Storages/StorageReplicatedMergeTree.cpp | 16 +- src/Storages/StorageS3Cluster.cpp | 2 +- .../System/StorageSystemPartsColumns.cpp | 2 +- .../StorageSystemProjectionPartsColumns.cpp | 2 +- ...4_shard_distributed_with_many_replicas.sql | 1 - .../01870_modulo_partition_key.sql | 8 +- ...el_processing_on_replicas_part_1.reference | 110 ++++++++ ..._parallel_processing_on_replicas_part_1.sh | 98 +++++++ 71 files changed, 2085 insertions(+), 179 deletions(-) create mode 100644 src/Storages/MergeTree/IntersectionsIndexes.h create mode 100644 src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp create mode 100644 src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h create mode 100644 src/Storages/MergeTree/RequestResponse.cpp create mode 100644 src/Storages/MergeTree/RequestResponse.h create mode 100644 src/Storages/MergeTree/tests/gtest_coordinator.cpp create mode 100644 tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.reference create mode 100755 tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 680392df43e..f8dee0f8bc9 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -61,6 +61,7 @@ chmod 777 -R /var/lib/clickhouse clickhouse-client --query "SHOW DATABASES" clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" + service clickhouse-server restart # Wait for server to start accepting connections @@ -109,8 +110,13 @@ function run_tests() fi set +e - clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + clickhouse-test --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \ + --skip 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt + + clickhouse-test --timeout 1200 --testname --shard --zookeeper --check-zookeeper-session --no-stateless --hung-check --print-time \ + 00168_parallel_processing_on_replicas "${ADDITIONAL_OPTIONS[@]}" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee -a test_output/test_result.txt + set -e } diff --git a/programs/server/config.xml b/programs/server/config.xml index 37f36aa5215..9a2a6d7729f 100644 --- a/programs/server/config.xml +++ b/programs/server/config.xml @@ -650,6 +650,38 @@ + + + false + + 127.0.0.1 + 9000 + + + 127.0.0.2 + 9000 + + + 127.0.0.3 + 9000 + + + + diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5f4ebaaa895..5e4a16cfda7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -517,6 +517,8 @@ if (USE_BZIP2) target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${BZIP2_INCLUDE_DIR}) endif() +dbms_target_link_libraries(PUBLIC consistent-hashing) + include ("${ClickHouse_SOURCE_DIR}/cmake/add_check.cmake") if (ENABLE_TESTS AND USE_GTEST) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index ca10160fa88..505a6514812 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -603,6 +603,14 @@ void Connection::sendReadTaskResponse(const String & response) out->next(); } + +void Connection::sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) +{ + writeVarUInt(Protocol::Client::MergeTreeReadTaskResponse, *out); + response.serialize(*out); + out->next(); +} + void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name) { /// NOTE 'Throttler' is not used in this method (could use, but it's not important right now). @@ -872,6 +880,10 @@ Packet Connection::receivePacket() case Protocol::Server::ReadTaskRequest: return res; + case Protocol::Server::MergeTreeReadTaskRequest: + res.request = receivePartitionReadRequest(); + return res; + case Protocol::Server::ProfileEvents: res.block = receiveProfileEvents(); return res; @@ -1023,6 +1035,13 @@ ProfileInfo Connection::receiveProfileInfo() const return profile_info; } +PartitionReadRequest Connection::receivePartitionReadRequest() const +{ + PartitionReadRequest request; + request.deserialize(*in); + return request; +} + void Connection::throwUnexpectedPacket(UInt64 packet_type, const char * expected) const { diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 3b49760ba10..2ea5334bbd3 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -16,6 +16,8 @@ #include +#include + #include #include @@ -104,6 +106,8 @@ public: void sendData(const Block & block, const String & name/* = "" */, bool scalar/* = false */) override; + void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) override; + void sendExternalTablesData(ExternalTablesData & data) override; bool poll(size_t timeout_microseconds/* = 0 */) override; @@ -255,6 +259,7 @@ private: std::vector receiveMultistringMessage(UInt64 msg_type) const; std::unique_ptr receiveException() const; Progress receiveProgress() const; + PartitionReadRequest receivePartitionReadRequest() const; ProfileInfo receiveProfileInfo() const; void initInputBuffers(); diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index c73bea53d10..791ac4c1ef1 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -132,7 +132,7 @@ void HedgedConnections::sendQuery( const String & query, const String & query_id, UInt64 stage, - const ClientInfo & client_info, + ClientInfo & client_info, bool with_pending_data) { std::lock_guard lock(cancel_mutex); @@ -171,7 +171,9 @@ void HedgedConnections::sendQuery( modified_settings.group_by_two_level_threshold_bytes = 0; } - if (offset_states.size() > 1) + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + + if (offset_states.size() > 1 && enable_sample_offset_parallel_processing) { modified_settings.parallel_replicas_count = offset_states.size(); modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset; diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index e39d9582cde..d64f7ea4286 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -86,7 +86,7 @@ public: const String & query, const String & query_id, UInt64 stage, - const ClientInfo & client_info, + ClientInfo & client_info, bool with_pending_data) override; void sendReadTaskResponse(const String &) override @@ -94,6 +94,11 @@ public: throw Exception("sendReadTaskResponse in not supported with HedgedConnections", ErrorCodes::LOGICAL_ERROR); } + void sendMergeTreeReadTaskResponse(PartitionReadResponse) override + { + throw Exception("sendMergeTreeReadTaskResponse in not supported with HedgedConnections", ErrorCodes::LOGICAL_ERROR); + } + Packet receivePacket() override; Packet receivePacketUnlocked(AsyncCallback async_callback, bool is_draining) override; @@ -112,6 +117,8 @@ public: bool hasActiveConnections() const override { return active_connection_count > 0; } + void setReplicaInfo(ReplicaInfo value) override { replica_info = value; } + private: /// If we don't receive data from replica and there is no progress in query /// execution for receive_data_timeout, we are trying to get new @@ -199,6 +206,8 @@ private: bool sent_query = false; bool cancelled = false; + ReplicaInfo replica_info; + mutable std::mutex cancel_mutex; }; diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index 53267cbbb3e..8dbd58c9598 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -1,6 +1,9 @@ #pragma once +#include + #include +#include namespace DB { @@ -27,10 +30,11 @@ public: const String & query, const String & query_id, UInt64 stage, - const ClientInfo & client_info, + ClientInfo & client_info, bool with_pending_data) = 0; virtual void sendReadTaskResponse(const String &) = 0; + virtual void sendMergeTreeReadTaskResponse(PartitionReadResponse response) = 0; /// Get packet from any replica. virtual Packet receivePacket() = 0; @@ -56,6 +60,17 @@ public: /// Get the replica addresses as a string. virtual std::string dumpAddresses() const = 0; + + struct ReplicaInfo + { + size_t all_replicas_count{0}; + size_t number_of_current_replica{0}; + }; + + /// This is needed in max_parallel_replicas case. + /// We create a RemoteQueryExecutor for each replica + virtual void setReplicaInfo(ReplicaInfo value) = 0; + /// Returns the number of replicas. virtual size_t size() const = 0; diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 9d6b54ef32f..b7c6ae314e2 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -12,6 +12,8 @@ #include #include +#include + #include @@ -32,10 +34,13 @@ struct Packet Progress progress; ProfileInfo profile_info; std::vector part_uuids; + PartitionReadRequest request; + PartitionReadResponse response; Packet() : type(Protocol::Server::Hello) {} }; + /// Struct which represents data we are going to send for external table. struct ExternalTableData { @@ -96,6 +101,8 @@ public: /// Send all contents of external (temporary) tables. virtual void sendExternalTablesData(ExternalTablesData & data) = 0; + virtual void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) = 0; + /// Check, if has data to read. virtual bool poll(size_t timeout_microseconds) = 0; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index 4f476b57c27..9eaa9ce883a 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -424,6 +424,11 @@ void LocalConnection::sendExternalTablesData(ExternalTablesData &) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); } +void LocalConnection::sendMergeTreeReadTaskResponse(const PartitionReadResponse &) +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Not implemented"); +} + ServerConnectionPtr LocalConnection::createConnection(const ConnectionParameters &, ContextPtr current_context, bool send_progress) { return std::make_unique(current_context, send_progress); diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index 1cc23defa6e..fbd054506e7 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -92,6 +92,8 @@ public: void sendExternalTablesData(ExternalTablesData &) override; + void sendMergeTreeReadTaskResponse(const PartitionReadResponse & response) override; + bool poll(size_t timeout_microseconds/* = 0 */) override; bool hasReadPendingData() const override; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index a27f7709555..c3000443a9c 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -1,9 +1,10 @@ #include + +#include +#include #include #include -#include -#include "Core/Protocol.h" - +#include namespace DB { @@ -110,7 +111,7 @@ void MultiplexedConnections::sendQuery( const String & query, const String & query_id, UInt64 stage, - const ClientInfo & client_info, + ClientInfo & client_info, bool with_pending_data) { std::lock_guard lock(cancel_mutex); @@ -131,16 +132,29 @@ void MultiplexedConnections::sendQuery( modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold_bytes = 0; } + + if (settings.allow_experimental_parallel_reading_from_replicas) + { + client_info.collaborate_with_initiator = true; + client_info.count_participating_replicas = replica_info.all_replicas_count; + client_info.number_of_current_replica = replica_info.number_of_current_replica; + } } + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + size_t num_replicas = replica_states.size(); if (num_replicas > 1) { - /// Use multiple replicas for parallel query processing. - modified_settings.parallel_replicas_count = num_replicas; + if (enable_sample_offset_parallel_processing) + /// Use multiple replicas for parallel query processing. + modified_settings.parallel_replicas_count = num_replicas; + for (size_t i = 0; i < num_replicas; ++i) { - modified_settings.parallel_replica_offset = i; + if (enable_sample_offset_parallel_processing) + modified_settings.parallel_replica_offset = i; + replica_states[i].connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); } @@ -179,6 +193,16 @@ void MultiplexedConnections::sendReadTaskResponse(const String & response) current_connection->sendReadTaskResponse(response); } + +void MultiplexedConnections::sendMergeTreeReadTaskResponse(PartitionReadResponse response) +{ + std::lock_guard lock(cancel_mutex); + if (cancelled) + return; + current_connection->sendMergeTreeReadTaskResponse(response); +} + + Packet MultiplexedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); @@ -234,6 +258,7 @@ Packet MultiplexedConnections::drain() switch (packet.type) { + case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: case Protocol::Server::PartUUIDs: case Protocol::Server::Data: @@ -313,6 +338,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac switch (packet.type) { + case Protocol::Server::MergeTreeReadTaskRequest: case Protocol::Server::ReadTaskRequest: case Protocol::Server::PartUUIDs: case Protocol::Server::Data: diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 4fb7d496b0c..e76d54218c7 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -38,10 +38,11 @@ public: const String & query, const String & query_id, UInt64 stage, - const ClientInfo & client_info, + ClientInfo & client_info, bool with_pending_data) override; void sendReadTaskResponse(const String &) override; + void sendMergeTreeReadTaskResponse(PartitionReadResponse response) override; Packet receivePacket() override; @@ -62,6 +63,7 @@ public: /// Without locking, because sendCancel() does not change the state of the replicas. bool hasActiveConnections() const override { return active_connection_count > 0; } + void setReplicaInfo(ReplicaInfo value) override { replica_info = value; } private: Packet receivePacketUnlocked(AsyncCallback async_callback, bool is_draining) override; @@ -102,6 +104,8 @@ private: bool sent_query = false; bool cancelled = false; + ReplicaInfo replica_info; + /// A mutex for the sendCancel function to execute safely /// in separate thread. mutable std::mutex cancel_mutex; diff --git a/src/Common/PoolBase.h b/src/Common/PoolBase.h index 3f7f340c5d1..85d4e84abca 100644 --- a/src/Common/PoolBase.h +++ b/src/Common/PoolBase.h @@ -163,4 +163,3 @@ protected: /** Creates a new object to put into the pool. */ virtual ObjectPtr allocObject() = 0; }; - diff --git a/src/Core/Protocol.h b/src/Core/Protocol.h index fb18e1135a5..08c675eb421 100644 --- a/src/Core/Protocol.h +++ b/src/Core/Protocol.h @@ -64,24 +64,26 @@ namespace Protocol { enum Enum { - Hello = 0, /// Name, version, revision. - Data = 1, /// A block of data (compressed or not). - Exception = 2, /// The exception during query execution. - Progress = 3, /// Query execution progress: rows read, bytes read. - Pong = 4, /// Ping response - EndOfStream = 5, /// All packets were transmitted - ProfileInfo = 6, /// Packet with profiling info. - Totals = 7, /// A block with totals (compressed or not). - Extremes = 8, /// A block with minimums and maximums (compressed or not). - TablesStatusResponse = 9, /// A response to TablesStatus request. - Log = 10, /// System logs of the query execution - TableColumns = 11, /// Columns' description for default values calculation - PartUUIDs = 12, /// List of unique parts ids. - ReadTaskRequest = 13, /// String (UUID) describes a request for which next task is needed - /// This is such an inverted logic, where server sends requests - /// And client returns back response - ProfileEvents = 14, /// Packet with profile events from server. - MAX = ProfileEvents, + Hello = 0, /// Name, version, revision. + Data = 1, /// A block of data (compressed or not). + Exception = 2, /// The exception during query execution. + Progress = 3, /// Query execution progress: rows read, bytes read. + Pong = 4, /// Ping response + EndOfStream = 5, /// All packets were transmitted + ProfileInfo = 6, /// Packet with profiling info. + Totals = 7, /// A block with totals (compressed or not). + Extremes = 8, /// A block with minimums and maximums (compressed or not). + TablesStatusResponse = 9, /// A response to TablesStatus request. + Log = 10, /// System logs of the query execution + TableColumns = 11, /// Columns' description for default values calculation + PartUUIDs = 12, /// List of unique parts ids. + ReadTaskRequest = 13, /// String (UUID) describes a request for which next task is needed + /// This is such an inverted logic, where server sends requests + /// And client returns back response + ProfileEvents = 14, /// Packet with profile events from server. + MergeTreeReadTaskRequest = 15, /// Request from a MergeTree replica to a coordinator + MAX = MergeTreeReadTaskRequest, + }; /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 @@ -106,6 +108,7 @@ namespace Protocol "PartUUIDs", "ReadTaskRequest", "ProfileEvents", + "MergeTreeReadTaskRequest", }; return packet <= MAX ? data[packet] @@ -130,20 +133,20 @@ namespace Protocol { enum Enum { - Hello = 0, /// Name, version, revision, default DB - Query = 1, /// Query id, query settings, stage up to which the query must be executed, - /// whether the compression must be used, - /// query text (without data for INSERTs). - Data = 2, /// A block of data (compressed or not). - Cancel = 3, /// Cancel the query execution. - Ping = 4, /// Check that connection to the server is alive. - TablesStatusRequest = 5, /// Check status of tables on the server. - KeepAlive = 6, /// Keep the connection alive - Scalar = 7, /// A block of data (compressed or not). - IgnoredPartUUIDs = 8, /// List of unique parts ids to exclude from query processing - ReadTaskResponse = 9, /// TODO: - - MAX = ReadTaskResponse, + Hello = 0, /// Name, version, revision, default DB + Query = 1, /// Query id, query settings, stage up to which the query must be executed, + /// whether the compression must be used, + /// query text (without data for INSERTs). + Data = 2, /// A block of data (compressed or not). + Cancel = 3, /// Cancel the query execution. + Ping = 4, /// Check that connection to the server is alive. + TablesStatusRequest = 5, /// Check status of tables on the server. + KeepAlive = 6, /// Keep the connection alive + Scalar = 7, /// A block of data (compressed or not). + IgnoredPartUUIDs = 8, /// List of unique parts ids to exclude from query processing + ReadTaskResponse = 9, /// A filename to read from s3 (used in s3Cluster) + MergeTreeReadTaskResponse = 10, /// Coordinator's decision with a modified set of mark ranges allowed to read + MAX = MergeTreeReadTaskResponse, }; inline const char * toString(UInt64 packet) @@ -159,6 +162,7 @@ namespace Protocol "Scalar", "IgnoredPartUUIDs", "ReadTaskResponse", + "MergeTreeReadTaskResponse" }; return packet <= MAX ? data[packet] diff --git a/src/Core/ProtocolDefines.h b/src/Core/ProtocolDefines.h index ac0fba384b8..36820788b91 100644 --- a/src/Core/ProtocolDefines.h +++ b/src/Core/ProtocolDefines.h @@ -31,6 +31,9 @@ #define DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION 1 +#define DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION 1 +#define DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS 54453 + /// Minimum revision supporting interserver secret. #define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441 @@ -48,6 +51,7 @@ /// NOTE: DBMS_TCP_PROTOCOL_VERSION has nothing common with VERSION_REVISION, /// later is just a number for server version (one number instead of commit SHA) /// for simplicity (sometimes it may be more convenient in some use cases). -#define DBMS_TCP_PROTOCOL_VERSION 54452 + +#define DBMS_TCP_PROTOCOL_VERSION 54453 #define DBMS_MIN_PROTOCOL_VERSION_WITH_INITIAL_QUERY_START_TIME 54449 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4e0e50cc521..47b01655c26 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -126,6 +126,8 @@ class IColumn; M(UInt64, parallel_replicas_count, 0, "", 0) \ M(UInt64, parallel_replica_offset, 0, "", 0) \ \ + M(Bool, allow_experimental_parallel_reading_from_replicas, false, "If true, ClickHouse will send a SELECT query to all replicas of a table. It will work for any kind on MergeTree table.", 0) \ + \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ M(UInt64, parallel_distributed_insert_select, 0, "Process distributed INSERT SELECT query in the same cluster on local tables on every shard, if 1 SELECT is executed on each shard, if 2 SELECT and INSERT is executed on each shard", 0) \ diff --git a/src/IO/WriteHelpers.h b/src/IO/WriteHelpers.h index 9cdc77df957..5498e1c90f3 100644 --- a/src/IO/WriteHelpers.h +++ b/src/IO/WriteHelpers.h @@ -118,6 +118,7 @@ inline void writeStringBinary(const std::string_view & s, WriteBuffer & buf) writeStringBinary(StringRef{s}, buf); } + template void writeVectorBinary(const std::vector & v, WriteBuffer & buf) { diff --git a/src/Interpreters/ClientInfo.cpp b/src/Interpreters/ClientInfo.cpp index 79956aaebed..827e7d27409 100644 --- a/src/Interpreters/ClientInfo.cpp +++ b/src/Interpreters/ClientInfo.cpp @@ -89,6 +89,13 @@ void ClientInfo::write(WriteBuffer & out, const UInt64 server_protocol_revision) writeBinary(uint8_t(0), out); } } + + if (server_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS) + { + writeVarUInt(static_cast(collaborate_with_initiator), out); + writeVarUInt(count_participating_replicas, out); + writeVarUInt(number_of_current_replica, out); + } } @@ -170,6 +177,15 @@ void ClientInfo::read(ReadBuffer & in, const UInt64 client_protocol_revision) readBinary(client_trace_context.trace_flags, in); } } + + if (client_protocol_revision >= DBMS_MIN_REVISION_WITH_PARALLEL_REPLICAS) + { + UInt64 value; + readVarUInt(value, in); + collaborate_with_initiator = static_cast(value); + readVarUInt(count_participating_replicas, in); + readVarUInt(number_of_current_replica, in); + } } diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index d42c34f07e2..3ce740c6436 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -108,6 +108,11 @@ public: bool is_replicated_database_internal = false; + /// For parallel processing on replicas + bool collaborate_with_initiator{false}; + UInt64 count_participating_replicas{0}; + UInt64 number_of_current_replica{0}; + bool empty() const { return query_kind == QueryKind::NO_QUERY; } /** Serialization and deserialization. diff --git a/src/Interpreters/Cluster.h b/src/Interpreters/Cluster.h index ec78abf574c..a64e17264b1 100644 --- a/src/Interpreters/Cluster.h +++ b/src/Interpreters/Cluster.h @@ -184,6 +184,8 @@ public: bool isLocal() const { return !local_addresses.empty(); } bool hasRemoteConnections() const { return local_addresses.size() != per_replica_pools.size(); } size_t getLocalNodeCount() const { return local_addresses.size(); } + size_t getRemoteNodeCount() const { return per_replica_pools.size() - local_addresses.size(); } + size_t getAllNodeCount() const { return per_replica_pools.size(); } bool hasInternalReplication() const { return has_internal_replication; } /// Name of directory for asynchronous write to StorageDistributed if has_internal_replication const std::string & insertPathForInternalReplication(bool prefer_localhost_replica, bool use_compact_format) const; diff --git a/src/Interpreters/ClusterProxy/IStreamFactory.h b/src/Interpreters/ClusterProxy/IStreamFactory.h index 6360aee2f55..483ce9dcab9 100644 --- a/src/Interpreters/ClusterProxy/IStreamFactory.h +++ b/src/Interpreters/ClusterProxy/IStreamFactory.h @@ -37,7 +37,9 @@ public: Block header; size_t shard_num = 0; + size_t num_replicas = 0; ConnectionPoolWithFailoverPtr pool; + ConnectionPoolPtrs per_replica_pools; /// If we connect to replicas lazily. /// (When there is a local replica with big delay). diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index b644f2936d9..a47874c475a 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -117,7 +117,9 @@ void SelectStreamFactory::createForShard( .query = modified_query_ast, .header = header, .shard_num = shard_info.shard_num, + .num_replicas = shard_info.getAllNodeCount(), .pool = shard_info.pool, + .per_replica_pools = shard_info.per_replica_pools, .lazy = lazy, .local_delay = local_delay, }); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9b2721cd15d..db1d6a37877 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -2962,7 +2962,7 @@ PartUUIDsPtr Context::getPartUUIDs() const ReadTaskCallback Context::getReadTaskCallback() const { if (!next_task_callback.has_value()) - throw Exception(fmt::format("Next task callback is not set for query {}", getInitialQueryId()), ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Next task callback is not set for query {}", getInitialQueryId()); return next_task_callback.value(); } @@ -2972,6 +2972,20 @@ void Context::setReadTaskCallback(ReadTaskCallback && callback) next_task_callback = callback; } + +MergeTreeReadTaskCallback Context::getMergeTreeReadTaskCallback() const +{ + if (!merge_tree_read_task_callback.has_value()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Next task callback for is not set for query {}", getInitialQueryId()); + + return merge_tree_read_task_callback.value(); +} + +void Context::setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback) +{ + merge_tree_read_task_callback = callback; +} + PartUUIDsPtr Context::getIgnoredPartUUIDs() const { auto lock = getLock(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 5948cc7f7a7..823bc028c15 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -14,6 +14,7 @@ #include #include #include +#include #include "config_core.h" @@ -148,6 +149,8 @@ using InputBlocksReader = std::function; /// Used in distributed task processing using ReadTaskCallback = std::function; +using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; + /// An empty interface for an arbitrary object that may be attached by a shared pointer /// to query context, when using ClickHouse as a library. struct IHostContext @@ -216,8 +219,12 @@ private: Scalars scalars; Scalars local_scalars; - /// Fields for distributed s3 function + /// Used in s3Cluster table function. With this callback, a worker node could ask an initiator + /// about next file to read from s3. std::optional next_task_callback; + /// Used in parallel reading from replicas. A replica tells about its intentions to read + /// some ranges from some part and initiator will tell the replica about whether it is accepted or denied. + std::optional merge_tree_read_task_callback; /// Record entities accessed by current query, and store this information in system.query_log. struct QueryAccessInfo @@ -865,6 +872,9 @@ public: ReadTaskCallback getReadTaskCallback() const; void setReadTaskCallback(ReadTaskCallback && callback); + MergeTreeReadTaskCallback getMergeTreeReadTaskCallback() const; + void setMergeTreeReadTaskCallback(MergeTreeReadTaskCallback && callback); + /// Background executors related methods void initializeBackgroundExecutorsIfNeeded(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 67ca6d3d8e0..eddbbb9138c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -74,7 +74,8 @@ ReadFromMergeTree::ReadFromMergeTree( bool sample_factor_column_queried_, std::shared_ptr max_block_numbers_to_read_, Poco::Logger * log_, - MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_) + MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_, + bool enable_parallel_reading) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( metadata_snapshot_->getSampleBlockForColumns(real_column_names_, data_.getVirtuals(), data_.getStorageID()), getPrewhereInfo(query_info_), @@ -107,6 +108,9 @@ ReadFromMergeTree::ReadFromMergeTree( auto type = std::make_shared(); output_stream->header.insert({type->createColumn(), type, "_sample_factor"}); } + + if (enable_parallel_reading) + read_task_callback = context->getMergeTreeReadTaskCallback(); } Pipe ReadFromMergeTree::readFromPool( @@ -127,6 +131,7 @@ Pipe ReadFromMergeTree::readFromPool( } const auto & settings = context->getSettingsRef(); + const auto & client_info = context->getClientInfo(); MergeTreeReadPool::BackoffSettings backoff_settings(settings); auto pool = std::make_shared( @@ -147,17 +152,30 @@ Pipe ReadFromMergeTree::readFromPool( for (size_t i = 0; i < max_streams; ++i) { + std::optional extension; + if (read_task_callback) + { + extension = ParallelReadingExtension + { + .callback = read_task_callback.value(), + .count_participating_replicas = client_info.count_participating_replicas, + .number_of_current_replica = client_info.number_of_current_replica, + .colums_to_read = required_columns + }; + } + auto source = std::make_shared( i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, metadata_snapshot, use_uncompressed_cache, - prewhere_info, actions_settings, reader_settings, virt_column_names); + prewhere_info, actions_settings, reader_settings, virt_column_names, std::move(extension)); - if (i == 0) - { - /// Set the approximate number of rows for the first source only + /// Set the approximate number of rows for the first source only + /// In case of parallel processing on replicas do not set approximate rows at all. + /// Because the value will be identical on every replicas and will be accounted + /// multiple times (settings.max_parallel_replicas times more) + if (i == 0 && !client_info.collaborate_with_initiator) source->addTotalRowsApprox(total_rows); - } pipes.emplace_back(std::move(source)); } @@ -172,10 +190,22 @@ ProcessorPtr ReadFromMergeTree::createSource( bool use_uncompressed_cache, bool has_limit_below_one_block) { + const auto & client_info = context->getClientInfo(); + std::optional extension; + if (read_task_callback) + { + extension = ParallelReadingExtension + { + .callback = read_task_callback.value(), + .count_participating_replicas = client_info.count_participating_replicas, + .number_of_current_replica = client_info.number_of_current_replica, + .colums_to_read = required_columns + }; + } return std::make_shared( data, metadata_snapshot, part.data_part, max_block_size, preferred_block_size_bytes, preferred_max_column_in_block_size_bytes, required_columns, part.ranges, use_uncompressed_cache, prewhere_info, - actions_settings, reader_settings, virt_column_names, part.part_index_in_query, has_limit_below_one_block); + actions_settings, reader_settings, virt_column_names, part.part_index_in_query, has_limit_below_one_block, std::move(extension)); } Pipe ReadFromMergeTree::readInOrder( diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 46b62467ae0..0bdfa66bcc7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -97,7 +97,8 @@ public: bool sample_factor_column_queried_, std::shared_ptr max_block_numbers_to_read_, Poco::Logger * log_, - MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_ + MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr_, + bool enable_parallel_reading ); String getName() const override { return "ReadFromMergeTree"; } @@ -184,6 +185,8 @@ private: MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts) const; ReadFromMergeTree::AnalysisResult getAnalysisResult() const; MergeTreeDataSelectAnalysisResultPtr analyzed_result_ptr; + + std::optional read_task_callback; }; struct MergeTreeDataSelectAnalysisResult diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 399e7d01839..8fcec03d746 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -12,6 +12,8 @@ #include #include #include +#include +#include namespace DB { @@ -112,7 +114,10 @@ ReadFromRemote::ReadFromRemote( { } -void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard) +void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, + std::shared_ptr coordinator, + std::shared_ptr pool, + std::optional replica_info) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -125,7 +130,10 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto } auto lazily_create_stream = [ - pool = shard.pool, shard_num = shard.shard_num, shard_count = shard_count, query = shard.query, header = shard.header, + replica_info = replica_info, + pool = pool ? pool : shard.pool, + coordinator = coordinator, + shard_num = shard.shard_num, shard_count = shard_count, query = shard.query, header = shard.header, context = context, throttler = throttler, main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables, @@ -161,9 +169,12 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto max_remote_delay = std::max(try_result.staleness, max_remote_delay); } - if (try_results.empty() || local_delay < max_remote_delay) + /// We disable this branch in case of parallel reading from replicas, because createLocalPlan will call + /// InterpreterSelectQuery directly and it will be too ugly to pass ParallelReplicasCoordinator or some callback there. + if (!context->getClientInfo().collaborate_with_initiator && (try_results.empty() || local_delay < max_remote_delay)) { auto plan = createLocalPlan(query, header, context, stage, shard_num, shard_count); + return QueryPipelineBuilder::getPipe(std::move(*plan->buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context)))); @@ -180,7 +191,8 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto scalars["_shard_num"] = Block{{DataTypeUInt32().createColumnConst(1, shard_num), std::make_shared(), "_shard_num"}}; auto remote_query_executor = std::make_shared( - pool, std::move(connections), query_string, header, context, throttler, scalars, external_tables, stage); + pool, std::move(connections), query_string, header, context, throttler, scalars, external_tables, stage, + RemoteQueryExecutor::Extension{.parallel_reading_coordinator = std::move(coordinator), .replica_info = replica_info}); return createRemoteSourcePipe(remote_query_executor, add_agg_info, add_totals, add_extremes, async_read); } @@ -191,7 +203,10 @@ void ReadFromRemote::addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFacto addConvertingActions(pipes.back(), output_stream->header); } -void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard) +void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, + std::shared_ptr coordinator, + std::shared_ptr pool, + std::optional replica_info) { bool add_agg_info = stage == QueryProcessingStage::WithMergeableState; bool add_totals = false; @@ -207,11 +222,20 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory:: scalars["_shard_num"] = Block{{DataTypeUInt32().createColumnConst(1, shard.shard_num), std::make_shared(), "_shard_num"}}; - auto remote_query_executor = std::make_shared( - shard.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage); + + std::shared_ptr remote_query_executor; + + remote_query_executor = std::make_shared( + pool ? pool : shard.pool, query_string, shard.header, context, throttler, scalars, external_tables, stage, + RemoteQueryExecutor::Extension{.parallel_reading_coordinator = std::move(coordinator), .replica_info = std::move(replica_info)}); + remote_query_executor->setLogger(log); - remote_query_executor->setPoolMode(PoolMode::GET_MANY); + /// In case of parallel reading from replicas we have a connection pool per replica. + /// Setting PoolMode will make no sense. + if (!pool) + remote_query_executor->setPoolMode(PoolMode::GET_MANY); + if (!table_func_ptr) remote_query_executor->setMainTable(main_table); @@ -223,12 +247,51 @@ void ReadFromRemote::addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory:: void ReadFromRemote::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { Pipes pipes; - for (const auto & shard : shards) + + const auto & settings = context->getSettingsRef(); + const bool enable_sample_offset_parallel_processing = settings.max_parallel_replicas > 1 && !settings.allow_experimental_parallel_reading_from_replicas; + + /// We have to create a pipe for each replica + /// FIXME: The second condition is only for tests to work, because hedged connections enabled by default. + if (settings.max_parallel_replicas > 1 && !enable_sample_offset_parallel_processing && !context->getSettingsRef().use_hedged_requests) { - if (shard.lazy) - addLazyPipe(pipes, shard); - else - addPipe(pipes, shard); + const Settings & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + + for (const auto & shard : shards) + { + auto coordinator = std::make_shared(); + + for (size_t replica_num = 0; replica_num < shard.num_replicas; ++replica_num) + { + IConnections::ReplicaInfo replica_info + { + .all_replicas_count = shard.num_replicas, + .number_of_current_replica = replica_num + }; + + auto pool = shard.per_replica_pools[replica_num]; + auto pool_with_failover = std::make_shared( + ConnectionPoolPtrs{pool}, current_settings.load_balancing); + + if (shard.lazy) + addLazyPipe(pipes, shard, coordinator, pool_with_failover, replica_info); + else + addPipe(pipes, shard, coordinator, pool_with_failover, replica_info); + } + } + } + else + { + for (const auto & shard : shards) + { + auto coordinator = std::make_shared(); + + if (shard.lazy) + addLazyPipe(pipes, shard, /*coordinator=*/nullptr, /*pool*/{}, /*replica_info*/std::nullopt); + else + addPipe(pipes, shard, /*coordinator=*/nullptr, /*pool*/{}, /*replica_info*/std::nullopt); + } } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index f963164dd3f..f361be93b5a 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -1,9 +1,11 @@ #pragma once #include #include +#include #include #include #include +#include namespace DB { @@ -37,6 +39,12 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; private: + enum class Mode + { + PerReplica, + PerShard + }; + ClusterProxy::IStreamFactory::Shards shards; QueryProcessingStage::Enum stage; @@ -52,8 +60,16 @@ private: Poco::Logger * log; UInt32 shard_count; - void addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard); - void addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard); + void addLazyPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, + std::shared_ptr coordinator, + std::shared_ptr pool, + std::optional replica_info); + void addPipe(Pipes & pipes, const ClusterProxy::IStreamFactory::Shard & shard, + std::shared_ptr coordinator, + std::shared_ptr pool, + std::optional replica_info); + + void addPipeForReplica(); }; } diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index ada16a1f201..653d9a2bbf8 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -7,6 +7,7 @@ #include #include #include "Core/Protocol.h" +#include "IO/ReadHelpers.h" #include #include #include @@ -20,6 +21,7 @@ #include #include #include +#include namespace CurrentMetrics @@ -42,21 +44,26 @@ namespace ErrorCodes RemoteQueryExecutor::RemoteQueryExecutor( const String & query_, const Block & header_, ContextPtr context_, const Scalars & scalars_, const Tables & external_tables_, - QueryProcessingStage::Enum stage_, std::shared_ptr task_iterator_) + QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_), scalars(scalars_) - , external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_) + , external_tables(external_tables_), stage(stage_) + , task_iterator(extension_ ? extension_->task_iterator : nullptr) + , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) {} RemoteQueryExecutor::RemoteQueryExecutor( Connection & connection, const String & query_, const Block & header_, ContextPtr context_, ThrottlerPtr throttler, const Scalars & scalars_, const Tables & external_tables_, - QueryProcessingStage::Enum stage_, std::shared_ptr task_iterator_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, task_iterator_) + QueryProcessingStage::Enum stage_, std::optional extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) { - create_connections = [this, &connection, throttler]() + create_connections = [this, &connection, throttler, extension_]() { - return std::make_shared(connection, context->getSettingsRef(), throttler); + auto res = std::make_shared(connection, context->getSettingsRef(), throttler); + if (extension_ && extension_->replica_info) + res->setReplicaInfo(*extension_->replica_info); + return res; }; } @@ -64,12 +71,15 @@ RemoteQueryExecutor::RemoteQueryExecutor( std::shared_ptr connection_ptr, const String & query_, const Block & header_, ContextPtr context_, ThrottlerPtr throttler, const Scalars & scalars_, const Tables & external_tables_, - QueryProcessingStage::Enum stage_, std::shared_ptr task_iterator_) - : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, task_iterator_) + QueryProcessingStage::Enum stage_, std::optional extension_) + : RemoteQueryExecutor(query_, header_, context_, scalars_, external_tables_, stage_, extension_) { - create_connections = [this, connection_ptr, throttler]() + create_connections = [this, connection_ptr, throttler, extension_]() { - return std::make_shared(connection_ptr, context->getSettingsRef(), throttler); + auto res = std::make_shared(connection_ptr, context->getSettingsRef(), throttler); + if (extension_ && extension_->replica_info) + res->setReplicaInfo(*extension_->replica_info); + return res; }; } @@ -78,12 +88,18 @@ RemoteQueryExecutor::RemoteQueryExecutor( std::vector && connections_, const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, - QueryProcessingStage::Enum stage_, std::shared_ptr task_iterator_) + QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_) - , scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_), pool(pool_) + , scalars(scalars_), external_tables(external_tables_), stage(stage_) + , task_iterator(extension_ ? extension_->task_iterator : nullptr) + , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , pool(pool_) { - create_connections = [this, connections_, throttler]() mutable { - return std::make_shared(std::move(connections_), context->getSettingsRef(), throttler); + create_connections = [this, connections_, throttler, extension_]() mutable { + auto res = std::make_shared(std::move(connections_), context->getSettingsRef(), throttler); + if (extension_ && extension_->replica_info) + res->setReplicaInfo(*extension_->replica_info); + return res; }; } @@ -91,11 +107,14 @@ RemoteQueryExecutor::RemoteQueryExecutor( const ConnectionPoolWithFailoverPtr & pool_, const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, - QueryProcessingStage::Enum stage_, std::shared_ptr task_iterator_) + QueryProcessingStage::Enum stage_, std::optional extension_) : header(header_), query(query_), context(context_) - , scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_), pool(pool_) + , scalars(scalars_), external_tables(external_tables_), stage(stage_) + , task_iterator(extension_ ? extension_->task_iterator : nullptr) + , parallel_reading_coordinator(extension_ ? extension_->parallel_reading_coordinator : nullptr) + , pool(pool_) { - create_connections = [this, throttler]()->std::shared_ptr + create_connections = [this, throttler, extension_]()->std::shared_ptr { const Settings & current_settings = context->getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); @@ -107,7 +126,10 @@ RemoteQueryExecutor::RemoteQueryExecutor( if (main_table) table_to_check = std::make_shared(main_table.getQualifiedName()); - return std::make_shared(pool, context, timeouts, throttler, pool_mode, table_to_check); + auto res = std::make_shared(pool, context, timeouts, throttler, pool_mode, table_to_check); + if (extension_ && extension_->replica_info) + res->setReplicaInfo(*extension_->replica_info); + return res; } #endif @@ -122,7 +144,10 @@ RemoteQueryExecutor::RemoteQueryExecutor( else connection_entries = pool->getMany(timeouts, ¤t_settings, pool_mode); - return std::make_shared(std::move(connection_entries), current_settings, throttler); + auto res = std::make_shared(std::move(connection_entries), current_settings, throttler); + if (extension_ && extension_->replica_info) + res->setReplicaInfo(*extension_->replica_info); + return res; }; } @@ -344,6 +369,9 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) { switch (packet.type) { + case Protocol::Server::MergeTreeReadTaskRequest: + processMergeTreeReadTaskRequest(packet.request); + break; case Protocol::Server::ReadTaskRequest: processReadTaskRequest(); break; @@ -440,6 +468,15 @@ void RemoteQueryExecutor::processReadTaskRequest() connections->sendReadTaskResponse(response); } +void RemoteQueryExecutor::processMergeTreeReadTaskRequest(PartitionReadRequest request) +{ + if (!parallel_reading_coordinator) + throw Exception("Coordinator for parallel reading from replicas is not initialized", ErrorCodes::LOGICAL_ERROR); + + auto response = parallel_reading_coordinator->handleRequest(std::move(request)); + connections->sendMergeTreeReadTaskResponse(response); +} + void RemoteQueryExecutor::finish(std::unique_ptr * read_context) { /** If one of: diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index b7a2509ea97..d5603fd2281 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -7,7 +9,7 @@ #include #include #include -#include +#include namespace DB @@ -35,20 +37,33 @@ class RemoteQueryExecutor public: using ReadContext = RemoteQueryExecutorReadContext; + /// We can provide additional logic for RemoteQueryExecutor + /// For example for s3Cluster table function we provide an Iterator over tasks to do. + /// Nodes involved into the query send request for a new task and we answer them using this object. + /// In case of parallel reading from replicas we provide a Coordinator object + /// Every replica will tell us about parts and mark ranges it wants to read and coordinator will + /// decide whether to deny or to accept that request. + struct Extension + { + std::shared_ptr task_iterator{nullptr}; + std::shared_ptr parallel_reading_coordinator; + std::optional replica_info; + }; + /// Takes already set connection. /// We don't own connection, thus we have to drain it synchronously. RemoteQueryExecutor( Connection & connection, const String & query_, const Block & header_, ContextPtr context_, ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr task_iterator_ = {}); + QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::optional extension_ = std::nullopt); /// Takes already set connection. RemoteQueryExecutor( std::shared_ptr connection, const String & query_, const Block & header_, ContextPtr context_, ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr task_iterator_ = {}); + QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::optional extension_ = std::nullopt); /// Accepts several connections already taken from pool. RemoteQueryExecutor( @@ -56,14 +71,14 @@ public: std::vector && connections_, const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr task_iterator_ = {}); + QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::optional extension_ = std::nullopt); /// Takes a pool and gets one or several connections from it. RemoteQueryExecutor( const ConnectionPoolWithFailoverPtr & pool, const String & query_, const Block & header_, ContextPtr context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), - QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr task_iterator_ = {}); + QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::optional extension_ = std::nullopt); ~RemoteQueryExecutor(); @@ -115,7 +130,7 @@ private: RemoteQueryExecutor( const String & query_, const Block & header_, ContextPtr context_, const Scalars & scalars_, const Tables & external_tables_, - QueryProcessingStage::Enum stage_, std::shared_ptr task_iterator_); + QueryProcessingStage::Enum stage_, std::optional extension_); Block header; Block totals; @@ -136,6 +151,13 @@ private: /// Initiator identifier for distributed task processing std::shared_ptr task_iterator; + std::shared_ptr parallel_reading_coordinator; + + /// This is needed only for parallel reading from replicas, because + /// we create a RemoteQueryExecutor per replica and have to store additional info + /// about the number of the current replica or the count of replicas at all. + IConnections::ReplicaInfo replica_info; + std::function()> create_connections; /// Hold a shared reference to the connection pool so that asynchronous connection draining will /// work safely. Make sure it's the first member so that we don't destruct it too early. @@ -203,6 +225,8 @@ private: void processReadTaskRequest(); + void processMergeTreeReadTaskRequest(PartitionReadRequest request); + /// Cancell query and restart it with info about duplicated UUIDs /// only for `allow_experimental_query_deduplication`. std::variant restartQueryWithoutDuplicatedUUIDs(std::unique_ptr * read_context = nullptr); diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 32154054cc0..cdf1838c06b 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -310,10 +310,25 @@ void TCPHandler::runImpl() query_context->setReadTaskCallback([this]() -> String { std::lock_guard lock(task_callback_mutex); + + if (state.is_cancelled) + return {}; + sendReadTaskRequestAssumeLocked(); return receiveReadTaskResponseAssumeLocked(); }); + query_context->setMergeTreeReadTaskCallback([this](PartitionReadRequest request) -> std::optional + { + std::lock_guard lock(task_callback_mutex); + + if (state.is_cancelled) + return std::nullopt; + + sendMergeTreeReadTaskRequstAssumeLocked(std::move(request)); + return receivePartitionMergeTreeReadTaskResponseAssumeLocked(); + }); + /// Processing Query state.io = executeQuery(state.query, query_context, false, state.stage); @@ -663,10 +678,13 @@ void TCPHandler::processOrdinaryQueryWithProcessors() Block block; while (executor.pull(block, interactive_delay / 1000)) { - std::lock_guard lock(task_callback_mutex); + std::unique_lock lock(task_callback_mutex); if (isQueryCancelled()) { + /// Several callback like callback for parallel reading could be called from inside the pipeline + /// and we have to unlock the mutex from our side to prevent deadlock. + lock.unlock(); /// A packet was received requesting to stop execution of the request. executor.cancel(); break; @@ -786,6 +804,15 @@ void TCPHandler::sendReadTaskRequestAssumeLocked() out->next(); } + +void TCPHandler::sendMergeTreeReadTaskRequstAssumeLocked(PartitionReadRequest request) +{ + writeVarUInt(Protocol::Server::MergeTreeReadTaskRequest, *out); + request.serialize(*out); + out->next(); +} + + void TCPHandler::sendProfileInfo(const ProfileInfo & info) { writeVarUInt(Protocol::Server::ProfileInfo, *out); @@ -1297,6 +1324,35 @@ String TCPHandler::receiveReadTaskResponseAssumeLocked() } +std::optional TCPHandler::receivePartitionMergeTreeReadTaskResponseAssumeLocked() +{ + UInt64 packet_type = 0; + readVarUInt(packet_type, *in); + if (packet_type != Protocol::Client::MergeTreeReadTaskResponse) + { + if (packet_type == Protocol::Client::Cancel) + { + state.is_cancelled = true; + /// For testing connection collector. + if (sleep_in_receive_cancel.totalMilliseconds()) + { + std::chrono::milliseconds ms(sleep_in_receive_cancel.totalMilliseconds()); + std::this_thread::sleep_for(ms); + } + return std::nullopt; + } + else + { + throw Exception(fmt::format("Received {} packet after requesting read task", + Protocol::Client::toString(packet_type)), ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT); + } + } + PartitionReadResponse response; + response.deserialize(*in); + return response; +} + + void TCPHandler::receiveClusterNameAndSalt() { readStringBinary(cluster, *in); @@ -1697,7 +1753,7 @@ bool TCPHandler::isQueryCancelled() return true; default: - throw NetException("Unknown packet from client", ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); + throw NetException("Unknown packet from client " + toString(packet_type), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT); } } diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 3cf3346cd72..4a340e328ed 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -15,6 +15,8 @@ #include #include +#include + #include "IServer.h" #include "base/types.h" @@ -201,6 +203,7 @@ private: void receiveQuery(); void receiveIgnoredPartUUIDs(); String receiveReadTaskResponseAssumeLocked(); + std::optional receivePartitionMergeTreeReadTaskResponseAssumeLocked(); bool receiveData(bool scalar); bool readDataNext(); void readData(); @@ -233,6 +236,7 @@ private: void sendEndOfStream(); void sendPartUUIDs(); void sendReadTaskRequestAssumeLocked(); + void sendMergeTreeReadTaskRequstAssumeLocked(PartitionReadRequest request); void sendProfileInfo(const ProfileInfo & info); void sendTotals(const Block & totals); void sendExtremes(const Block & extremes); diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index cf7b075a204..8432e5c48d1 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -20,6 +20,7 @@ #include #include +#include namespace DB diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 1255bf975e6..21dbedbb6ac 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -546,7 +546,7 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(const StorageM if (!hasColumnFiles(column)) continue; - const auto size = getColumnSize(column_name, *column_type).data_compressed; + const auto size = getColumnSize(column_name).data_compressed; if (size < minimum_size) { minimum_size = size; @@ -747,7 +747,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const for (const auto & part_column : columns) { /// It was compressed with default codec and it's not empty - auto column_size = getColumnSize(part_column.name, *part_column.type); + auto column_size = getColumnSize(part_column.name); if (column_size.data_compressed != 0 && !storage_columns.hasCompressionCodec(part_column.name)) { auto serialization = IDataType::getSerialization(part_column, @@ -885,7 +885,7 @@ void IMergeTreeDataPart::loadRowsCount() /// Most trivial types if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes()) { - auto size = getColumnSize(column.name, *column.type); + auto size = getColumnSize(column.name); if (size.data_uncompressed == 0) continue; @@ -933,7 +933,7 @@ void IMergeTreeDataPart::loadRowsCount() if (!column_col->isFixedAndContiguous() || column_col->lowCardinality()) continue; - size_t column_size = getColumnSize(column.name, *column.type).data_uncompressed; + size_t column_size = getColumnSize(column.name).data_uncompressed; if (!column_size) continue; @@ -1490,7 +1490,7 @@ void IMergeTreeDataPart::calculateSecondaryIndicesSizesOnDisk() } } -ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name, const IDataType & /* type */) const +ColumnSize IMergeTreeDataPart::getColumnSize(const String & column_name) const { /// For some types of parts columns_size maybe not calculated auto it = columns_sizes.find(column_name); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 1467b0ef03f..3515da20fa9 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -103,7 +103,7 @@ public: /// NOTE: Returns zeros if column files are not found in checksums. /// Otherwise return information about column size on disk. - ColumnSize getColumnSize(const String & column_name, const IDataType & /* type */) const; + ColumnSize getColumnSize(const String & column_name) const; /// NOTE: Returns zeros if secondary indexes are not found in checksums. /// Otherwise return information about secondary index size on disk. diff --git a/src/Storages/MergeTree/IntersectionsIndexes.h b/src/Storages/MergeTree/IntersectionsIndexes.h new file mode 100644 index 00000000000..68ccbc4a0b1 --- /dev/null +++ b/src/Storages/MergeTree/IntersectionsIndexes.h @@ -0,0 +1,237 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +/// A boundary of a segment (left or right) +struct PartToRead +{ + PartBlockRange range; + struct PartAndProjectionNames + { + String part; + String projection; + bool operator<(const PartAndProjectionNames & rhs) const + { + if (part == rhs.part) + return projection < rhs.projection; + return part < rhs.part; + } + bool operator==(const PartAndProjectionNames & rhs) const + { + return part == rhs.part && projection == rhs.projection; + } + }; + + PartAndProjectionNames name; + + bool operator==(const PartToRead & rhs) const + { + return range == rhs.range && name == rhs.name; + } + + bool operator<(const PartToRead & rhs) const + { + /// We allow only consecutive non-intersecting ranges + const bool intersection = + (range.begin <= rhs.range.begin && rhs.range.begin < range.end) || + (rhs.range.begin <= range.begin && range.begin <= rhs.range.end); + if (intersection) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Got intersecting parts. First [{}, {}]. Second [{}, {}]", + range.begin, range.end, rhs.range.begin, rhs.range.end); + return range.begin < rhs.range.begin && range.end <= rhs.range.begin; + } +}; + +/// MergeTreeDataPart is described as a segment (min block and max block) +/// During request handling we have to know how many intersection +/// current part has with already saved parts in our state. +struct PartSegments +{ + enum class IntersectionResult + { + NO_INTERSECTION, + EXACTLY_ONE_INTERSECTION, + REJECT + }; + + void addPart(PartToRead part) { segments.insert(std::move(part)); } + + IntersectionResult getIntersectionResult(PartToRead part) + { + bool intersected_before = false; + for (const auto & segment: segments) + { + auto are_intersect = [](auto & x, auto & y) + { + /// <= is important here, because we are working with segments [a, b] + if ((x.begin <= y.begin) && (y.begin <= x.end)) + return true; + if ((y.begin <= x.begin) && (x.begin <= y.end)) + return true; + return false; + }; + + if (are_intersect(segment.range, part.range)) + { + /// We have two or possibly more intersections + if (intersected_before) + return IntersectionResult::REJECT; + + /// We have intersection with part with different name + /// or with different min or max block + /// It could happens if we have merged part on one replica + /// but not on another. + if (segment != part) + return IntersectionResult::REJECT; + + /// We allow only the intersection with the same part as we have + intersected_before = true; + } + } + + return intersected_before ? IntersectionResult::EXACTLY_ONE_INTERSECTION : IntersectionResult::NO_INTERSECTION; + } + + using OrderedSegments = std::set; + OrderedSegments segments; +}; + +/// This is used only in parallel reading from replicas +/// This struct is an ordered set of half intervals and it is responsible for +/// giving an inversion of that intervals (e.g. [a, b) => {[-inf, a), [b, +inf)}) +/// or giving an intersection of two sets of intervals +/// This is needed, because MarkRange is actually a half-opened interval +/// and during the query execution we receive some kind of request from every replica +/// to read some ranges from a specific part. +/// We have to avoid the situation, where some range is read twice. +/// This struct helps us to do it using only two operations (intersection and inversion) +/// over a set of half opened intervals. +struct HalfIntervals +{ + static HalfIntervals initializeWithEntireSpace() + { + auto left_inf = std::numeric_limits::min(); + auto right_inf = std::numeric_limits::max(); + return HalfIntervals{{{left_inf, right_inf}}}; + } + + static HalfIntervals initializeFromMarkRanges(MarkRanges ranges) + { + OrderedRanges new_intervals; + for (const auto & range : ranges) + new_intervals.insert(range); + + return HalfIntervals{std::move(new_intervals)}; + } + + MarkRanges convertToMarkRangesFinal() + { + MarkRanges result; + std::move(intervals.begin(), intervals.end(), std::back_inserter(result)); + return result; + } + + HalfIntervals & intersect(const HalfIntervals & rhs) + { + /** + * first [ ) [ ) [ ) [ ) [ ) + * second [ ) [ ) [ ) [ ) + */ + OrderedRanges intersected; + + const auto & first_intervals = intervals; + auto first = first_intervals.begin(); + const auto & second_intervals = rhs.intervals; + auto second = second_intervals.begin(); + + while (first != first_intervals.end() && second != second_intervals.end()) + { + auto curr_intersection = MarkRange{ + std::max(second->begin, first->begin), + std::min(second->end, first->end) + }; + + /// Insert only if segments are intersect + if (curr_intersection.begin < curr_intersection.end) + intersected.insert(std::move(curr_intersection)); + + if (first->end <= second->end) + ++first; + else + ++second; + } + + std::swap(intersected, intervals); + + return *this; + } + + HalfIntervals & negate() + { + auto left_inf = std::numeric_limits::min(); + auto right_inf = std::numeric_limits::max(); + + if (intervals.empty()) + { + intervals.insert(MarkRange{left_inf, right_inf}); + return *this; + } + + OrderedRanges new_ranges; + + /// Possibly add (-inf; begin) + if (auto begin = intervals.begin()->begin; begin != left_inf) + new_ranges.insert(MarkRange{left_inf, begin}); + + auto prev = intervals.begin(); + for (auto it = std::next(intervals.begin()); it != intervals.end(); ++it) + { + if (prev->end != it->begin) + new_ranges.insert(MarkRange{prev->end, it->begin}); + prev = it; + } + + /// Try to add (end; +inf) + if (auto end = intervals.rbegin()->end; end != right_inf) + new_ranges.insert(MarkRange{end, right_inf}); + + std::swap(new_ranges, intervals); + + return *this; + } + + bool operator==(const HalfIntervals & rhs) const + { + return intervals == rhs.intervals; + } + + using OrderedRanges = std::set; + OrderedRanges intervals; +}; + + +[[ maybe_unused ]] static std::ostream & operator<< (std::ostream & out, const HalfIntervals & ranges) +{ + for (const auto & range: ranges.intervals) + out << fmt::format("({}, {}) ", range.begin, range.end); + return out; +} + +/// This is needed for tests where we don't need to modify objects +[[ maybe_unused ]] static HalfIntervals getIntersection(const HalfIntervals & first, const HalfIntervals & second) +{ + auto result = first; + result.intersect(second); + return result; +} + +} diff --git a/src/Storages/MergeTree/MarkRange.cpp b/src/Storages/MergeTree/MarkRange.cpp index 7f097cd7106..343c4ecaf22 100644 --- a/src/Storages/MergeTree/MarkRange.cpp +++ b/src/Storages/MergeTree/MarkRange.cpp @@ -3,6 +3,31 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +bool MarkRange::operator==(const MarkRange & rhs) const +{ + return begin == rhs.begin && end == rhs.end; +} + +bool MarkRange::operator<(const MarkRange & rhs) const +{ + /// We allow only consecutive non-intersecting ranges + /// Here we check whether a beginning of one range lies inside another range + /// (ranges are intersect) + const bool is_intersection = (begin <= rhs.begin && rhs.begin < end) || + (rhs.begin <= begin && begin < rhs.end); + + if (is_intersection) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Intersecting mark ranges are not allowed, it is a bug! First range ({}, {}), second range ({}, {})", begin, end, rhs.begin, rhs.end); + + return begin < rhs.begin && end <= rhs.begin; +} + size_t getLastMark(const MarkRanges & ranges) { size_t current_task_last_mark = 0; diff --git a/src/Storages/MergeTree/MarkRange.h b/src/Storages/MergeTree/MarkRange.h index b46913db30c..4f32be6ab14 100644 --- a/src/Storages/MergeTree/MarkRange.h +++ b/src/Storages/MergeTree/MarkRange.h @@ -2,7 +2,9 @@ #include #include +#include +#include namespace DB { @@ -18,6 +20,10 @@ struct MarkRange MarkRange() = default; MarkRange(const size_t begin_, const size_t end_) : begin{begin_}, end{end_} {} + + bool operator==(const MarkRange & rhs) const; + + bool operator<(const MarkRange & rhs) const; }; using MarkRanges = std::deque; diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 2f46543b03c..fbc818a7de9 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -12,6 +13,8 @@ #include +#include + namespace DB { @@ -33,7 +36,8 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( UInt64 preferred_max_column_in_block_size_bytes_, const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, - const Names & virt_column_names_) + const Names & virt_column_names_, + std::optional extension_) : SourceWithProgress(transformHeader(std::move(header), prewhere_info_, storage_.getPartitionValueType(), virt_column_names_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) @@ -45,6 +49,7 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( , use_uncompressed_cache(use_uncompressed_cache_) , virt_column_names(virt_column_names_) , partition_value_type(storage.getPartitionValueType()) + , extension(extension_) { header_without_virtual_columns = getPort().getHeader(); @@ -71,6 +76,91 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( } +bool MergeTreeBaseSelectProcessor::getNewTask() +{ + /// No parallel reading feature + if (!extension.has_value()) + { + if (getNewTaskImpl()) + { + finalizeNewTask(); + return true; + } + return false; + } + return getNewTaskParallelReading(); +} + + +bool MergeTreeBaseSelectProcessor::getNewTaskParallelReading() +{ + if (getTaskFromBuffer()) + return true; + + if (no_more_tasks) + return getDelayedTasks(); + + while (true) + { + /// The end of execution. No task. + if (!getNewTaskImpl()) + { + no_more_tasks = true; + return getDelayedTasks(); + } + + splitCurrentTaskRangesAndFillBuffer(); + + if (getTaskFromBuffer()) + return true; + } +} + + +bool MergeTreeBaseSelectProcessor::getTaskFromBuffer() +{ + while (!buffered_ranges.empty()) + { + auto ranges = std::move(buffered_ranges.front()); + buffered_ranges.pop_front(); + + assert(!ranges.empty()); + + auto res = performRequestToCoordinator(ranges, /*delayed=*/false); + + if (Status::Accepted == res) + return true; + + if (Status::Cancelled == res) + break; + } + return false; +} + + +bool MergeTreeBaseSelectProcessor::getDelayedTasks() +{ + while (!delayed_tasks.empty()) + { + task = std::move(delayed_tasks.front()); + delayed_tasks.pop_front(); + + assert(!task->mark_ranges.empty()); + + auto res = performRequestToCoordinator(task->mark_ranges, /*delayed=*/true); + + if (Status::Accepted == res) + return true; + + if (Status::Cancelled == res) + break; + } + + finish(); + return false; +} + + Chunk MergeTreeBaseSelectProcessor::generate() { while (!isCancelled()) @@ -479,6 +569,163 @@ std::unique_ptr MergeTreeBaseSelectProcessor::getSi data_part, Names(complete_column_names.begin(), complete_column_names.end()), sample_block); } + +MergeTreeBaseSelectProcessor::Status MergeTreeBaseSelectProcessor::performRequestToCoordinator(MarkRanges requested_ranges, bool delayed) +{ + String partition_id = task->data_part->info.partition_id; + String part_name; + String projection_name; + + if (task->data_part->isProjectionPart()) + { + part_name = task->data_part->getParentPart()->name; + projection_name = task->data_part->name; + } + else + { + part_name = task->data_part->name; + projection_name = ""; + } + + PartBlockRange block_range + { + .begin = task->data_part->info.min_block, + .end = task->data_part->info.max_block + }; + + PartitionReadRequest request + { + .partition_id = std::move(partition_id), + .part_name = std::move(part_name), + .projection_name = std::move(projection_name), + .block_range = std::move(block_range), + .mark_ranges = std::move(requested_ranges) + }; + + /// Constistent hashing won't work with reading in order, because at the end of the execution + /// we could possibly seek back + if (!delayed && canUseConsistentHashingForParallelReading()) + { + const auto hash = request.getConsistentHash(extension->count_participating_replicas); + if (hash != extension->number_of_current_replica) + { + auto delayed_task = std::make_unique(*task); // Create a copy + delayed_task->mark_ranges = std::move(request.mark_ranges); + delayed_tasks.emplace_back(std::move(delayed_task)); + return Status::Denied; + } + } + + auto optional_response = extension.value().callback(std::move(request)); + + if (!optional_response.has_value()) + return Status::Cancelled; + + auto response = optional_response.value(); + + task->mark_ranges = std::move(response.mark_ranges); + + if (response.denied || task->mark_ranges.empty()) + return Status::Denied; + + finalizeNewTask(); + + return Status::Accepted; +} + + +size_t MergeTreeBaseSelectProcessor::estimateMaxBatchSizeForHugeRanges() +{ + /// This is an empirical number and it is so, + /// because we have an adaptive granularity by default. + const size_t average_granule_size_bytes = 8UL * 1024 * 1024 * 10; // 10 MiB + + /// We want to have one RTT per one gigabyte of data read from disk + /// this could be configurable. + const size_t max_size_for_one_request = 8UL * 1024 * 1024 * 1024; // 1 GiB + + size_t sum_average_marks_size = 0; + /// getColumnSize is not fully implemented for compact parts + if (task->data_part->getType() == IMergeTreeDataPart::Type::COMPACT) + { + sum_average_marks_size = average_granule_size_bytes; + } + else + { + for (const auto & name : extension->colums_to_read) + { + auto size = task->data_part->getColumnSize(name); + + assert(size.marks != 0); + sum_average_marks_size += size.data_uncompressed / size.marks; + } + } + + if (sum_average_marks_size == 0) + sum_average_marks_size = average_granule_size_bytes; // 10 MiB + + LOG_TEST(log, "Reading from {} part, average mark size is {}", + task->data_part->getTypeName(), sum_average_marks_size); + + return max_size_for_one_request / sum_average_marks_size; +} + +void MergeTreeBaseSelectProcessor::splitCurrentTaskRangesAndFillBuffer() +{ + const size_t max_batch_size = estimateMaxBatchSizeForHugeRanges(); + + size_t current_batch_size = 0; + buffered_ranges.emplace_back(); + + for (const auto & range : task->mark_ranges) + { + auto expand_if_needed = [&] + { + if (current_batch_size > max_batch_size) + { + buffered_ranges.emplace_back(); + current_batch_size = 0; + } + }; + + expand_if_needed(); + + if (range.end - range.begin < max_batch_size) + { + buffered_ranges.back().push_back(range); + current_batch_size += range.end - range.begin; + continue; + } + + auto current_begin = range.begin; + auto current_end = range.begin + max_batch_size; + + while (current_end < range.end) + { + auto current_range = MarkRange{current_begin, current_end}; + buffered_ranges.back().push_back(current_range); + current_batch_size += current_end - current_begin; + + current_begin = current_end; + current_end = current_end + max_batch_size; + + expand_if_needed(); + } + + if (range.end - current_begin > 0) + { + auto current_range = MarkRange{current_begin, range.end}; + buffered_ranges.back().push_back(current_range); + current_batch_size += range.end - current_begin; + + expand_if_needed(); + } + } + + if (buffered_ranges.back().empty()) + buffered_ranges.pop_back(); +} + MergeTreeBaseSelectProcessor::~MergeTreeBaseSelectProcessor() = default; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index d102e4f07a4..c462c34aa83 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -15,6 +16,18 @@ class UncompressedCache; class MarkCache; struct PrewhereExprInfo; + +struct ParallelReadingExtension +{ + MergeTreeReadTaskCallback callback; + size_t count_participating_replicas{0}; + size_t number_of_current_replica{0}; + /// This is needed to estimate the number of bytes + /// between a pair of marks to perform one request + /// over the network for a 1Gb of data. + Names colums_to_read; +}; + /// Base class for MergeTreeThreadSelectProcessor and MergeTreeSelectProcessor class MergeTreeBaseSelectProcessor : public SourceWithProgress { @@ -30,7 +43,8 @@ public: UInt64 preferred_max_column_in_block_size_bytes_, const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, - const Names & virt_column_names_ = {}); + const Names & virt_column_names_ = {}, + std::optional extension = {}); ~MergeTreeBaseSelectProcessor() override; @@ -43,10 +57,22 @@ public: const Block & sample_block); protected: + Chunk generate() final; - /// Creates new this->task, and initializes readers. - virtual bool getNewTask() = 0; + /// Creates new this->task and return a flag whether it was successful or not + virtual bool getNewTaskImpl() = 0; + /// Creates new readers for a task it is needed. These methods are separate, because + /// in case of parallel reading from replicas the whole task could be denied by a coodinator + /// or it could modified somehow. + virtual void finalizeNewTask() = 0; + + size_t estimateMaxBatchSizeForHugeRanges(); + + virtual bool canUseConsistentHashingForParallelReading() { return false; } + + /// Closes readers and unlock part locks + virtual void finish() = 0; virtual Chunk readFromPart(); @@ -82,14 +108,62 @@ protected: /// This header is used for chunks from readFromPart(). Block header_without_virtual_columns; - std::unique_ptr task; - std::shared_ptr owned_uncompressed_cache; std::shared_ptr owned_mark_cache; using MergeTreeReaderPtr = std::unique_ptr; MergeTreeReaderPtr reader; MergeTreeReaderPtr pre_reader; + + MergeTreeReadTaskPtr task; + + std::optional extension; + bool no_more_tasks{false}; + std::deque delayed_tasks; + std::deque buffered_ranges; + +private: + Poco::Logger * log = &Poco::Logger::get("MergeTreeBaseSelectProcessor"); + + enum class Status + { + Accepted, + Cancelled, + Denied + }; + + /// Calls getNewTaskImpl() to get new task, then performs a request to a coordinator + /// The coordinator may modify the set of ranges to read from a part or could + /// deny the whole request. In the latter case it creates new task and retries. + /// Then it calls finalizeNewTask() to create readers for a task if it is needed. + bool getNewTask(); + bool getNewTaskParallelReading(); + + /// After PK analysis the range of marks could be extremely big + /// We divide this range to a set smaller consecutive ranges + /// Then, depending on the type of reading (concurrent, in order or in reverse order) + /// we can calculate a consistent hash function with the number of buckets equal to + /// the number of replicas involved. And after that we can throw away some ranges with + /// hash not equals to the number of the current replica. + bool getTaskFromBuffer(); + + /// But we can't throw that ranges completely, because if we have different sets of parts + /// on replicas (have merged part on one, but not on another), then such a situation is possible + /// - Coordinator allows to read from a big merged part, but this part is present only on one replica. + /// And that replica calculates consistent hash and throws away some ranges + /// - Coordinator denies other replicas to read from another parts (source parts for that big one) + /// At the end, the result of the query is wrong, because we didn't read all the data. + /// So, we have to remember parts and mark ranges with hash different then current replica number. + /// An we have to ask the coordinator about its permission to read from that "delayed" parts. + /// It won't work with reading in order or reading in reverse order, because we can possibly seek back. + bool getDelayedTasks(); + + /// It will form a request a request to coordinator and + /// then reinitialize the mark ranges of this->task object + Status performRequestToCoordinator(MarkRanges requested_ranges, bool delayed); + + void splitCurrentTaskRangesAndFillBuffer(); + }; } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 88f3052e833..07d51d25700 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -128,8 +128,6 @@ MergeTreeReadTask::MergeTreeReadTask( { } -MergeTreeReadTask::~MergeTreeReadTask() = default; - MergeTreeBlockSizePredictor::MergeTreeBlockSizePredictor( const MergeTreeData::DataPartPtr & data_part_, const Names & columns, const Block & sample_block) @@ -175,8 +173,7 @@ void MergeTreeBlockSizePredictor::initialize(const Block & sample_block, const C ColumnInfo info; info.name = column_name; /// If column isn't fixed and doesn't have checksum, than take first - ColumnSize column_size = data_part->getColumnSize( - column_name, *column_with_type_and_name.type); + ColumnSize column_size = data_part->getColumnSize(column_name); info.bytes_per_row_global = column_size.data_uncompressed ? column_size.data_uncompressed / number_of_rows_in_part diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 2dfe6fcf06d..b931a13c027 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -14,7 +14,7 @@ struct MergeTreeReadTask; struct MergeTreeBlockSizePredictor; using MergeTreeReadTaskPtr = std::unique_ptr; -using MergeTreeBlockSizePredictorPtr = std::unique_ptr; +using MergeTreeBlockSizePredictorPtr = std::shared_ptr; /** If some of the requested columns are not in the part, @@ -59,8 +59,6 @@ struct MergeTreeReadTask const Names & ordered_names_, const NameSet & column_name_set_, const NamesAndTypesList & columns_, const NamesAndTypesList & pre_columns_, const bool remove_prewhere_column_, const bool should_reorder_, MergeTreeBlockSizePredictorPtr && size_predictor_); - - virtual ~MergeTreeReadTask(); }; struct MergeTreeReadTaskColumns diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3cf7023053f..e58472e572b 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3163,7 +3163,7 @@ void MergeTreeData::addPartContributionToColumnAndSecondaryIndexSizes(const Data for (const auto & column : part->getColumns()) { ColumnSize & total_column_size = column_sizes[column.name]; - ColumnSize part_column_size = part->getColumnSize(column.name, *column.type); + ColumnSize part_column_size = part->getColumnSize(column.name); total_column_size.add(part_column_size); } @@ -3181,7 +3181,7 @@ void MergeTreeData::removePartContributionToColumnAndSecondaryIndexSizes(const D for (const auto & column : part->getColumns()) { ColumnSize & total_column_size = column_sizes[column.name]; - ColumnSize part_column_size = part->getColumnSize(column.name, *column.type); + ColumnSize part_column_size = part->getColumnSize(column.name); auto log_subtract = [&](size_t & from, size_t value, const char * field) { diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 036e7d89c5a..cdedd37e14a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -124,7 +124,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( const UInt64 max_block_size, const unsigned num_streams, QueryProcessingStage::Enum processed_stage, - std::shared_ptr max_block_numbers_to_read) const + std::shared_ptr max_block_numbers_to_read, + bool enable_parallel_reading) const { if (query_info.merge_tree_empty_result) return std::make_unique(); @@ -142,7 +143,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( max_block_size, num_streams, max_block_numbers_to_read, - query_info.merge_tree_select_result_ptr); + query_info.merge_tree_select_result_ptr, + enable_parallel_reading); if (plan->isInitialized() && settings.allow_experimental_projection_optimization && settings.force_optimize_projection && !metadata_snapshot->projections.empty()) @@ -184,7 +186,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read( max_block_size, num_streams, max_block_numbers_to_read, - query_info.projection->merge_tree_projection_select_result_ptr); + query_info.projection->merge_tree_projection_select_result_ptr, + enable_parallel_reading); } if (projection_plan->isInitialized()) @@ -1210,7 +1213,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( const UInt64 max_block_size, const unsigned num_streams, std::shared_ptr max_block_numbers_to_read, - MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr) const + MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr, + bool enable_parallel_reading) const { /// If merge_tree_select_result_ptr != nullptr, we use analyzed result so parts will always be empty. if (merge_tree_select_result_ptr) @@ -1243,7 +1247,8 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( sample_factor_column_queried, max_block_numbers_to_read, log, - merge_tree_select_result_ptr + merge_tree_select_result_ptr, + enable_parallel_reading ); QueryPlanPtr plan = std::make_unique(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index f19d145fc93..3dde324ce22 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -34,7 +34,8 @@ public: UInt64 max_block_size, unsigned num_streams, QueryProcessingStage::Enum processed_stage, - std::shared_ptr max_block_numbers_to_read = nullptr) const; + std::shared_ptr max_block_numbers_to_read = nullptr, + bool enable_parallel_reading = false) const; /// The same as read, but with specified set of parts. QueryPlanPtr readFromParts( @@ -47,7 +48,8 @@ public: UInt64 max_block_size, unsigned num_streams, std::shared_ptr max_block_numbers_to_read = nullptr, - MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr = nullptr) const; + MergeTreeDataSelectAnalysisResultPtr merge_tree_select_result_ptr = nullptr, + bool enable_parallel_reading = false) const; /// Get an estimation for the number of marks we are going to read. /// Reads nothing. Secondary indexes are not used. diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp index 48a9d62d872..961106af51b 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.cpp @@ -8,14 +8,11 @@ namespace ErrorCodes extern const int MEMORY_LIMIT_EXCEEDED; } -bool MergeTreeInOrderSelectProcessor::getNewTask() +bool MergeTreeInOrderSelectProcessor::getNewTaskImpl() try { if (all_mark_ranges.empty()) - { - finish(); return false; - } if (!reader) initializeReaders(); diff --git a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h index ecf648b0291..467292d88bb 100644 --- a/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeInOrderSelectProcessor.h @@ -12,7 +12,7 @@ class MergeTreeInOrderSelectProcessor final : public MergeTreeSelectProcessor { public: template - MergeTreeInOrderSelectProcessor(Args &&... args) + explicit MergeTreeInOrderSelectProcessor(Args &&... args) : MergeTreeSelectProcessor{std::forward(args)...} { LOG_DEBUG(log, "Reading {} ranges in order from part {}, approx. {} rows starting from {}", @@ -23,7 +23,8 @@ public: String getName() const override { return "MergeTreeInOrder"; } private: - bool getNewTask() override; + bool getNewTaskImpl() override; + void finalizeNewTask() override {} Poco::Logger * log = &Poco::Logger::get("MergeTreeInOrderSelectProcessor"); }; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 124f13b14a8..8481cee0f86 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -14,6 +14,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int BAD_ARGUMENTS; } @@ -185,7 +186,7 @@ MergeTreeRangeReader::Stream::Stream( void MergeTreeRangeReader::Stream::checkNotFinished() const { if (isFinished()) - throw Exception("Cannot read out of marks range.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Cannot read out of marks range.", ErrorCodes::BAD_ARGUMENTS); } void MergeTreeRangeReader::Stream::checkEnoughSpaceInCurrentGranule(size_t num_rows) const @@ -290,7 +291,7 @@ void MergeTreeRangeReader::ReadResult::adjustLastGranule() size_t num_rows_to_subtract = total_rows_per_granule - num_read_rows; if (rows_per_granule.empty()) - throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Can't adjust last granule because no granules were added", ErrorCodes::LOGICAL_ERROR); if (num_rows_to_subtract > rows_per_granule.back()) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index 16ce9823ebb..6c4059d64d0 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -8,14 +8,11 @@ namespace ErrorCodes extern const int MEMORY_LIMIT_EXCEEDED; } -bool MergeTreeReverseSelectProcessor::getNewTask() +bool MergeTreeReverseSelectProcessor::getNewTaskImpl() try { if (chunks.empty() && all_mark_ranges.empty()) - { - finish(); return false; - } /// We have some blocks to return in buffer. /// Return true to continue reading, but actually don't create a task. diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index 18ab51c03a0..395f5d5cd2a 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -13,7 +13,7 @@ class MergeTreeReverseSelectProcessor final : public MergeTreeSelectProcessor { public: template - MergeTreeReverseSelectProcessor(Args &&... args) + explicit MergeTreeReverseSelectProcessor(Args &&... args) : MergeTreeSelectProcessor{std::forward(args)...} { LOG_DEBUG(log, "Reading {} ranges in reverse order from part {}, approx. {} rows starting from {}", @@ -24,7 +24,9 @@ public: String getName() const override { return "MergeTreeReverse"; } private: - bool getNewTask() override; + bool getNewTaskImpl() override; + void finalizeNewTask() override {} + Chunk readFromPart() override; Chunks chunks; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 203ce7a57d2..2d4d3617cee 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -22,12 +22,13 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_, size_t part_index_in_query_, - bool has_limit_below_one_block_) + bool has_limit_below_one_block_, + std::optional extension_) : MergeTreeBaseSelectProcessor{ metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), storage_, metadata_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, - reader_settings_, use_uncompressed_cache_, virt_column_names_}, + reader_settings_, use_uncompressed_cache_, virt_column_names_, extension_}, required_columns{std::move(required_columns_)}, data_part{owned_data_part_}, sample_block(metadata_snapshot_->getSampleBlock()), @@ -36,7 +37,11 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( has_limit_below_one_block(has_limit_below_one_block_), total_rows(data_part->index_granularity.getRowsCountInRanges(all_mark_ranges)) { - addTotalRowsApprox(total_rows); + /// Actually it means that parallel reading from replicas enabled + /// and we have to collaborate with initiator. + /// In this case we won't set approximate rows, because it will be accounted multiple times + if (!extension_.has_value()) + addTotalRowsApprox(total_rows); ordered_names = header_without_virtual_columns.getNames(); } @@ -64,6 +69,7 @@ void MergeTreeSelectProcessor::initializeReaders() } + void MergeTreeSelectProcessor::finish() { /** Close the files (before destroying the object). diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index f9b19f9f692..2ecdc3b59a8 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -31,17 +31,16 @@ public: const MergeTreeReaderSettings & reader_settings, const Names & virt_column_names = {}, size_t part_index_in_query_ = 0, - bool has_limit_below_one_block_ = false); + bool has_limit_below_one_block_ = false, + std::optional extension_ = {}); ~MergeTreeSelectProcessor() override; - /// Closes readers and unlock part locks - void finish(); - protected: /// Defer initialization from constructor, because it may be heavy - /// and it's better to do it lazily in `getNewTask`, which is executing in parallel. + /// and it's better to do it lazily in `getNewTaskImpl`, which is executing in parallel. void initializeReaders(); + void finish() override final; /// Used by Task Names required_columns; diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index df8d6a7c127..687458ee681 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -36,6 +36,8 @@ MergeTreeSequentialSource::MergeTreeSequentialSource( data_part->getMarksCount(), data_part->name, data_part->rows_count); } + /// Note, that we don't check setting collaborate_with_coordinator presence, because this source + /// is only used in background merges. addTotalRowsApprox(data_part->rows_count); /// Add columns because we don't want to read empty blocks diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp index 6a8ef860c87..6a44da06f1f 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.cpp @@ -7,6 +7,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} MergeTreeThreadSelectProcessor::MergeTreeThreadSelectProcessor( const size_t thread_, @@ -21,12 +25,13 @@ MergeTreeThreadSelectProcessor::MergeTreeThreadSelectProcessor( const PrewhereInfoPtr & prewhere_info_, ExpressionActionsSettings actions_settings, const MergeTreeReaderSettings & reader_settings_, - const Names & virt_column_names_) + const Names & virt_column_names_, + std::optional extension_) : MergeTreeBaseSelectProcessor{ pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_, std::move(actions_settings), max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, - reader_settings_, use_uncompressed_cache_, virt_column_names_}, + reader_settings_, use_uncompressed_cache_, virt_column_names_, extension_}, thread{thread_}, pool{pool_} { @@ -39,28 +44,61 @@ MergeTreeThreadSelectProcessor::MergeTreeThreadSelectProcessor( min_marks_to_read = (min_marks_to_read_ * fixed_index_granularity + max_block_size_rows - 1) / max_block_size_rows * max_block_size_rows / fixed_index_granularity; } + else if (extension.has_value()) + { + /// Parallel reading from replicas is enabled. + /// We try to estimate the average number of bytes in a granule + /// to make one request over the network per one gigabyte of data + /// Actually we will ask MergeTreeReadPool to provide us heavier tasks to read + /// because the most part of each task will be postponed + /// (due to using consistent hash for better cache affinity) + const size_t amount_of_read_bytes_per_one_request = 1024 * 1024 * 1024; // 1GiB + /// In case of reading from compact parts (for which we can't estimate the average size of marks) + /// we will use this value + const size_t empirical_size_of_mark = 1024 * 1024 * 10; // 10 MiB + + if (extension->colums_to_read.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "A set of column to read is empty. It is a bug"); + + size_t sum_average_marks_size = 0; + auto column_sizes = storage.getColumnSizes(); + for (const auto & name : extension->colums_to_read) + { + auto it = column_sizes.find(name); + if (it == column_sizes.end()) + continue; + auto size = it->second; + + if (size.data_compressed == 0 || size.data_uncompressed == 0 || size.marks == 0) + continue; + + sum_average_marks_size += size.data_uncompressed / size.marks; + } + + if (sum_average_marks_size == 0) + sum_average_marks_size = empirical_size_of_mark * extension->colums_to_read.size(); + + min_marks_to_read = extension->count_participating_replicas * amount_of_read_bytes_per_one_request / sum_average_marks_size; + } else + { min_marks_to_read = min_marks_to_read_; + } + ordered_names = getPort().getHeader().getNames(); } /// Requests read task from MergeTreeReadPool and signals whether it got one -bool MergeTreeThreadSelectProcessor::getNewTask() +bool MergeTreeThreadSelectProcessor::getNewTaskImpl() { task = pool->getTask(min_marks_to_read, thread, ordered_names); + return static_cast(task); +} - if (!task) - { - /** Close the files (before destroying the object). - * When many sources are created, but simultaneously reading only a few of them, - * buffers don't waste memory. - */ - reader.reset(); - pre_reader.reset(); - return false; - } +void MergeTreeThreadSelectProcessor::finalizeNewTask() +{ const std::string part_name = task->data_part->isProjectionPart() ? task->data_part->getParentPart()->name : task->data_part->name; /// Allows pool to reduce number of threads in case of too slow reads. @@ -99,8 +137,13 @@ bool MergeTreeThreadSelectProcessor::getNewTask() } last_readed_part_name = part_name; +} - return true; + +void MergeTreeThreadSelectProcessor::finish() +{ + reader.reset(); + pre_reader.reset(); } diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h index d17b15c3635..110c4fa34e6 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectProcessor.h @@ -11,7 +11,7 @@ class MergeTreeReadPool; /** Used in conjunction with MergeTreeReadPool, asking it for more work to do and performing whatever reads it is asked * to perform. */ -class MergeTreeThreadSelectProcessor : public MergeTreeBaseSelectProcessor +class MergeTreeThreadSelectProcessor final : public MergeTreeBaseSelectProcessor { public: MergeTreeThreadSelectProcessor( @@ -27,8 +27,8 @@ public: const PrewhereInfoPtr & prewhere_info_, ExpressionActionsSettings actions_settings, const MergeTreeReaderSettings & reader_settings_, - - const Names & virt_column_names_); + const Names & virt_column_names_, + std::optional extension_); String getName() const override { return "MergeTreeThread"; } @@ -36,7 +36,13 @@ public: protected: /// Requests read task from MergeTreeReadPool and signals whether it got one - bool getNewTask() override; + bool getNewTaskImpl() override; + + void finalizeNewTask() override; + + void finish() override; + + bool canUseConsistentHashingForParallelReading() override { return true; } private: /// "thread" index (there are N threads and each thread is assigned index in interval [0..N-1]) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp new file mode 100644 index 00000000000..80f438a46db --- /dev/null +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -0,0 +1,143 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#include +#include +#include +#include +#include "IO/WriteBufferFromString.h" +#include +#include + +namespace DB +{ + +class ParallelReplicasReadingCoordinator::Impl +{ +public: + using PartitionReadRequestPtr = std::unique_ptr; + using PartToMarkRanges = std::map; + + struct PartitionReading + { + PartSegments part_ranges; + PartToMarkRanges mark_ranges_in_part; + }; + + using PartitionToBlockRanges = std::map; + PartitionToBlockRanges partitions; + + std::mutex mutex; + + PartitionReadResponse handleRequest(PartitionReadRequest request); +}; + + +PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(PartitionReadRequest request) +{ + AtomicStopwatch watch; + std::lock_guard lock(mutex); + + auto partition_it = partitions.find(request.partition_id); + + SCOPE_EXIT({ + LOG_TRACE(&Poco::Logger::get("ParallelReplicasReadingCoordinator"), "Time for handling request: {}ns", watch.elapsed()); + }); + + PartToRead::PartAndProjectionNames part_and_projection + { + .part = request.part_name, + .projection = request.projection_name + }; + + /// We are the first who wants to process parts in partition + if (partition_it == partitions.end()) + { + PartitionReading partition_reading; + + PartToRead part_to_read; + part_to_read.range = request.block_range; + part_to_read.name = part_and_projection; + + partition_reading.part_ranges.addPart(std::move(part_to_read)); + + /// As this query is first in partition, we will accept all ranges from it. + /// We need just to update our state. + auto request_ranges = HalfIntervals::initializeFromMarkRanges(request.mark_ranges); + auto mark_ranges_index = HalfIntervals::initializeWithEntireSpace(); + mark_ranges_index.intersect(request_ranges.negate()); + + partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); + partitions.insert({request.partition_id, std::move(partition_reading)}); + + return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; + } + + auto & partition_reading = partition_it->second; + + PartToRead part_to_read; + part_to_read.range = request.block_range; + part_to_read.name = part_and_projection; + + auto part_intersection_res = partition_reading.part_ranges.getIntersectionResult(part_to_read); + + switch (part_intersection_res) + { + case PartSegments::IntersectionResult::REJECT: + { + return {.denied = true, .mark_ranges = {}}; + } + case PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION: + { + auto marks_it = partition_reading.mark_ranges_in_part.find(part_and_projection); + + auto & intervals_to_do = marks_it->second; + auto result = HalfIntervals::initializeFromMarkRanges(request.mark_ranges); + result.intersect(intervals_to_do); + + /// Update intervals_to_do + intervals_to_do.intersect(HalfIntervals::initializeFromMarkRanges(std::move(request.mark_ranges)).negate()); + + auto result_ranges = result.convertToMarkRangesFinal(); + const bool denied = result_ranges.empty(); + return {.denied = denied, .mark_ranges = std::move(result_ranges)}; + } + case PartSegments::IntersectionResult::NO_INTERSECTION: + { + partition_reading.part_ranges.addPart(std::move(part_to_read)); + + auto mark_ranges_index = HalfIntervals::initializeWithEntireSpace().intersect( + HalfIntervals::initializeFromMarkRanges(request.mark_ranges).negate() + ); + partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); + + return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; + } + } + + __builtin_unreachable(); +} + +PartitionReadResponse ParallelReplicasReadingCoordinator::handleRequest(PartitionReadRequest request) +{ + return pimpl->handleRequest(std::move(request)); +} + +ParallelReplicasReadingCoordinator::ParallelReplicasReadingCoordinator() +{ + pimpl = std::make_unique(); +} + +ParallelReplicasReadingCoordinator::~ParallelReplicasReadingCoordinator() = default; + +} diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h new file mode 100644 index 00000000000..af74e0fae49 --- /dev/null +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class ParallelReplicasReadingCoordinator +{ +public: + ParallelReplicasReadingCoordinator(); + ~ParallelReplicasReadingCoordinator(); + PartitionReadResponse handleRequest(PartitionReadRequest request); +private: + class Impl; + std::unique_ptr pimpl; +}; + +} diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp new file mode 100644 index 00000000000..a266540b99a --- /dev/null +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -0,0 +1,141 @@ +#include + +#include +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_PROTOCOL; +} + +static void readMarkRangesBinary(MarkRanges & ranges, ReadBuffer & buf, size_t MAX_RANGES_SIZE = DEFAULT_MAX_STRING_SIZE) +{ + size_t size = 0; + readVarUInt(size, buf); + + if (size > MAX_RANGES_SIZE) + throw Poco::Exception("Too large ranges size."); + + ranges.resize(size); + for (size_t i = 0; i < size; ++i) + { + readBinary(ranges[i].begin, buf); + readBinary(ranges[i].end, buf); + } +} + + +static void writeMarkRangesBinary(const MarkRanges & ranges, WriteBuffer & buf) +{ + writeVarUInt(ranges.size(), buf); + + for (const auto & [begin, end] : ranges) + { + writeBinary(begin, buf); + writeBinary(end, buf); + } +} + + +void PartitionReadRequest::serialize(WriteBuffer & out) const +{ + /// Must be the first + writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + + writeStringBinary(partition_id, out); + writeStringBinary(part_name, out); + writeStringBinary(projection_name, out); + + writeVarInt(block_range.begin, out); + writeVarInt(block_range.end, out); + + writeMarkRangesBinary(mark_ranges, out); +} + + +void PartitionReadRequest::describe(WriteBuffer & out) const +{ + String result; + result += fmt::format("partition_id: {} \n", partition_id); + result += fmt::format("part_name: {} \n", part_name); + result += fmt::format("projection_name: {} \n", projection_name); + result += fmt::format("block_range: ({}, {}) \n", block_range.begin, block_range.end); + result += "mark_ranges: "; + for (const auto & range : mark_ranges) + result += fmt::format("({}, {}) ", range.begin, range.end); + result += '\n'; + out.write(result.c_str(), result.size()); +} + +void PartitionReadRequest::deserialize(ReadBuffer & in) +{ + UInt64 version; + readVarUInt(version, in); + if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading \ + from replicas differ. Got: {}, supported version: {}", + version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + + readStringBinary(partition_id, in); + readStringBinary(part_name, in); + readStringBinary(projection_name, in); + + readVarInt(block_range.begin, in); + readVarInt(block_range.end, in); + + readMarkRangesBinary(mark_ranges, in); +} + +UInt64 PartitionReadRequest::getConsistentHash(size_t buckets) const +{ + auto hash = SipHash(); + hash.update(partition_id); + hash.update(part_name); + hash.update(projection_name); + + hash.update(block_range.begin); + hash.update(block_range.end); + + for (const auto & range : mark_ranges) + { + hash.update(range.begin); + hash.update(range.end); + } + + return ConsistentHashing(hash.get64(), buckets); +} + + +void PartitionReadResponse::serialize(WriteBuffer & out) const +{ + /// Must be the first + writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); + + writeVarUInt(static_cast(denied), out); + writeMarkRangesBinary(mark_ranges, out); +} + + +void PartitionReadResponse::deserialize(ReadBuffer & in) +{ + UInt64 version; + readVarUInt(version, in); + if (version != DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION) + throw Exception(ErrorCodes::UNKNOWN_PROTOCOL, "Protocol versions for parallel reading \ + from replicas differ. Got: {}, supported version: {}", + version, DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION); + + UInt64 value; + readVarUInt(value, in); + denied = static_cast(value); + readMarkRangesBinary(mark_ranges, in); +} + +} diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h new file mode 100644 index 00000000000..85c8f7181af --- /dev/null +++ b/src/Storages/MergeTree/RequestResponse.h @@ -0,0 +1,57 @@ +#pragma once + +#include +#include + +#include + +#include +#include + +#include + + +namespace DB +{ + +/// Represents a segment [left; right] +struct PartBlockRange +{ + Int64 begin; + Int64 end; + + bool operator==(const PartBlockRange & rhs) const + { + return begin == rhs.begin && end == rhs.end; + } +}; + +struct PartitionReadRequest +{ + String partition_id; + String part_name; + String projection_name; + PartBlockRange block_range; + MarkRanges mark_ranges; + + void serialize(WriteBuffer & out) const; + void describe(WriteBuffer & out) const; + void deserialize(ReadBuffer & in); + + UInt64 getConsistentHash(size_t buckets) const; +}; + +struct PartitionReadResponse +{ + bool denied{false}; + MarkRanges mark_ranges{}; + + void serialize(WriteBuffer & out) const; + void deserialize(ReadBuffer & in); +}; + + +using MergeTreeReadTaskCallback = std::function(PartitionReadRequest)>; + + +} diff --git a/src/Storages/MergeTree/tests/gtest_coordinator.cpp b/src/Storages/MergeTree/tests/gtest_coordinator.cpp new file mode 100644 index 00000000000..7bcf3304c2b --- /dev/null +++ b/src/Storages/MergeTree/tests/gtest_coordinator.cpp @@ -0,0 +1,240 @@ +#include + +#include +#include +#include + +#include + +#include + +using namespace DB; + + +TEST(HalfIntervals, Simple) +{ + ASSERT_TRUE(( + HalfIntervals{{{1, 2}, {3, 4}}}.negate() == + HalfIntervals{{{0, 1}, {2, 3}, {4, 18446744073709551615UL}}} + )); + + { + auto left = HalfIntervals{{{0, 2}, {4, 6}}}.negate(); + ASSERT_TRUE(( + left == + HalfIntervals{{{2, 4}, {6, 18446744073709551615UL}}} + )); + } + + { + auto left = HalfIntervals{{{0, 2}, {4, 6}}}; + auto right = HalfIntervals{{{1, 5}}}.negate(); + auto intersection = left.intersect(right); + + ASSERT_TRUE(( + intersection == + HalfIntervals{{{0, 1}, {5, 6}}} + )); + } + + { + auto left = HalfIntervals{{{1, 2}, {2, 3}}}; + auto right = HalfIntervals::initializeWithEntireSpace(); + auto intersection = right.intersect(left.negate()); + + ASSERT_TRUE(( + intersection == + HalfIntervals{{{0, 1}, {3, 18446744073709551615UL}}} + )); + } + + { + auto left = HalfIntervals{{{1, 2}, {2, 3}, {3, 4}, {4, 5}}}; + + ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 4}}}).convertToMarkRangesFinal().size(), 3); + ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 5}}}).convertToMarkRangesFinal().size(), 4); + } + + { + auto left = HalfIntervals{{{1, 3}, {3, 5}, {5, 7}}}; + + ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 5}}}).convertToMarkRangesFinal().size(), 1); + ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 7}}}).convertToMarkRangesFinal().size(), 2); + ASSERT_EQ(getIntersection(left, HalfIntervals{{{4, 6}}}).convertToMarkRangesFinal().size(), 2); + ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 7}}}).convertToMarkRangesFinal().size(), 3); + } + + { + auto left = HalfIntervals{{{1, 3}}}; + + ASSERT_EQ(getIntersection(left, HalfIntervals{{{3, 4}}}).convertToMarkRangesFinal().size(), 0); + } + + { + auto left = HalfIntervals{{{1, 2}, {3, 4}, {5, 6}}}; + + ASSERT_EQ(getIntersection(left, HalfIntervals{{{2, 3}}}).convertToMarkRangesFinal().size(), 0); + ASSERT_EQ(getIntersection(left, HalfIntervals{{{4, 5}}}).convertToMarkRangesFinal().size(), 0); + ASSERT_EQ(getIntersection(left, HalfIntervals{{{1, 6}}}).convertToMarkRangesFinal().size(), 3); + } +} + +TEST(HalfIntervals, TwoRequests) +{ + auto left = HalfIntervals{{{1, 2}, {2, 3}}}; + auto right = HalfIntervals{{{2, 3}, {3, 4}}}; + auto intersection = left.intersect(right); + + ASSERT_TRUE(( + intersection == + HalfIntervals{{{2, 3}}} + )); + + /// With negation + left = HalfIntervals{{{1, 2}, {2, 3}}}.negate(); + right = HalfIntervals{{{2, 3}, {3, 4}}}; + intersection = left.intersect(right); + + + ASSERT_TRUE(( + intersection == + HalfIntervals{{{3, 4}}} + )); +} + +TEST(HalfIntervals, SelfIntersection) +{ + auto left = HalfIntervals{{{1, 2}, {2, 3}, {4, 5}}}; + auto right = left; + auto intersection = left.intersect(right); + + ASSERT_TRUE(( + intersection == right + )); + + left = HalfIntervals{{{1, 2}, {2, 3}, {4, 5}}}; + right = left; + right.negate(); + intersection = left.intersect(right); + + ASSERT_TRUE(( + intersection == HalfIntervals{} + )); +} + + +TEST(Coordinator, Simple) +{ + PartitionReadRequest request; + request.partition_id = "a"; + request.part_name = "b"; + request.projection_name = "c"; + request.block_range = PartBlockRange{1, 2}; + request.mark_ranges = MarkRanges{{1, 2}, {3, 4}}; + + ParallelReplicasReadingCoordinator coordinator; + auto response = coordinator.handleRequest(request); + + ASSERT_FALSE(response.denied) << "Process request at first has to be accepted"; + + ASSERT_EQ(response.mark_ranges.size(), request.mark_ranges.size()); + + for (int i = 0; i < response.mark_ranges.size(); ++i) + EXPECT_EQ(response.mark_ranges[i], request.mark_ranges[i]); + + response = coordinator.handleRequest(request); + ASSERT_TRUE(response.denied) << "Process the same request second time"; +} + + +TEST(Coordinator, TwoRequests) +{ + PartitionReadRequest first; + first.partition_id = "a"; + first.part_name = "b"; + first.projection_name = "c"; + first.block_range = PartBlockRange{0, 0}; + first.mark_ranges = MarkRanges{{1, 2}, {2, 3}}; + + auto second = first; + second.mark_ranges = MarkRanges{{2, 3}, {3, 4}}; + + ParallelReplicasReadingCoordinator coordinator; + auto response = coordinator.handleRequest(first); + + ASSERT_FALSE(response.denied) << "First request must me accepted"; + + ASSERT_EQ(response.mark_ranges.size(), first.mark_ranges.size()); + for (int i = 0; i < response.mark_ranges.size(); ++i) + EXPECT_EQ(response.mark_ranges[i], first.mark_ranges[i]); + + response = coordinator.handleRequest(second); + ASSERT_FALSE(response.denied); + ASSERT_EQ(response.mark_ranges.size(), 1); + ASSERT_EQ(response.mark_ranges.front(), (MarkRange{3, 4})); +} + + +TEST(Coordinator, PartIntersections) +{ + { + PartSegments boundaries; + + boundaries.addPart(PartToRead{{1, 1}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{2, 2}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{3, 3}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{4, 4}, {"TestPart", "TestProjection"}}); + + ASSERT_EQ(boundaries.getIntersectionResult({{1, 4}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{0, 5}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"ClickHouse", "AnotherProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 2}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + + boundaries.addPart(PartToRead{{5, 5}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{0, 0}, {"TestPart", "TestProjection"}}); + + ASSERT_EQ(boundaries.getIntersectionResult({{0, 5}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"ClickHouse", "AnotherProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 2}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{0, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + } + + { + PartSegments boundaries; + boundaries.addPart(PartToRead{{1, 3}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{4, 5}, {"TestPart", "TestProjection"}}); + + ASSERT_EQ(boundaries.getIntersectionResult({{2, 4}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{0, 6}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + } + + { + PartSegments boundaries; + boundaries.addPart(PartToRead{{1, 3}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{4, 6}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{7, 9}, {"TestPart", "TestProjection"}}); + + ASSERT_EQ(boundaries.getIntersectionResult({{2, 8}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{4, 6}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); + ASSERT_EQ(boundaries.getIntersectionResult({{3, 7}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{5, 7}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + } + + { + PartSegments boundaries; + + ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); + ASSERT_EQ(boundaries.getIntersectionResult({{0, 100500}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); + + boundaries.addPart(PartToRead{{1, 1}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{2, 2}, {"TestPart", "TestProjection"}}); + boundaries.addPart(PartToRead{{3, 3}, {"TestPart", "TestProjection"}}); + + ASSERT_EQ(boundaries.getIntersectionResult({{1, 1}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION); + ASSERT_EQ(boundaries.getIntersectionResult({{1, 3}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::REJECT); + ASSERT_EQ(boundaries.getIntersectionResult({{100, 100500}, {"TestPart", "TestProjection"}}), PartSegments::IntersectionResult::NO_INTERSECTION); + } +} diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a45afd847e6..e033d319fc8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -110,6 +110,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INFINITE_LOOP; + extern const int ILLEGAL_FINAL; extern const int TYPE_MISMATCH; extern const int TOO_MANY_ROWS; extern const int UNABLE_TO_SKIP_UNUSED_SHARDS; @@ -273,7 +274,7 @@ size_t getClusterQueriedNodes(const Settings & settings, const ClusterPtr & clus { size_t num_local_shards = cluster->getLocalShardCount(); size_t num_remote_shards = cluster->getRemoteShardCount(); - return (num_remote_shards * settings.max_parallel_replicas) + num_local_shards; + return (num_remote_shards + num_local_shards) * settings.max_parallel_replicas; } } @@ -590,6 +591,10 @@ void StorageDistributed::read( const size_t /*max_block_size*/, const unsigned /*num_streams*/) { + const auto * select_query = query_info.query->as(); + if (select_query->final() && local_context->getSettingsRef().allow_experimental_parallel_reading_from_replicas) + throw Exception(ErrorCodes::ILLEGAL_FINAL, "Final modifier is not allowed together with parallel reading from replicas feature"); + const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 8a3d786532e..470a406dbe4 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -191,7 +191,14 @@ void StorageMergeTree::read( size_t max_block_size, unsigned num_streams) { - if (auto plan = reader.read(column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage)) + /// If true, then we will ask initiator if we can read chosen ranges + bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; + + if (enable_parallel_reading) + LOG_TRACE(log, "Parallel reading from replicas enabled {}", enable_parallel_reading); + + if (auto plan = reader.read( + column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading)) query_plan = std::move(*plan); } diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 89506184354..d0d52fd488a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -48,8 +48,10 @@ #include #include +#include #include #include +#include #include #include @@ -61,6 +63,8 @@ #include #include #include +#include +#include #include @@ -4228,6 +4232,9 @@ void StorageReplicatedMergeTree::read( const size_t max_block_size, const unsigned num_streams) { + /// If true, then we will ask initiator if we can read chosen ranges + const bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; + /** The `select_sequential_consistency` setting has two meanings: * 1. To throw an exception if on a replica there are not all parts which have been written down on quorum of remaining replicas. * 2. Do not read parts that have not yet been written to the quorum of the replicas. @@ -4237,13 +4244,18 @@ void StorageReplicatedMergeTree::read( { auto max_added_blocks = std::make_shared(getMaxAddedBlocks()); if (auto plan = reader.read( - column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, std::move(max_added_blocks))) + column_names, metadata_snapshot, query_info, local_context, + max_block_size, num_streams, processed_stage, std::move(max_added_blocks), enable_parallel_reading)) query_plan = std::move(*plan); return; } - if (auto plan = reader.read(column_names, metadata_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage)) + if (auto plan = reader.read( + column_names, metadata_snapshot, query_info, local_context, + max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading)) + { query_plan = std::move(*plan); + } } Pipe StorageReplicatedMergeTree::read( diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index f49fd35044d..659071b392d 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -126,7 +126,7 @@ Pipe StorageS3Cluster::read( scalars, Tables(), processed_stage, - callback); + RemoteQueryExecutor::Extension{.task_iterator = callback}); pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); } diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index 33ec5c457f6..8dbd73628ca 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -206,7 +206,7 @@ void StorageSystemPartsColumns::processNextStorage( columns[res_index++]->insertDefault(); } - ColumnSize column_size = part->getColumnSize(column.name, *column.type); + ColumnSize column_size = part->getColumnSize(column.name); if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.data_compressed + column_size.marks); if (columns_mask[src_index++]) diff --git a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp index bdbe9a46846..f6490177014 100644 --- a/src/Storages/System/StorageSystemProjectionPartsColumns.cpp +++ b/src/Storages/System/StorageSystemProjectionPartsColumns.cpp @@ -237,7 +237,7 @@ void StorageSystemProjectionPartsColumns::processNextStorage( columns[res_index++]->insertDefault(); } - ColumnSize column_size = part->getColumnSize(column.name, *column.type); + ColumnSize column_size = part->getColumnSize(column.name); if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.data_compressed + column_size.marks); if (columns_mask[src_index++]) diff --git a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql index 6bc5fe268d6..f9cbf92db41 100644 --- a/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql +++ b/tests/queries/0_stateless/00124_shard_distributed_with_many_replicas.sql @@ -1,7 +1,6 @@ -- Tags: replica, distributed SET max_parallel_replicas = 2; - DROP TABLE IF EXISTS report; CREATE TABLE report(id UInt32, event_date Date, priority UInt32, description String) ENGINE = MergeTree(event_date, intHash32(id), (id, event_date, intHash32(id)), 8192); diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 06b6fc86d3e..3d839de9c64 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -50,7 +50,7 @@ SELECT count() FROM table4 WHERE id % 10 = 7; SELECT 'comparison:'; SELECT v, v-205 as vv, modulo(vv, 200), moduloLegacy(vv, 200) FROM table1 ORDER BY v; -DROP TABLE table1; -DROP TABLE table2; -DROP TABLE table3; -DROP TABLE table4; +DROP TABLE table1 SYNC; +DROP TABLE table2 SYNC; +DROP TABLE table3 SYNC; +DROP TABLE table4 SYNC; diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.reference b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.reference new file mode 100644 index 00000000000..2675904dea0 --- /dev/null +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.reference @@ -0,0 +1,110 @@ +Testing 00001_count_hits.sql ----> Ok! ✅ +Testing 00002_count_visits.sql ----> Ok! ✅ +Testing 00004_top_counters.sql ----> Ok! ✅ +Testing 00005_filtering.sql ----> Ok! ✅ +Testing 00006_agregates.sql ----> Ok! ✅ +Testing 00007_uniq.sql ----> Ok! ✅ +Testing 00008_uniq.sql ----> Ok! ✅ +Testing 00009_uniq_distributed.sql ----> Ok! ✅ +Testing 00010_quantiles_segfault.sql ----> Ok! ✅ +Testing 00011_sorting.sql ----> Ok! ✅ +Testing 00012_sorting_distributed.sql ----> Ok! ✅ +Skipping 00013_sorting_of_nested.sql +Testing 00014_filtering_arrays.sql ----> Ok! ✅ +Testing 00015_totals_and_no_aggregate_functions.sql ----> Ok! ✅ +Testing 00016_any_if_distributed_cond_always_false.sql ----> Ok! ✅ +Testing 00017_aggregation_uninitialized_memory.sql ----> Ok! ✅ +Testing 00020_distinct_order_by_distributed.sql ----> Ok! ✅ +Testing 00021_1_select_with_in.sql ----> Ok! ✅ +Testing 00021_2_select_with_in.sql ----> Ok! ✅ +Testing 00021_3_select_with_in.sql ----> Ok! ✅ +Testing 00022_merge_prewhere.sql ----> Ok! ✅ +Testing 00023_totals_limit.sql ----> Ok! ✅ +Testing 00024_random_counters.sql ----> Ok! ✅ +Testing 00030_array_enumerate_uniq.sql ----> Ok! ✅ +Testing 00031_array_enumerate_uniq.sql ----> Ok! ✅ +Testing 00032_aggregate_key64.sql ----> Ok! ✅ +Testing 00033_aggregate_key_string.sql ----> Ok! ✅ +Testing 00034_aggregate_key_fixed_string.sql ----> Ok! ✅ +Testing 00035_aggregate_keys128.sql ----> Ok! ✅ +Testing 00036_aggregate_hashed.sql ----> Ok! ✅ +Testing 00037_uniq_state_merge1.sql ----> Ok! ✅ +Testing 00038_uniq_state_merge2.sql ----> Ok! ✅ +Testing 00039_primary_key.sql ----> Ok! ✅ +Testing 00040_aggregating_materialized_view.sql ----> Ok! ✅ +Testing 00041_aggregating_materialized_view.sql ----> Ok! ✅ +Testing 00042_any_left_join.sql ----> Ok! ✅ +Testing 00043_any_left_join.sql ----> Ok! ✅ +Testing 00044_any_left_join_string.sql ----> Ok! ✅ +Testing 00045_uniq_upto.sql ----> Ok! ✅ +Testing 00046_uniq_upto_distributed.sql ----> Ok! ✅ +Testing 00047_bar.sql ----> Ok! ✅ +Testing 00048_min_max.sql ----> Ok! ✅ +Testing 00049_max_string_if.sql ----> Ok! ✅ +Testing 00050_min_max.sql ----> Ok! ✅ +Testing 00051_min_max_array.sql ----> Ok! ✅ +Testing 00052_group_by_in.sql ----> Ok! ✅ +Testing 00053_replicate_segfault.sql ----> Ok! ✅ +Testing 00054_merge_tree_partitions.sql ----> Ok! ✅ +Testing 00055_index_and_not.sql ----> Ok! ✅ +Testing 00056_view.sql ----> Ok! ✅ +Testing 00059_merge_sorting_empty_array_joined.sql ----> Ok! ✅ +Testing 00060_move_to_prewhere_and_sets.sql ----> Ok! ✅ +Skipping 00061_storage_buffer.sql +Testing 00062_loyalty.sql ----> Ok! ✅ +Testing 00063_loyalty_joins.sql ----> Ok! ✅ +Testing 00065_loyalty_with_storage_join.sql ----> Ok! ✅ +Testing 00066_sorting_distributed_many_replicas.sql ----> Ok! ✅ +Testing 00067_union_all.sql ----> Ok! ✅ +Testing 00068_subquery_in_prewhere.sql ----> Ok! ✅ +Testing 00069_duplicate_aggregation_keys.sql ----> Ok! ✅ +Testing 00071_merge_tree_optimize_aio.sql ----> Ok! ✅ +Testing 00072_compare_date_and_string_index.sql ----> Ok! ✅ +Testing 00073_uniq_array.sql ----> Ok! ✅ +Testing 00074_full_join.sql ----> Ok! ✅ +Testing 00075_left_array_join.sql ----> Ok! ✅ +Testing 00076_system_columns_bytes.sql ----> Ok! ✅ +Testing 00077_log_tinylog_stripelog.sql ----> Ok! ✅ +Testing 00078_group_by_arrays.sql ----> Ok! ✅ +Testing 00079_array_join_not_used_joined_column.sql ----> Ok! ✅ +Testing 00080_array_join_and_union.sql ----> Ok! ✅ +Testing 00081_group_by_without_key_and_totals.sql ----> Ok! ✅ +Testing 00082_quantiles.sql ----> Ok! ✅ +Testing 00083_array_filter.sql ----> Ok! ✅ +Testing 00084_external_aggregation.sql ----> Ok! ✅ +Testing 00085_monotonic_evaluation_segfault.sql ----> Ok! ✅ +Testing 00086_array_reduce.sql ----> Ok! ✅ +Testing 00087_where_0.sql ----> Ok! ✅ +Testing 00088_global_in_one_shard_and_rows_before_limit.sql ----> Ok! ✅ +Testing 00089_position_functions_with_non_constant_arg.sql ----> Ok! ✅ +Testing 00091_prewhere_two_conditions.sql ----> Ok! ✅ +Testing 00093_prewhere_array_join.sql ----> Ok! ✅ +Testing 00094_order_by_array_join_limit.sql ----> Ok! ✅ +Skipping 00095_hyperscan_profiler.sql +Testing 00139_like.sql ----> Ok! ✅ +Skipping 00140_rename.sql +Testing 00141_transform.sql ----> Ok! ✅ +Testing 00142_system_columns.sql ----> Ok! ✅ +Testing 00143_transform_non_const_default.sql ----> Ok! ✅ +Testing 00144_functions_of_aggregation_states.sql ----> Ok! ✅ +Testing 00145_aggregate_functions_statistics.sql ----> Ok! ✅ +Testing 00146_aggregate_function_uniq.sql ----> Ok! ✅ +Testing 00147_global_in_aggregate_function.sql ----> Ok! ✅ +Testing 00148_monotonic_functions_and_index.sql ----> Ok! ✅ +Testing 00149_quantiles_timing_distributed.sql ----> Ok! ✅ +Testing 00150_quantiles_timing_precision.sql ----> Ok! ✅ +Testing 00151_order_by_read_in_order.sql ----> Ok! ✅ +Skipping 00151_replace_partition_with_different_granularity.sql +Skipping 00152_insert_different_granularity.sql +Testing 00153_aggregate_arena_race.sql ----> Ok! ✅ +Skipping 00154_avro.sql +Testing 00156_max_execution_speed_sample_merge.sql ----> Ok! ✅ +Skipping 00157_cache_dictionary.sql +Skipping 00158_cache_dictionary_has.sql +Testing 00160_decode_xml_component.sql ----> Ok! ✅ +Testing 00162_mmap_compression_none.sql ----> Ok! ✅ +Testing 00164_quantileBfloat16.sql ----> Ok! ✅ +Testing 00165_jit_aggregate_functions.sql ----> Ok! ✅ +Skipping 00166_explain_estimate.sql +Testing 00167_read_bytes_from_fs.sql ----> Ok! ✅ +Total failed tests: diff --git a/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh new file mode 100755 index 00000000000..ba1245d9679 --- /dev/null +++ b/tests/queries/1_stateful/00168_parallel_processing_on_replicas_part_1.sh @@ -0,0 +1,98 @@ +#!/usr/bin/env bash +# Tags: no-tsan + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# set -e + +# All replicas are localhost, disable `prefer_localhost_replica` option to test network interface +# Currently this feature could not work with hedged requests +# Enabling `enable_sample_offset_parallel_processing` feature could lead to intersecting marks, so some of them would be thrown away and it will lead to incorrect result of SELECT query +SETTINGS="--max_parallel_replicas=3 --prefer_localhost_replica=false --use_hedged_requests=false --async_socket_for_remote=false --allow_experimental_parallel_reading_from_replicas=true" + +# Prepare tables +$CLICKHOUSE_CLIENT $SETTINGS -nm -q ''' + drop table if exists test.dist_hits SYNC; + drop table if exists test.dist_visits SYNC; + + create table test.dist_hits as test.hits engine = Distributed("test_cluster_one_shard_three_replicas_localhost", test, hits, rand()); + create table test.dist_visits as test.visits engine = Distributed("test_cluster_one_shard_three_replicas_localhost", test, visits, rand()); +'''; + +FAILED=() + +# PreviouslyFailed=( +# ) + +SkipList=( + "00013_sorting_of_nested.sql" # It contains FINAL, which is not allowed together with parallel reading + + "00061_storage_buffer.sql" + "00095_hyperscan_profiler.sql" # too long in debug (there is a --no-debug tag inside a test) + + "00140_rename.sql" # Multiple renames are not allowed with DatabaseReplicated and tags are not forwarded through this test + + "00154_avro.sql" # Plain select * with limit with Distributed table is not deterministic + "00151_replace_partition_with_different_granularity.sql" # Replace partition from Distributed is not allowed + "00152_insert_different_granularity.sql" # The same as above + + "00157_cache_dictionary.sql" # Too long in debug mode, but result is correct + "00158_cache_dictionary_has.sql" # The same as above + + "00166_explain_estimate.sql" # Distributed table returns nothing +) + +# for TESTPATH in "${PreviouslyFailed[@]}" +for TESTPATH in "$CURDIR"/*.sql; +do + TESTNAME=$(basename $TESTPATH) + + if [[ " ${SkipList[*]} " =~ ${TESTNAME} ]]; then + echo "Skipping $TESTNAME " + continue + fi + + echo -n "Testing $TESTNAME ----> " + + # prepare test + NEW_TESTNAME="/tmp/dist_$TESTNAME" + # Added g to sed command to replace all tables, not the first + cat $TESTPATH | sed -e 's/test.hits/test.dist_hits/g' | sed -e 's/test.visits/test.dist_visits/g' > $NEW_TESTNAME + + TESTNAME_RESULT="/tmp/result_$TESTNAME" + NEW_TESTNAME_RESULT="/tmp/result_dist_$TESTNAME" + + $CLICKHOUSE_CLIENT $SETTINGS -nm --testmode < $TESTPATH > $TESTNAME_RESULT + $CLICKHOUSE_CLIENT $SETTINGS -nm --testmode < $NEW_TESTNAME > $NEW_TESTNAME_RESULT + + expected=$(cat $TESTNAME_RESULT | md5sum) + actual=$(cat $NEW_TESTNAME_RESULT | md5sum) + + if [[ "$expected" != "$actual" ]]; then + FAILED+=("$TESTNAME") + echo "Failed! ❌ " + echo "Plain:" + cat $TESTNAME_RESULT + echo "Distributed:" + cat $NEW_TESTNAME_RESULT + else + echo "Ok! ✅" + fi +done + + +echo "Total failed tests: " +# Iterate the loop to read and print each array element +for value in "${FAILED[@]}" +do + echo "🔺 $value" +done + +# Drop tables + +$CLICKHOUSE_CLIENT $SETTINGS -nm -q ''' + drop table if exists test.dist_hits SYNC; + drop table if exists test.dist_visits SYNC; +'''; From de0a9a84a94c61900d08774c57cb6d223b16af6d Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 9 Dec 2021 13:51:39 +0300 Subject: [PATCH 598/609] 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 351b4a3c541..b8fe9dcbc4f 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -35,7 +35,7 @@ function clone fi git diff --name-only master HEAD | tee ci-changed-files.txt else - if [ -v COMMIT_SHA ]; then + if [ -v SHA_TO_TEST ]; then git fetch --depth 2 origin "$SHA_TO_TEST" git checkout "$SHA_TO_TEST" echo "Checked out nominal SHA $SHA_TO_TEST for master" From f4cbbfd8c04e0320842d2d8d475ba6e3f2774029 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 9 Dec 2021 14:17:55 +0300 Subject: [PATCH 599/609] 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 b8fe9dcbc4f..fe9aa199a08 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -1,5 +1,5 @@ #!/bin/bash -# shellcheck disable=SC2086,SC2001,SC2046 +# shellcheck disable=SC2086,SC2001,SC2046,SC2030,SC2031 set -eux set -o pipefail From 19f8b416bcd75fa29b79375ba4b8e1c3f3230ca3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 16:28:15 +0300 Subject: [PATCH 600/609] Add automatic workflow rerun --- .../Dockerfile | 0 .../app.py | 59 ++++++++++++++++++- .../requirements.txt | 0 3 files changed, 57 insertions(+), 2 deletions(-) rename tests/ci/{approve_lambda => workflow_approve_rerun_lambda}/Dockerfile (100%) rename tests/ci/{approve_lambda => workflow_approve_rerun_lambda}/app.py (82%) rename tests/ci/{approve_lambda => workflow_approve_rerun_lambda}/requirements.txt (100%) diff --git a/tests/ci/approve_lambda/Dockerfile b/tests/ci/workflow_approve_rerun_lambda/Dockerfile similarity index 100% rename from tests/ci/approve_lambda/Dockerfile rename to tests/ci/workflow_approve_rerun_lambda/Dockerfile diff --git a/tests/ci/approve_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py similarity index 82% rename from tests/ci/approve_lambda/app.py rename to tests/ci/workflow_approve_rerun_lambda/app.py index 619c80ce299..436e9b06ede 100644 --- a/tests/ci/approve_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -25,8 +25,8 @@ SUSPICIOUS_PATTERNS = [ MAX_RETRY = 5 WorkflowDescription = namedtuple('WorkflowDescription', - ['name', 'action', 'run_id', 'event', 'workflow_id', - 'fork_owner_login', 'fork_branch']) + ['name', 'action', 'run_id', 'event', 'workflow_id', 'conclusion', 'status', + 'fork_owner_login', 'fork_branch', 'rerun_url', 'jobs_url', 'attempt', 'url']) TRUSTED_WORKFLOW_IDS = { 14586616, # Cancel workflows, always trusted @@ -38,6 +38,12 @@ TRUSTED_ORG_IDS = { 54801242, # clickhouse } +NEED_RERUN_WORKFLOWS = { + 13241696, # PR + 15834118, # Docs + 15522500, # MasterCI +} + # Individual trusted contirbutors who are not in any trusted organization. # Can be changed in runtime: we will append users that we learned to be in # a trusted org, to save GitHub API calls. @@ -180,6 +186,12 @@ def get_workflow_description_from_event(event): fork_branch = event['workflow_run']['head_branch'] name = event['workflow_run']['name'] workflow_id = event['workflow_run']['workflow_id'] + conclusion = event['workflow_run']['conclusion'] + attempt = event['workflow_run']['run_attempt'] + status = event['workflow_run']['status'] + jobs_url = event['workflow_run']['jobs_url'] + rerun_url = event['workflow_run']['rerun_url'] + url = event['workflow_run']['html_url'] return WorkflowDescription( name=name, action=action, @@ -188,6 +200,12 @@ def get_workflow_description_from_event(event): fork_owner_login=fork_owner, fork_branch=fork_branch, workflow_id=workflow_id, + conclusion=conclusion, + attempt=attempt, + status=status, + jobs_url=jobs_url, + rerun_url=rerun_url, + url=url ) def get_pr_author_and_orgs(pull_request): @@ -255,12 +273,49 @@ def get_token_from_aws(): installation_id = get_installation_id(encoded_jwt) return get_access_token(encoded_jwt, installation_id) +def check_need_to_rerun(workflow_description): + if workflow_description.attempt >= 2: + print("Not going to rerun workflow because it's already tried more than two times") + return False + print("Going to check jobs") + + jobs = _exec_get_with_retry(workflow_description.jobs_url + "?per_page=100") + print("Got jobs", len(jobs['jobs'])) + for job in jobs['jobs']: + if job['conclusion'] not in ('success', 'skipped'): + print("Job", job['name'], "failed, checking steps") + for step in job['steps']: + # always the last job + if step['name'] == 'Complete job': + print("Found Complete job step for job", job['name']) + break + else: + print("Checked all steps and doesn't found Complete job, going to rerun") + return True + + return False + +def rerun_workflow(workflow_description, token): + print("Going to rerun workflow") + _exec_post_with_retry(workflow_description.rerun_url, token) + def main(event): token = get_token_from_aws() event_data = json.loads(event['body']) workflow_description = get_workflow_description_from_event(event_data) print("Got workflow description", workflow_description) + if workflow_description.action == 'completed' and workflow_description.conclusion == 'failure': + print("Workflow", workflow_description.url, "completed and failed, let's check for rerun") + + if workflow_description.workflow_id not in NEED_RERUN_WORKFLOWS: + print("Workflow", workflow_description.workflow_id, "not in list of rerunable workflows") + return + + if check_need_to_rerun(workflow_description): + rerun_workflow(workflow_description, token) + return + if workflow_description.action != "requested": print("Exiting, event action is", workflow_description.action) return diff --git a/tests/ci/approve_lambda/requirements.txt b/tests/ci/workflow_approve_rerun_lambda/requirements.txt similarity index 100% rename from tests/ci/approve_lambda/requirements.txt rename to tests/ci/workflow_approve_rerun_lambda/requirements.txt From 952e975d160af5c4632520acbf8f56343e19ab87 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 17:32:00 +0300 Subject: [PATCH 601/609] Fix unit tests on master --- .github/workflows/master.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 26921b8ea48..d60c2889cc8 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1623,7 +1623,7 @@ jobs: env: TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (msan, actions)' + CHECK_NAME: 'Unit tests (ubsan, actions)' REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse run: | sudo rm -fr $TEMP_PATH From 1eb36c72364b3ae8c43dbbd2f014c1f151fdde84 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 17:48:03 +0300 Subject: [PATCH 602/609] Remove dependency between integration and functional tests --- .github/workflows/main.yml | 6 +++--- .github/workflows/master.yml | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 2adfbce3577..69a863b75a0 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -1498,7 +1498,7 @@ jobs: ############################# INTEGRATION TESTS ############################################# ############################################################################################# IntegrationTestsAsan: - needs: [BuilderDebAsan, FunctionalStatelessTestAsan] + needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports @@ -1526,7 +1526,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH IntegrationTestsTsan: - needs: [BuilderDebTsan, FunctionalStatelessTestTsan] + needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports @@ -1554,7 +1554,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH IntegrationTestsRelease: - needs: [BuilderDebRelease, FunctionalStatelessTestRelease] + needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index d60c2889cc8..5d4dec16303 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1268,7 +1268,7 @@ jobs: ############################# INTEGRATION TESTS ############################################# ############################################################################################# IntegrationTestsAsan: - needs: [BuilderDebAsan, FunctionalStatelessTestAsan] + needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports @@ -1296,7 +1296,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH IntegrationTestsTsan: - needs: [BuilderDebTsan, FunctionalStatelessTestTsan] + needs: [BuilderDebTsan] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports @@ -1324,7 +1324,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH IntegrationTestsRelease: - needs: [BuilderDebRelease, FunctionalStatelessTestRelease] + needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports From b74af1af5d3c66330369a0404f335ce8bacb9c3a Mon Sep 17 00:00:00 2001 From: Vxider Date: Fri, 10 Dec 2021 00:51:31 +0800 Subject: [PATCH 603/609] fix window view docs --- docs/en/sql-reference/statements/create/view.md | 2 +- .../sql-reference/functions/window-view-functions.md | 2 +- docs/zh/sql-reference/statements/create/view.md | 12 ++++++------ 3 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md index 464de02eac6..1e2e10b5cb6 100644 --- a/docs/en/sql-reference/statements/create/view.md +++ b/docs/en/sql-reference/statements/create/view.md @@ -296,7 +296,7 @@ CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTE Note that elements emitted by a late firing should be treated as updated results of a previous computation. Instead of firing at the end of windows, the window view will fire immediately when the late event arrives. Thus, it will result in multiple outputs for the same window. Users need to take these duplicated results into account or deduplicate them. -### Monitoring New Windows{#window-view-monitoring} +### Monitoring New Windows {#window-view-monitoring} Window view supports the `WATCH` query to constantly append the processing results to the console or use `TO` syntax to output the results to a table. diff --git a/docs/zh/sql-reference/functions/window-view-functions.md b/docs/zh/sql-reference/functions/window-view-functions.md index a8afac9a85d..b203fc41206 100644 --- a/docs/zh/sql-reference/functions/window-view-functions.md +++ b/docs/zh/sql-reference/functions/window-view-functions.md @@ -3,7 +3,7 @@ toc_priority: 68 toc_title: Window View --- -# Window View 函数{#window-view-han-shu} +# Window View 函数 {#window-view-han-shu} Window view函数用于获取窗口的起始(包含边界)和结束时间(不包含边界)。系统支持的window view函数如下: diff --git a/docs/zh/sql-reference/statements/create/view.md b/docs/zh/sql-reference/statements/create/view.md index ed64b578150..967db792038 100644 --- a/docs/zh/sql-reference/statements/create/view.md +++ b/docs/zh/sql-reference/statements/create/view.md @@ -257,11 +257,11 @@ Window view可以通过时间窗口聚合数据,并在满足窗口触发条件 创建window view的方式和创建物化视图类似。Window view使用默认为`AggregatingMergeTree`的内部存储引擎存储计算中间状态。 -### Window View 函数{#window-view-han-shu} +### Window View 函数 {#window-view-han-shu} [Window view函数](../../functions/window-view-functions.md)用于获取窗口的起始和结束时间。Window view需要和window view函数配合使用。 -### 时间属性{#window-view-shi-jian-shu-xing} +### 时间属性 {#window-view-shi-jian-shu-xing} Window view 支持**处理时间**和**事件时间**两种时间类型。 @@ -295,7 +295,7 @@ CREATE WINDOW VIEW test.wv TO test.dst WATERMARK=ASCENDING ALLOWED_LATENESS=INTE 需要注意的是,迟到消息需要更新之前的处理结果。与在窗口结束时触发不同,迟到消息到达时window view会立即触发计算。因此,会导致同一个窗口输出多次计算结果。用户需要注意这种情况,并消除重复结果。 -### 新窗口监控{#window-view-xin-chuang-kou-jian-kong} +### 新窗口监控 {#window-view-xin-chuang-kou-jian-kong} Window view可以通过`WATCH`语句将处理结果推送至终端,或通过`TO`语句将结果推送至数据表。 @@ -305,12 +305,12 @@ WATCH [db.]name [LIMIT n] `WATCH`语句和`LIVE VIEW`中的类似。支持设置`LIMIT`参数,输出消息数目达到`LIMIT`限制时结束查询。 -### 设置{#window-view-she-zhi} +### 设置 {#window-view-she-zhi} - `window_view_clean_interval`: window view清除过期数据间隔(单位为秒)。系统会定期清除过期数据,尚未触发的窗口数据不会被清除。 - `window_view_heartbeat_interval`: 用于判断watch查询活跃的心跳时间间隔。 -### 示例{#window-view-shi-li} +### 示例 {#window-view-shi-li} 假设我们需要每10秒统计一次`data`表中的点击日志,且`data`表的结构如下: @@ -352,7 +352,7 @@ CREATE WINDOW VIEW wv TO dst AS SELECT count(id), tumbleStart(w_id) as window_st ClickHouse测试中提供了更多的示例(以`*window_view*`命名)。 -### Window View 使用场景{#window-view-shi-yong-chang-jing} +### Window View 使用场景 {#window-view-shi-yong-chang-jing} Window view 在以下场景有用: From f0362d83016862e6b1c3ab27e381d12d8ef1b9a1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 9 Dec 2021 20:39:18 +0300 Subject: [PATCH 604/609] Update odbc-bridge.md --- docs/en/operations/utilities/odbc-bridge.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/utilities/odbc-bridge.md b/docs/en/operations/utilities/odbc-bridge.md index 70b413c9c1f..e5967085c49 100644 --- a/docs/en/operations/utilities/odbc-bridge.md +++ b/docs/en/operations/utilities/odbc-bridge.md @@ -26,7 +26,7 @@ Query is send in post body. Response is returned in RowBinary format. ```bash $ clickhouse-odbc-bridge --http-port 9018 --daemon -$ curl -d "query=SELECT PageID, ImpID, AdType FROM Keys ORDER BY PageID, ImpID" --data-urlencode "connection_string=DSN=ClickHouse;DATABASE=stat" --data-urlencode "columns=columns format version: 1 +$ curl -d "query=SELECT PageID, ImpID, AdType FROM Keys ORDER BY PageID, ImpID" --data-urlencode "connection_string=DSN=ClickHouse;DATABASE=stat" --data-urlencode "sample_block=columns format version: 1 3 columns: \`PageID\` String \`ImpID\` String From c155de58193deecdd3ac3496a58279b099e17f7b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 9 Dec 2021 19:00:08 +0100 Subject: [PATCH 605/609] 02122_parallel_formatting: Address grep binary warnings --- tests/queries/0_stateless/02122_parallel_formatting.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02122_parallel_formatting.sh b/tests/queries/0_stateless/02122_parallel_formatting.sh index 8061cbe58b2..f0c24344329 100755 --- a/tests/queries/0_stateless/02122_parallel_formatting.sh +++ b/tests/queries/0_stateless/02122_parallel_formatting.sh @@ -11,14 +11,14 @@ formats="RowBinary RowBinaryWithNames RowBinaryWithNamesAndTypes XML Markdown Ve for format in ${formats}; do echo $format-1 - $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -v "elapsed" > $non_parallel_file - $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -v "elapsed" > $parallel_file + $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file + $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) format $format" --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file echo $format-2 - $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -v "elapsed" > $non_parallel_file - $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -v "elapsed" > $parallel_file + $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=0 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $non_parallel_file + $CLICKHOUSE_CLIENT -q "select number, number + 1, concat('string: ', toString(number)) from numbers(200000) group by number with totals limit 190000 format $format" --extremes=1 --output_format_parallel_formatting=1 --output_format_pretty_max_rows=1000000 | grep -a -v "elapsed" > $parallel_file diff $non_parallel_file $parallel_file done From eee3fe6639b553b781ad8df77555a3270f920ac9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 23:17:19 +0300 Subject: [PATCH 606/609] Fix two unused builds --- .github/workflows/backport_branches.yml | 2 -- 1 file changed, 2 deletions(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 98a33927667..b61b74f86d3 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -207,8 +207,6 @@ jobs: - BuilderDebRelease - BuilderDebAsan - BuilderDebTsan - - BuilderDebUBsan - - BuilderDebMsan - BuilderDebDebug runs-on: [self-hosted, style-checker] steps: From 80dcaae0c079483d57925c8b0a654da1ca82075b Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 9 Dec 2021 23:18:13 +0300 Subject: [PATCH 607/609] Fix one more time --- .github/workflows/backport_branches.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index b61b74f86d3..859756f07af 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -331,7 +331,7 @@ jobs: ############################# INTEGRATION TESTS ############################################# ############################################################################################# IntegrationTestsRelease: - needs: [BuilderDebRelease, FunctionalStatelessTestRelease] + needs: [BuilderDebRelease] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports From 85e53b1b1fd0a407263e1317542ad5fe26398a6f Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 10 Dec 2021 00:12:45 +0300 Subject: [PATCH 608/609] Try fix attaching gdb in tests (#32448) * attach gdb with sudo * fix * Update run.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- docker/test/performance-comparison/compare.sh | 2 +- docker/test/stress/run.sh | 3 ++- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/stress_check.py | 2 +- 5 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 75d188e3190..764fa9a0f76 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -165,7 +165,7 @@ thread apply all backtrace continue " > script.gdb - gdb -batch -command script.gdb -p $server_pid & + sudo gdb -batch -command script.gdb -p $server_pid & # Check connectivity after we attach gdb, because it might cause the server # to freeze and the fuzzer will fail. diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index c32b50a3cbe..02d881347af 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -291,7 +291,7 @@ function get_profiles_watchdog for pid in $(pgrep -f clickhouse) do - gdb -p "$pid" --batch --ex "info proc all" --ex "thread apply all bt" --ex quit &> "$pid.gdb.log" & + sudo gdb -p "$pid" --batch --ex "info proc all" --ex "thread apply all bt" --ex quit &> "$pid.gdb.log" & done wait diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 6d720d02cdc..2ed4050d514 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -1,6 +1,7 @@ #!/bin/bash # shellcheck disable=SC2094 # shellcheck disable=SC2086 +# shellcheck disable=SC2024 set -x @@ -142,7 +143,7 @@ quit # FIXME Hung check may work incorrectly because of attached gdb # 1. False positives are possible # 2. We cannot attach another gdb to get stacktraces if some queries hung - gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & + sudo gdb -batch -command script.gdb -p "$(cat /var/run/clickhouse-server/clickhouse-server.pid)" >> /test_output/gdb.log & } configure diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 656e9fdbe50..bbf822c3879 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -21,7 +21,7 @@ IMAGE_NAME = 'clickhouse/fuzzer' def get_run_command(pr_number, sha, download_url, workspace_path, image): return f'docker run --network=host --volume={workspace_path}:/workspace ' \ - '--cap-add syslog --cap-add sys_admin ' \ + '--cap-add syslog --cap-add sys_admin --cap-add=SYS_PTRACE ' \ f'-e PR_TO_TEST={pr_number} -e SHA_TO_TEST={sha} -e BINARY_URL_TO_DOWNLOAD="{download_url}" '\ f'{image}' diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 7ec1a978cb9..911d370a594 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -22,7 +22,7 @@ from tee_popen import TeePopen def get_run_command(build_path, result_folder, server_log_folder, image): - cmd = "docker run -e S3_URL='https://clickhouse-datasets.s3.amazonaws.com' " + \ + cmd = "docker run --cap-add=SYS_PTRACE -e S3_URL='https://clickhouse-datasets.s3.amazonaws.com' " + \ f"--volume={build_path}:/package_folder " \ f"--volume={result_folder}:/test_output " \ f"--volume={server_log_folder}:/var/log/clickhouse-server {image}" From db83d82f5a301ece3e401d2972fe80d9d07b0907 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 10 Dec 2021 10:54:37 +0300 Subject: [PATCH 609/609] Fix integration tests path --- tests/ci/docker_pull_helper.py | 5 +++++ tests/ci/integration_test_check.py | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/tests/ci/docker_pull_helper.py b/tests/ci/docker_pull_helper.py index f9804744820..50354da6801 100644 --- a/tests/ci/docker_pull_helper.py +++ b/tests/ci/docker_pull_helper.py @@ -25,6 +25,11 @@ def get_images_with_versions(reports_path, required_image, pull=True): images_path = os.path.join(root, 'changed_images.json') break + if not images_path: + logging.info("Images file not found") + else: + logging.info("Images file path %s", images_path) + if images_path is not None and os.path.exists(images_path): logging.info("Images file exists") with open(images_path, 'r', encoding='utf-8') as images_fd: diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 723e81d63cb..69c4603b3ea 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -120,7 +120,7 @@ if __name__ == "__main__": logging.info("Check is already finished according to github status, exiting") sys.exit(0) - images = get_images_with_versions(temp_path, IMAGES) + images = get_images_with_versions(reports_path, IMAGES) images_with_versions = {i.name: i.version for i in images} result_path = os.path.join(temp_path, "output_dir") if not os.path.exists(result_path):